From ee1b3042848b0dc9c1362e4dce89a44bb7f8abbe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 20:56:10 +0100 Subject: [PATCH 001/856] Asynchronous load of tables by default --- programs/server/config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 52a1c528040..86fb7dfab1a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -371,7 +371,7 @@ - + true From 6db8b7a2d18ad495a469d71d3243964f0ef9e0f4 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Thu, 29 Feb 2024 20:35:18 +0800 Subject: [PATCH 002/856] Reduce unless virtual func call in ColumnNullable --- src/Columns/ColumnNullable.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 08b598d6a3b..17bc8ffe1e6 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -53,7 +53,7 @@ public: std::string getName() const override { return "Nullable(" + nested_column->getName() + ")"; } TypeIndex getDataType() const override { return TypeIndex::Nullable; } MutableColumnPtr cloneResized(size_t size) const override; - size_t size() const override { return nested_column->size(); } + size_t size() const override { return assert_cast(*null_map).size(); } bool isNullAt(size_t n) const override { return assert_cast(*null_map).getData()[n] != 0;} Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; From 545cc731dcc1c33a3e016b68d47450aaa5e388bd Mon Sep 17 00:00:00 2001 From: Philipp Schreiber Date: Fri, 1 Mar 2024 09:39:02 +0100 Subject: [PATCH 003/856] fix column ddl expression order documentation --- docs/en/sql-reference/statements/create/table.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 0edf158e981..2967a11d35c 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -17,8 +17,8 @@ By default, tables are created only on the current server. Distributed DDL queri ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( - name1 [type1] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr1] [compression_codec] [TTL expr1] [COMMENT 'comment for column'], - name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [compression_codec] [TTL expr2] [COMMENT 'comment for column'], + name1 [type1] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr1] [COMMENT 'comment for column'] [compression_codec] [TTL expr1], + name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [COMMENT 'comment for column'] [compression_codec] [TTL expr2], ... ) ENGINE = engine COMMENT 'comment for table' From 7cbbb1601b7d4418d061179968f89c25e0ee3b75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Mar 2024 22:39:29 +0100 Subject: [PATCH 004/856] Update test --- .../0_stateless/01676_dictget_in_default_expression.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql index 54e46a2b718..1785979f60b 100644 --- a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql +++ b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql @@ -22,7 +22,8 @@ DETACH DATABASE test_01676; ATTACH DATABASE test_01676; SELECT 'status_after_detach_and_attach:'; -SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict'; +-- It can be not loaded, or not even finish attaching in case of asynchronous tables loading. +SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict')::String, 'NOT_LOADED'); INSERT INTO test_01676.table (x) VALUES (toInt64(4)); SELECT * FROM test_01676.table ORDER BY x; From 3c1e6b5d2c52d8f54705428199d007a0f58857bb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 7 Mar 2024 13:11:42 +0000 Subject: [PATCH 005/856] 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 006/856] 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 007/856] 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 008/856] 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 009/856] 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 010/856] 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 011/856] 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 012/856] 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 013/856] 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 014/856] 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 015/856] 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 016/856] 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 017/856] 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 018/856] 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 019/856] 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 020/856] 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 021/856] 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 022/856] 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 023/856] 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 024/856] 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 025/856] 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 026/856] 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 027/856] 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 028/856] 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 d52d13da22698239c6fed00fe9e4ec50e5661cb6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 22 Apr 2024 09:38:33 -0300 Subject: [PATCH 029/856] Tmp --- .../EnvironmentProxyConfigurationResolver.cpp | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index f2c60afa1a8..6a925f4ac00 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -1,4 +1,5 @@ #include "EnvironmentProxyConfigurationResolver.h" +#include #include #include @@ -34,6 +35,34 @@ namespace return std::getenv(PROXY_HTTPS_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) } } + + std::vector getNoProxyHosts() + { + std::vector result; + + const char * no_proxy = std::getenv("NO_PROXY"); // NOLINT(concurrency-mt-unsafe) + if (!no_proxy) + { + return result; + } + + std::string no_proxy_str(no_proxy); + std::istringstream no_proxy_stream(no_proxy_str); + std::string host; + while (std::getline(no_proxy_stream, host, ',')) + { + try + { + result.emplace(host); + } + catch (const Poco::SyntaxException & e) + { + LOG_WARNING(getLogger("EnvironmentProxyConfigurationResolver"), "Failed to parse NO_PROXY host '{}': {}", host, e.displayText()); + } + } + + return result; + } } ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() 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 030/856] empty commit From 0065a4cc6e8ac7eff3e72765f5ae0a7eb593ed2d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 24 Apr 2024 14:55:01 +0000 Subject: [PATCH 031/856] 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 032/856] 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 033/856] 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 034/856] 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 035/856] 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 036/856] empty commit From 0bc664ac5bd0104219e061660cb6cd1cb0698b7c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 29 Apr 2024 18:21:19 +0000 Subject: [PATCH 037/856] 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 7aaf7e0e03d1d1e192a68277c4f6836ce92587e4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 10:44:28 +0200 Subject: [PATCH 038/856] Fix 02488_zero_copy_detached_parts_drop_table test --- .../0_stateless/02488_zero_copy_detached_parts_drop_table.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh index b01f16e1cad..60cec5caea3 100755 --- a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh +++ b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is rm -f $path/count.txt $CLICKHOUSE_CLIENT -q "detach table rmt2 sync" -$CLICKHOUSE_CLIENT --send_logs_level='fatal' -q "attach table rmt2" +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level='fatal' -q "attach table rmt2" $CLICKHOUSE_CLIENT -q "select reason, name from system.detached_parts where database='$CLICKHOUSE_DATABASE' and table='rmt2'" From 6556bd28dcb7912add1c192e318f4e9dd0f9cd31 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 15:23:36 +0200 Subject: [PATCH 039/856] Fix parallel write for Azure --- src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f52ab803012..cd154ec0f79 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -163,6 +163,8 @@ public: return client.get(); } + bool supportParallelWrite() const override { return true; } + private: using SharedAzureClientPtr = std::shared_ptr; void removeObjectImpl(const StoredObject & object, const SharedAzureClientPtr & client_ptr, bool if_exists); From 697b5556a3e16e8b0cbba43d5226a2003ef21f0a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 22:10:36 +0200 Subject: [PATCH 040/856] Enable azure test --- tests/ci/functional_test_check.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index e898138fb3a..6418ee4e0ce 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -318,9 +318,6 @@ def main(): state, description, test_results, additional_logs = process_results( result_path, server_log_path ) - # FIXME (alesapin) - if "azure" in check_name: - state = "success" else: print( "This is validate bugfix or flaky check run, but no changes test to run - skip with success" From ccabc292ad6647855d5b99c1631b8ee77413b855 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 1 May 2024 08:48:24 +0200 Subject: [PATCH 041/856] Temporarily remove running azure tests only for master --- tests/ci/ci_config.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index bdfff12db0b..f7b32e9177c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1205,7 +1205,7 @@ CI_CONFIG = CIConfig( ), JobNames.STATELESS_TEST_AZURE_ASAN: TestConfig( Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **statless_test_common_params, release_only=True), # type: ignore + job_config=JobConfig(num_batches=4, **statless_test_common_params), # type: ignore ), JobNames.STATELESS_TEST_S3_TSAN: TestConfig( Build.PACKAGE_TSAN, @@ -1230,7 +1230,7 @@ CI_CONFIG = CIConfig( Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore ), JobNames.STRESS_TEST_AZURE_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params, release_only=True) # type: ignore + Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params) # type: ignore ), JobNames.UPGRADE_TEST_TSAN: TestConfig( Build.PACKAGE_TSAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore From 068ddc2d65f2ba4bce98aaafe134a94f843d5974 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 1 May 2024 11:03:52 +0200 Subject: [PATCH 042/856] Add allow_repeated_settings before appending send_logs_level in tests --- .../0_stateless/02494_query_cache_nested_query_bug.sh | 4 ++-- .../0_stateless/02535_max_parallel_replicas_custom_key.sh | 2 +- .../0_stateless/02922_deduplication_with_zero_copy.sh | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index 8712c7c84c6..59b107d06fe 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -19,7 +19,7 @@ SETTINGS="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 9850406eb3a..480517bf17f 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --allow_repeated_settings --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" 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 bb013dccb65..dad4e6747e0 100755 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh @@ -58,9 +58,9 @@ function filter_temporary_locks() function insert_duplicates() { - $CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --send_logs_level="error" & + $CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --allow_repeated_settings --send_logs_level="error" & - $CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --send_logs_level="error" + $CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --allow_repeated_settings --send_logs_level="error" wait @@ -137,8 +137,8 @@ function list_keeper_nodes() { list_keeper_nodes "${table_shared_id}" -$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --send_logs_level="error" & -$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --allow_repeated_settings --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --allow_repeated_settings --send_logs_level="error" & wait list_keeper_nodes "${table_shared_id}" From d5ff9de6fa995cf4c0ddd14799b0367d1d3db592 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 May 2024 17:29:23 -0300 Subject: [PATCH 043/856] env resolver impl --- .../EnvironmentProxyConfigurationResolver.cpp | 26 ++++++++++--------- src/Common/ProxyConfiguration.h | 1 + src/Common/tests/gtest_helper_functions.h | 10 ++++++- .../gtest_proxy_environment_configuration.cpp | 5 +++- 4 files changed, 28 insertions(+), 14 deletions(-) diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index 6a925f4ac00..bc46cd39bd6 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -1,7 +1,9 @@ #include "EnvironmentProxyConfigurationResolver.h" #include +#include #include +#include #include namespace DB @@ -13,6 +15,7 @@ namespace DB * */ static constexpr auto PROXY_HTTP_ENVIRONMENT_VARIABLE = "http_proxy"; static constexpr auto PROXY_HTTPS_ENVIRONMENT_VARIABLE = "https_proxy"; +static constexpr auto NO_PROXY_ENVIRONMENT_VARIABLE = "no_proxy"; EnvironmentProxyConfigurationResolver::EnvironmentProxyConfigurationResolver( Protocol request_protocol_, bool disable_tunneling_for_https_requests_over_http_proxy_) @@ -36,28 +39,26 @@ namespace } } - std::vector getNoProxyHosts() + std::vector getNoProxyHosts() { - std::vector result; + std::vector result; + + const char * no_proxy = std::getenv(NO_PROXY_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) - const char * no_proxy = std::getenv("NO_PROXY"); // NOLINT(concurrency-mt-unsafe) if (!no_proxy) { return result; } - std::string no_proxy_str(no_proxy); - std::istringstream no_proxy_stream(no_proxy_str); + std::istringstream no_proxy_stream(no_proxy); std::string host; while (std::getline(no_proxy_stream, host, ',')) { - try + trim(host); + + if (!host.empty()) { - result.emplace(host); - } - catch (const Poco::SyntaxException & e) - { - LOG_WARNING(getLogger("EnvironmentProxyConfigurationResolver"), "Failed to parse NO_PROXY host '{}': {}", host, e.displayText()); + result.emplace_back(host); } } @@ -86,7 +87,8 @@ ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() ProxyConfiguration::protocolFromString(scheme), port, useTunneling(request_protocol, ProxyConfiguration::protocolFromString(scheme), disable_tunneling_for_https_requests_over_http_proxy), - request_protocol + request_protocol, + getNoProxyHosts() }; } diff --git a/src/Common/ProxyConfiguration.h b/src/Common/ProxyConfiguration.h index 289ae2b6813..be0750f1f82 100644 --- a/src/Common/ProxyConfiguration.h +++ b/src/Common/ProxyConfiguration.h @@ -49,6 +49,7 @@ struct ProxyConfiguration uint16_t port = 0; bool tunneling = false; Protocol original_request_protocol = Protocol::HTTP; + std::vector no_proxy_hosts = {}; bool isEmpty() const { return host.empty(); } }; diff --git a/src/Common/tests/gtest_helper_functions.h b/src/Common/tests/gtest_helper_functions.h index 54c9ae9170d..e3aeea407a1 100644 --- a/src/Common/tests/gtest_helper_functions.h +++ b/src/Common/tests/gtest_helper_functions.h @@ -76,7 +76,10 @@ inline std::string xmlNodeAsString(Poco::XML::Node *pNode) struct EnvironmentProxySetter { - EnvironmentProxySetter(const Poco::URI & http_proxy, const Poco::URI & https_proxy) + EnvironmentProxySetter( + const Poco::URI & http_proxy, + const Poco::URI & https_proxy, + const std::string & no_proxy = {}) { if (!http_proxy.empty()) { @@ -87,6 +90,11 @@ struct EnvironmentProxySetter { setenv("https_proxy", https_proxy.toString().c_str(), 1); // NOLINT(concurrency-mt-unsafe) } + + if (!no_proxy.empty()) + { + setenv("no_proxy", no_proxy.c_str(), 1); // NOLINT(concurrency-mt-unsafe) + } } ~EnvironmentProxySetter() diff --git a/src/Common/tests/gtest_proxy_environment_configuration.cpp b/src/Common/tests/gtest_proxy_environment_configuration.cpp index 377bef385f6..4956604f6d8 100644 --- a/src/Common/tests/gtest_proxy_environment_configuration.cpp +++ b/src/Common/tests/gtest_proxy_environment_configuration.cpp @@ -15,7 +15,8 @@ namespace TEST(EnvironmentProxyConfigurationResolver, TestHTTP) { - EnvironmentProxySetter setter(http_proxy_server, {}); + std::vector no_proxy_hosts = {"localhost", "127.0.0.1", "some_other_domain", "last_domain"}; + EnvironmentProxySetter setter(http_proxy_server, {}, "localhost,,127.0.0.1,some_other_domain,,,, last_domain,"); EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTP); @@ -24,6 +25,7 @@ TEST(EnvironmentProxyConfigurationResolver, TestHTTP) ASSERT_EQ(configuration.host, http_proxy_server.getHost()); ASSERT_EQ(configuration.port, http_proxy_server.getPort()); ASSERT_EQ(configuration.protocol, ProxyConfiguration::protocolFromString(http_proxy_server.getScheme())); + ASSERT_EQ(configuration.no_proxy_hosts, no_proxy_hosts); } TEST(EnvironmentProxyConfigurationResolver, TestHTTPNoEnv) @@ -35,6 +37,7 @@ TEST(EnvironmentProxyConfigurationResolver, TestHTTPNoEnv) ASSERT_EQ(configuration.host, ""); ASSERT_EQ(configuration.protocol, ProxyConfiguration::Protocol::HTTP); ASSERT_EQ(configuration.port, 0u); + ASSERT_TRUE(configuration.no_proxy_hosts.empty()); } TEST(EnvironmentProxyConfigurationResolver, TestHTTPs) From d30aef4b547cd67acdb1a916b77bfbc2a890123d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 May 2024 18:03:32 -0300 Subject: [PATCH 044/856] pass it to poco http --- src/Common/HTTPConnectionPool.cpp | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 167aeee68f3..f559fdc786e 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -70,6 +70,24 @@ namespace CurrentMetrics namespace { + std::string buildPocoNonProxyHosts(const DB::ProxyConfiguration & proxy_configuration) + { + bool first = true; + std::string ret; + + for (const auto & host : proxy_configuration.no_proxy_hosts) + { + if (!first) + { + ret.append("|"); + } + ret.append(host); + first = false; + } + + return ret; + } + Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration) { Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; @@ -79,6 +97,7 @@ namespace poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol); poco_proxy_config.tunnel = proxy_configuration.tunneling; poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol); + poco_proxy_config.nonProxyHosts = buildPocoNonProxyHosts(proxy_configuration); return poco_proxy_config; } @@ -696,7 +715,8 @@ struct EndpointPoolKey proxy_config.port, proxy_config.protocol, proxy_config.tunneling, - proxy_config.original_request_protocol) + proxy_config.original_request_protocol, + proxy_config.no_proxy_hosts) == std::tie( rhs.connection_group, rhs.target_host, @@ -706,7 +726,8 @@ struct EndpointPoolKey rhs.proxy_config.port, rhs.proxy_config.protocol, rhs.proxy_config.tunneling, - rhs.proxy_config.original_request_protocol); + rhs.proxy_config.original_request_protocol, + rhs.proxy_config.no_proxy_hosts); } }; From f4da4f1eb0fe9a5034367d46f3c12710f6a8de7d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 May 2024 20:29:05 -0300 Subject: [PATCH 045/856] env no proxy tests --- .../helpers/s3_url_proxy_tests_util.py | 22 ++++++++++++++----- .../test.py | 16 ++++++++++++++ 2 files changed, 33 insertions(+), 5 deletions(-) diff --git a/tests/integration/helpers/s3_url_proxy_tests_util.py b/tests/integration/helpers/s3_url_proxy_tests_util.py index 9059fda08ae..16df446b0f7 100644 --- a/tests/integration/helpers/s3_url_proxy_tests_util.py +++ b/tests/integration/helpers/s3_url_proxy_tests_util.py @@ -2,8 +2,8 @@ import os import time -def check_proxy_logs( - cluster, proxy_instance, protocol, bucket, http_methods={"POST", "PUT", "GET"} +def has_any_proxy_related_logs( +cluster, proxy_instance, protocol, bucket, http_methods={"POST", "PUT", "GET"} ): for i in range(10): logs = cluster.get_container_logs(proxy_instance) @@ -13,10 +13,10 @@ def check_proxy_logs( logs.find(http_method + f" {protocol}://minio1:9001/root/data/{bucket}") >= 0 ): - return + return True time.sleep(1) else: - assert False, f"{http_methods} method not found in logs of {proxy_instance}" + return False def wait_resolver(cluster): @@ -85,4 +85,16 @@ def simple_test(cluster, proxies, protocol, bucket): perform_simple_queries(node, minio_endpoint) for proxy in proxies: - check_proxy_logs(cluster, proxy, protocol, bucket) + has_proxy_logs = has_any_proxy_related_logs(cluster, proxy, protocol, bucket) + assert has_proxy_logs, f"Did not find any proxy related logs in {proxy}" + + +def simple_test_assert_no_proxy(cluster, proxies, protocol, bucket): + minio_endpoint = build_s3_endpoint(protocol, bucket) + node = cluster.instances[f"{bucket}"] + + perform_simple_queries(node, minio_endpoint) + + for proxy in proxies: + no_proxy_logs = not has_any_proxy_related_logs(cluster, proxy, protocol, bucket) + assert no_proxy_logs, f"Found proxy logs in {proxy} and it should not have found it" diff --git a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py index ae872a33cd4..fefc98ea1f7 100644 --- a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py +++ b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py @@ -39,6 +39,19 @@ def cluster(): instance_env_variables=True, ) + cluster.add_instance( + "env_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_env.xml", + ], + with_minio=True, + env_variables={ + "https_proxy": "http://proxy1", + "no_proxy": "not_important_host,, minio1 ," + }, + instance_env_variables=True, + ) + logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -61,3 +74,6 @@ def test_s3_with_https_remote_proxy(cluster): def test_s3_with_https_env_proxy(cluster): proxy_util.simple_test(cluster, ["proxy1"], "https", "env_node") + +def test_s3_with_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "env_node_no_proxy") \ No newline at end of file From 326938086955ae4931f7b19abbedf22fc91b9afa Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 May 2024 20:35:21 -0300 Subject: [PATCH 046/856] update tests --- .../test.py | 4 ---- .../test.py | 14 ++++++++++++++ .../test.py | 17 +++++++++++++++++ 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py index fefc98ea1f7..00f88cf4c14 100644 --- a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py +++ b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py @@ -47,7 +47,6 @@ def cluster(): with_minio=True, env_variables={ "https_proxy": "http://proxy1", - "no_proxy": "not_important_host,, minio1 ," }, instance_env_variables=True, ) @@ -74,6 +73,3 @@ def test_s3_with_https_remote_proxy(cluster): def test_s3_with_https_env_proxy(cluster): proxy_util.simple_test(cluster, ["proxy1"], "https", "env_node") - -def test_s3_with_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "env_node_no_proxy") \ No newline at end of file diff --git a/tests/integration/test_s3_table_function_with_http_proxy/test.py b/tests/integration/test_s3_table_function_with_http_proxy/test.py index 1619b413099..cc0d59e51ba 100644 --- a/tests/integration/test_s3_table_function_with_http_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_http_proxy/test.py @@ -36,6 +36,16 @@ def cluster(): instance_env_variables=True, ) + cluster.add_instance( + "env_node_no_proxy", + with_minio=True, + env_variables={ + "http_proxy": "http://proxy1", + "no_proxy": "not_important_host,, minio1 ,", + }, + instance_env_variables=True, + ) + logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -58,3 +68,7 @@ def test_s3_with_http_remote_proxy(cluster): def test_s3_with_http_env_proxy(cluster): proxy_util.simple_test(cluster, ["proxy1"], "http", "env_node") + + +def test_s3_with_http_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "http", "env_node_no_proxy") diff --git a/tests/integration/test_s3_table_function_with_https_proxy/test.py b/tests/integration/test_s3_table_function_with_https_proxy/test.py index 83af407093c..87285415d85 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_https_proxy/test.py @@ -44,6 +44,19 @@ def cluster(): instance_env_variables=True, ) + cluster.add_instance( + "env_node_no_proxy", + main_configs=[ + "configs/config.d/ssl.xml", + ], + with_minio=True, + env_variables={ + "https_proxy": "https://proxy1", + "no_proxy": "not_important_host,, minio1 ,", + }, + instance_env_variables=True, + ) + logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -66,3 +79,7 @@ def test_s3_with_https_remote_proxy(cluster): def test_s3_with_https_env_proxy(cluster): proxy_util.simple_test(cluster, ["proxy1"], "https", "env_node") + + +def test_s3_with_https_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "env_node_no_proxy") From f2083a979b0baeff33108507b188863d087f15a7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 May 2024 20:43:14 -0300 Subject: [PATCH 047/856] some comments --- src/Common/HTTPConnectionPool.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index f559fdc786e..a17ed770f19 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -70,6 +70,11 @@ namespace CurrentMetrics namespace { + /* + * ClickHouse holds a list of hosts, while Poco expects a regex. Build an or-based regex with all the hosts + * Favoring simplicity. https://about.gitlab.com/blog/2021/01/27/we-need-to-talk-no-proxy/ + * Open for discussions + * */ std::string buildPocoNonProxyHosts(const DB::ProxyConfiguration & proxy_configuration) { bool first = true; From 1b577a81b97c1f6f896e527598be9d8865dee105 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 3 May 2024 11:48:50 -0300 Subject: [PATCH 048/856] keep no_proxy string as is in clickhouse memory, convert to or-based regex with match anything wildcard for poco --- .../EnvironmentProxyConfigurationResolver.cpp | 22 +---- src/Common/HTTPConnectionPool.cpp | 39 +-------- src/Common/ProxyConfiguration.h | 2 +- .../proxyConfigurationToPocoProxyConfig.cpp | 80 +++++++++++++++++++ .../proxyConfigurationToPocoProxyConfig.h | 11 +++ ...oxy_configuration_to_poco_proxy_config.cpp | 25 ++++++ .../gtest_proxy_environment_configuration.cpp | 6 +- 7 files changed, 125 insertions(+), 60 deletions(-) create mode 100644 src/Common/proxyConfigurationToPocoProxyConfig.cpp create mode 100644 src/Common/proxyConfigurationToPocoProxyConfig.h create mode 100644 src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index bc46cd39bd6..3abaa4ec9e3 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -39,30 +39,16 @@ namespace } } - std::vector getNoProxyHosts() + std::string getNoProxyHostsString() { - std::vector result; - const char * no_proxy = std::getenv(NO_PROXY_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) if (!no_proxy) { - return result; + return ""; } - std::istringstream no_proxy_stream(no_proxy); - std::string host; - while (std::getline(no_proxy_stream, host, ',')) - { - trim(host); - - if (!host.empty()) - { - result.emplace_back(host); - } - } - - return result; + return no_proxy; } } @@ -88,7 +74,7 @@ ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() port, useTunneling(request_protocol, ProxyConfiguration::protocolFromString(scheme), disable_tunneling_for_https_requests_over_http_proxy), request_protocol, - getNoProxyHosts() + getNoProxyHostsString() }; } diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index a17ed770f19..f3ff09bc90a 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -70,44 +71,6 @@ namespace CurrentMetrics namespace { - /* - * ClickHouse holds a list of hosts, while Poco expects a regex. Build an or-based regex with all the hosts - * Favoring simplicity. https://about.gitlab.com/blog/2021/01/27/we-need-to-talk-no-proxy/ - * Open for discussions - * */ - std::string buildPocoNonProxyHosts(const DB::ProxyConfiguration & proxy_configuration) - { - bool first = true; - std::string ret; - - for (const auto & host : proxy_configuration.no_proxy_hosts) - { - if (!first) - { - ret.append("|"); - } - ret.append(host); - first = false; - } - - return ret; - } - - Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration) - { - Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; - - poco_proxy_config.host = proxy_configuration.host; - poco_proxy_config.port = proxy_configuration.port; - poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol); - poco_proxy_config.tunnel = proxy_configuration.tunneling; - poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol); - poco_proxy_config.nonProxyHosts = buildPocoNonProxyHosts(proxy_configuration); - - return poco_proxy_config; - } - - constexpr size_t roundUp(size_t x, size_t rounding) { chassert(rounding > 0); diff --git a/src/Common/ProxyConfiguration.h b/src/Common/ProxyConfiguration.h index be0750f1f82..78264d66cb7 100644 --- a/src/Common/ProxyConfiguration.h +++ b/src/Common/ProxyConfiguration.h @@ -49,7 +49,7 @@ struct ProxyConfiguration uint16_t port = 0; bool tunneling = false; Protocol original_request_protocol = Protocol::HTTP; - std::vector no_proxy_hosts = {}; + std::string no_proxy_hosts = ""; bool isEmpty() const { return host.empty(); } }; diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.cpp b/src/Common/proxyConfigurationToPocoProxyConfig.cpp new file mode 100644 index 00000000000..ec2a5ba19bd --- /dev/null +++ b/src/Common/proxyConfigurationToPocoProxyConfig.cpp @@ -0,0 +1,80 @@ +#include + +#include + +#include + +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" +#pragma clang diagnostic ignored "-Wgnu-anonymous-struct" +#pragma clang diagnostic ignored "-Wnested-anon-types" +#pragma clang diagnostic ignored "-Wunused-parameter" +#pragma clang diagnostic ignored "-Wshadow-field-in-constructor" +#pragma clang diagnostic ignored "-Wdtor-name" +#include +#pragma clang diagnostic pop + +namespace DB +{ + +/* + * Even though there is not an RFC that defines NO_PROXY, it is usually a comma-separated list of domains. + * Different tools implement their own versions of `NO_PROXY` support. Some support CIDR blocks, some support wildcard etc. + * Opting for a simple implementation that covers most use cases: + * * Support only single wildcard * (match anything) + * * No regex + * * No CIDR blocks + * * No leading dot strip + * * No fancy stuff about loopback IPs + * https://about.gitlab.com/blog/2021/01/27/we-need-to-talk-no-proxy/ + * Open for discussions + * */ +std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts) +{ + bool match_any_host = no_proxy_hosts.size() == 1 && no_proxy_hosts[0] == '*'; + + if (match_any_host) + { + return "(.*?)"; + } + + std::string host; + std::istringstream no_proxy_stream(no_proxy_hosts); + + bool first = true; + std::string result; + + while (std::getline(no_proxy_stream, host, ',')) + { + trim(host); + + if (!host.empty()) + { + if (!first) + { + result.append("|"); + } + + result.append(RE2::QuoteMeta(host)); + first = false; + } + } + + return result; +} + +Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration) +{ + Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; + + poco_proxy_config.host = proxy_configuration.host; + poco_proxy_config.port = proxy_configuration.port; + poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol); + poco_proxy_config.tunnel = proxy_configuration.tunneling; + poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol); + poco_proxy_config.nonProxyHosts = buildPocoNonProxyHosts(proxy_configuration.no_proxy_hosts); + + return poco_proxy_config; +} + +} diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.h b/src/Common/proxyConfigurationToPocoProxyConfig.h new file mode 100644 index 00000000000..d093b0f3521 --- /dev/null +++ b/src/Common/proxyConfigurationToPocoProxyConfig.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration); + +} diff --git a/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp b/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp new file mode 100644 index 00000000000..d161bb9f63e --- /dev/null +++ b/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp @@ -0,0 +1,25 @@ +#include + +#include + +TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuild) +{ + DB::ProxyConfiguration proxy_configuration; + + proxy_configuration.no_proxy_hosts = "localhost,127.0.0.1,some_other_domain:8080,sub-domain.domain.com"; + + auto poco_proxy_configuration = DB::proxyConfigurationToPocoProxyConfig(proxy_configuration); + + ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, "localhost|127\\.0\\.0\\.1|some_other_domain\\:8080|sub\\-domain\\.domain\\.com"); +} + +TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildMatchAnything) +{ + DB::ProxyConfiguration proxy_configuration; + + proxy_configuration.no_proxy_hosts = "*"; + + auto poco_proxy_configuration = DB::proxyConfigurationToPocoProxyConfig(proxy_configuration); + + ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, "(.*?)"); +} diff --git a/src/Common/tests/gtest_proxy_environment_configuration.cpp b/src/Common/tests/gtest_proxy_environment_configuration.cpp index 4956604f6d8..81388fd877f 100644 --- a/src/Common/tests/gtest_proxy_environment_configuration.cpp +++ b/src/Common/tests/gtest_proxy_environment_configuration.cpp @@ -15,8 +15,8 @@ namespace TEST(EnvironmentProxyConfigurationResolver, TestHTTP) { - std::vector no_proxy_hosts = {"localhost", "127.0.0.1", "some_other_domain", "last_domain"}; - EnvironmentProxySetter setter(http_proxy_server, {}, "localhost,,127.0.0.1,some_other_domain,,,, last_domain,"); + std::string no_proxy_string = "localhost,,127.0.0.1,some_other_domain,,,, last_domain,"; + EnvironmentProxySetter setter(http_proxy_server, {}, no_proxy_string); EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTP); @@ -25,7 +25,7 @@ TEST(EnvironmentProxyConfigurationResolver, TestHTTP) ASSERT_EQ(configuration.host, http_proxy_server.getHost()); ASSERT_EQ(configuration.port, http_proxy_server.getPort()); ASSERT_EQ(configuration.protocol, ProxyConfiguration::protocolFromString(http_proxy_server.getScheme())); - ASSERT_EQ(configuration.no_proxy_hosts, no_proxy_hosts); + ASSERT_EQ(configuration.no_proxy_hosts, no_proxy_string); } TEST(EnvironmentProxyConfigurationResolver, TestHTTPNoEnv) From a157d6abd61f27aa3ba35f22fcfd83e069d936f6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 3 May 2024 11:49:12 -0300 Subject: [PATCH 049/856] remove unused node --- .../test.py | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py index 00f88cf4c14..ae872a33cd4 100644 --- a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py +++ b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py @@ -39,18 +39,6 @@ def cluster(): instance_env_variables=True, ) - cluster.add_instance( - "env_node_no_proxy", - main_configs=[ - "configs/config.d/proxy_env.xml", - ], - with_minio=True, - env_variables={ - "https_proxy": "http://proxy1", - }, - instance_env_variables=True, - ) - logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") From 4103a730e2b3f644cb69c9b6e423a6a9a54c9dec Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 3 May 2024 13:51:02 -0300 Subject: [PATCH 050/856] add empty test --- src/Common/EnvironmentProxyConfigurationResolver.cpp | 3 --- ...gtest_proxy_configuration_to_poco_proxy_config.cpp | 11 +++++++++++ 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index 3abaa4ec9e3..387674feaae 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -1,9 +1,6 @@ #include "EnvironmentProxyConfigurationResolver.h" -#include -#include #include -#include #include namespace DB diff --git a/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp b/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp index d161bb9f63e..4a8488d4055 100644 --- a/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp +++ b/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp @@ -23,3 +23,14 @@ TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildMatchA ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, "(.*?)"); } + +TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildEmpty) +{ + DB::ProxyConfiguration proxy_configuration; + + proxy_configuration.no_proxy_hosts = ""; + + auto poco_proxy_configuration = DB::proxyConfigurationToPocoProxyConfig(proxy_configuration); + + ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, ""); +} From c97f5cf3b7b33a06b86fa18049f069df46238860 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 3 May 2024 14:33:48 -0300 Subject: [PATCH 051/856] more basic tests --- .../ProxyConfigurationResolverProvider.cpp | 14 ++++++++++++-- src/Common/ProxyListConfigurationResolver.cpp | 10 +++++++--- src/Common/ProxyListConfigurationResolver.h | 7 ++++++- src/Common/RemoteProxyConfigurationResolver.cpp | 7 +++++-- src/Common/RemoteProxyConfigurationResolver.h | 4 +++- ...est_proxy_configuration_resolver_provider.cpp | 16 +++++++++++++++- 6 files changed, 48 insertions(+), 10 deletions(-) diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index d15b4d98615..360db80e360 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -17,6 +17,11 @@ namespace ErrorCodes namespace { + std::string getNoProxyHosts(const Poco::Util::AbstractConfiguration & configuration) + { + return configuration.getString("proxy.no_proxy", ""); + } + bool isTunnelingDisabledForHTTPSRequestsOverHTTPProxy( const Poco::Util::AbstractConfiguration & configuration) { @@ -49,7 +54,8 @@ namespace return std::make_shared( server_configuration, request_protocol, - isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); + isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration), + getNoProxyHosts(configuration)); } auto extractURIList(const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) @@ -87,7 +93,11 @@ namespace return uris.empty() ? nullptr - : std::make_shared(uris, request_protocol, isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); + : std::make_shared( + uris, + request_protocol, + isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration), + getNoProxyHosts(configuration)); } bool hasRemoteResolver(const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) diff --git a/src/Common/ProxyListConfigurationResolver.cpp b/src/Common/ProxyListConfigurationResolver.cpp index c9b8923929a..2560b3b8d1d 100644 --- a/src/Common/ProxyListConfigurationResolver.cpp +++ b/src/Common/ProxyListConfigurationResolver.cpp @@ -9,8 +9,11 @@ namespace DB ProxyListConfigurationResolver::ProxyListConfigurationResolver( std::vector proxies_, - Protocol request_protocol_, bool disable_tunneling_for_https_requests_over_http_proxy_) - : ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_), proxies(std::move(proxies_)) + Protocol request_protocol_, + bool disable_tunneling_for_https_requests_over_http_proxy_, + const std::string & no_proxy_hosts_) + : ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_), + proxies(std::move(proxies_)), no_proxy_hosts(no_proxy_hosts_) { } @@ -31,7 +34,8 @@ ProxyConfiguration ProxyListConfigurationResolver::resolve() ProxyConfiguration::protocolFromString(proxy.getScheme()), proxy.getPort(), useTunneling(request_protocol, ProxyConfiguration::protocolFromString(proxy.getScheme()), disable_tunneling_for_https_requests_over_http_proxy), - request_protocol + request_protocol, + no_proxy_hosts }; } diff --git a/src/Common/ProxyListConfigurationResolver.h b/src/Common/ProxyListConfigurationResolver.h index 95e0073d779..db69e7eb6c9 100644 --- a/src/Common/ProxyListConfigurationResolver.h +++ b/src/Common/ProxyListConfigurationResolver.h @@ -15,7 +15,11 @@ namespace DB class ProxyListConfigurationResolver : public ProxyConfigurationResolver { public: - ProxyListConfigurationResolver(std::vector proxies_, Protocol request_protocol_, bool disable_tunneling_for_https_requests_over_http_proxy_ = false); + ProxyListConfigurationResolver( + std::vector proxies_, + Protocol request_protocol_, + bool disable_tunneling_for_https_requests_over_http_proxy_ = false, + const std::string & no_proxy_hosts_ = ""); ProxyConfiguration resolve() override; @@ -23,6 +27,7 @@ public: private: std::vector proxies; + std::string no_proxy_hosts; /// Access counter to get proxy using round-robin strategy. std::atomic access_counter; diff --git a/src/Common/RemoteProxyConfigurationResolver.cpp b/src/Common/RemoteProxyConfigurationResolver.cpp index ef972a8e318..064840d2e04 100644 --- a/src/Common/RemoteProxyConfigurationResolver.cpp +++ b/src/Common/RemoteProxyConfigurationResolver.cpp @@ -19,9 +19,11 @@ namespace ErrorCodes RemoteProxyConfigurationResolver::RemoteProxyConfigurationResolver( const RemoteServerConfiguration & remote_server_configuration_, Protocol request_protocol_, - bool disable_tunneling_for_https_requests_over_http_proxy_ + bool disable_tunneling_for_https_requests_over_http_proxy_, + const std::string & no_proxy_hosts_ ) -: ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_), remote_server_configuration(remote_server_configuration_) +: ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_), + remote_server_configuration(remote_server_configuration_), no_proxy_hosts(no_proxy_hosts_) { } @@ -105,6 +107,7 @@ ProxyConfiguration RemoteProxyConfigurationResolver::resolve() cached_config.port = proxy_port; cached_config.tunneling = use_tunneling_for_https_requests_over_http_proxy; cached_config.original_request_protocol = request_protocol; + cached_config.no_proxy_hosts = no_proxy_hosts; cache_timestamp = std::chrono::system_clock::now(); cache_valid = true; diff --git a/src/Common/RemoteProxyConfigurationResolver.h b/src/Common/RemoteProxyConfigurationResolver.h index 3275202215a..986fd233835 100644 --- a/src/Common/RemoteProxyConfigurationResolver.h +++ b/src/Common/RemoteProxyConfigurationResolver.h @@ -28,7 +28,8 @@ public: RemoteProxyConfigurationResolver( const RemoteServerConfiguration & remote_server_configuration_, Protocol request_protocol_, - bool disable_tunneling_for_https_requests_over_http_proxy_ = true); + bool disable_tunneling_for_https_requests_over_http_proxy_ = true, + const std::string & no_proxy_hosts_ = ""); ProxyConfiguration resolve() override; @@ -36,6 +37,7 @@ public: private: RemoteServerConfiguration remote_server_configuration; + std::string no_proxy_hosts; std::mutex cache_mutex; bool cache_valid = false; diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index d5d6f86f661..246d0121255 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -31,9 +31,11 @@ Poco::URI https_env_proxy_server = Poco::URI("http://https_environment_proxy:312 Poco::URI http_list_proxy_server = Poco::URI("http://http_list_proxy:3128"); Poco::URI https_list_proxy_server = Poco::URI("http://https_list_proxy:3128"); +static std::string no_proxy_hosts = "localhost,,127.0.0.1,some_other_domain,,,, last_domain,"; + TEST_F(ProxyConfigurationResolverProviderTests, EnvironmentResolverShouldBeUsedIfNoSettings) { - EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server); + EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server, no_proxy_hosts); const auto & config = getContext().context->getConfigRef(); auto http_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, config)->resolve(); @@ -42,10 +44,12 @@ TEST_F(ProxyConfigurationResolverProviderTests, EnvironmentResolverShouldBeUsedI ASSERT_EQ(http_configuration.host, http_env_proxy_server.getHost()); ASSERT_EQ(http_configuration.port, http_env_proxy_server.getPort()); ASSERT_EQ(http_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_env_proxy_server.getScheme())); + ASSERT_EQ(http_configuration.no_proxy_hosts, no_proxy_hosts); ASSERT_EQ(https_configuration.host, https_env_proxy_server.getHost()); ASSERT_EQ(https_configuration.port, https_env_proxy_server.getPort()); ASSERT_EQ(https_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_env_proxy_server.getScheme())); + ASSERT_EQ(https_configuration.no_proxy_hosts, no_proxy_hosts); } TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPOnly) @@ -53,6 +57,7 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPOnly) ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); config->setString("proxy", ""); + config->setString("proxy.no_proxy", no_proxy_hosts); config->setString("proxy.http", ""); config->setString("proxy.http.uri", http_list_proxy_server.toString()); context->setConfig(config); @@ -62,18 +67,21 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPOnly) ASSERT_EQ(http_proxy_configuration.host, http_list_proxy_server.getHost()); ASSERT_EQ(http_proxy_configuration.port, http_list_proxy_server.getPort()); ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_list_proxy_server.getScheme())); + ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, no_proxy_hosts); auto https_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve(); // No https configuration since it's not set ASSERT_EQ(https_proxy_configuration.host, ""); ASSERT_EQ(https_proxy_configuration.port, 0); + ASSERT_EQ(https_proxy_configuration.no_proxy_hosts, ""); } TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPSOnly) { ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); + config->setString("proxy.no_proxy", no_proxy_hosts); config->setString("proxy", ""); config->setString("proxy.https", ""); config->setString("proxy.https.uri", https_list_proxy_server.toString()); @@ -91,12 +99,14 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPSOnly) // still HTTP because the proxy host is not HTTPS ASSERT_EQ(https_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_list_proxy_server.getScheme())); ASSERT_EQ(https_proxy_configuration.port, https_list_proxy_server.getPort()); + ASSERT_EQ(https_proxy_configuration.no_proxy_hosts, no_proxy_hosts); } TEST_F(ProxyConfigurationResolverProviderTests, ListBoth) { ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); + config->setString("proxy.no_proxy", no_proxy_hosts); config->setString("proxy", ""); config->setString("proxy.http", ""); config->setString("proxy.http.uri", http_list_proxy_server.toString()); @@ -112,6 +122,7 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListBoth) ASSERT_EQ(http_proxy_configuration.host, http_list_proxy_server.getHost()); ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_list_proxy_server.getScheme())); ASSERT_EQ(http_proxy_configuration.port, http_list_proxy_server.getPort()); + ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, no_proxy_hosts); auto https_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve(); @@ -120,6 +131,7 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListBoth) // still HTTP because the proxy host is not HTTPS ASSERT_EQ(https_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_list_proxy_server.getScheme())); ASSERT_EQ(https_proxy_configuration.port, https_list_proxy_server.getPort()); + ASSERT_EQ(https_proxy_configuration.no_proxy_hosts, no_proxy_hosts); } TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTP) @@ -157,6 +169,7 @@ TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolC ASSERT_EQ(http_proxy_configuration.host, http_env_proxy_server.getHost()); ASSERT_EQ(http_proxy_configuration.port, http_env_proxy_server.getPort()); ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_env_proxy_server.getScheme())); + ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, no_proxy_hosts); } TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTPS) @@ -194,6 +207,7 @@ TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolC ASSERT_EQ(http_proxy_configuration.host, https_env_proxy_server.getHost()); ASSERT_EQ(http_proxy_configuration.port, https_env_proxy_server.getPort()); ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_env_proxy_server.getScheme())); + ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, no_proxy_hosts); } // remote resolver is tricky to be tested in unit tests From 11dacf040ebebf5a4f268a99a34163a9a1fe7b5b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sat, 4 May 2024 11:03:40 -0300 Subject: [PATCH 052/856] fix tests --- ..._proxy_configuration_resolver_provider.cpp | 4 +- .../configs/config.d/proxy_list_no_proxy.xml | 9 +++ .../config.d/proxy_remote_no_proxy.xml | 18 ++++++ .../test.py | 34 +++++++++-- .../test.py | 56 +++++++++++++++---- 5 files changed, 103 insertions(+), 18 deletions(-) create mode 100644 tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_list_no_proxy.xml create mode 100644 tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_remote_no_proxy.xml diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index 246d0121255..174e9564cee 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -149,6 +149,7 @@ TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolC ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); config->setString("proxy", ""); + config->setString("proxy.no_proxy", no_proxy_hosts); config->setString("proxy.https", ""); config->setString("proxy.https.resolver", ""); config->setString("proxy.https.resolver.endpoint", "http://resolver:8080/hostname"); @@ -169,7 +170,7 @@ TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolC ASSERT_EQ(http_proxy_configuration.host, http_env_proxy_server.getHost()); ASSERT_EQ(http_proxy_configuration.port, http_env_proxy_server.getPort()); ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_env_proxy_server.getScheme())); - ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, no_proxy_hosts); + ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, ""); } TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTPS) @@ -244,4 +245,3 @@ TEST_F(ProxyConfigurationResolverProviderTests, TunnelingForHTTPSRequestsOverHTT test_tunneling(context); test_tunneling(context); } - diff --git a/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_list_no_proxy.xml b/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_list_no_proxy.xml new file mode 100644 index 00000000000..a1601153151 --- /dev/null +++ b/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_list_no_proxy.xml @@ -0,0 +1,9 @@ + + + not_important_host,, minio1 , + + http://proxy1 + http://proxy2 + + + diff --git a/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_remote_no_proxy.xml b/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_remote_no_proxy.xml new file mode 100644 index 00000000000..6c16a65b154 --- /dev/null +++ b/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_remote_no_proxy.xml @@ -0,0 +1,18 @@ + + + not_important_host,, minio1 , + + + + http://resolver:8080/hostname + http + 80 + 10 + + + + diff --git a/tests/integration/test_s3_table_function_with_http_proxy/test.py b/tests/integration/test_s3_table_function_with_http_proxy/test.py index cc0d59e51ba..a935c5290f4 100644 --- a/tests/integration/test_s3_table_function_with_http_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_http_proxy/test.py @@ -19,6 +19,15 @@ def cluster(): with_minio=True, ) + cluster.add_instance( + "remote_proxy_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_remote_no_proxy.xml", + ], + instance_env_variables=True, + with_minio=True, + ) + cluster.add_instance( "proxy_list_node", main_configs=[ @@ -27,6 +36,15 @@ def cluster(): with_minio=True, ) + cluster.add_instance( + "proxy_list_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_list_no_proxy.xml", + ], + instance_env_variables=True, + with_minio=True, + ) + cluster.add_instance( "env_node", with_minio=True, @@ -58,6 +76,18 @@ def cluster(): cluster.shutdown() +def test_s3_with_http_proxy_list_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1", "proxy2"], "http", "proxy_list_node_no_proxy") + + +def test_s3_with_http_remote_proxy_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "http", "remote_proxy_node_no_proxy") + + +def test_s3_with_http_env_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "http", "env_node_no_proxy") + + def test_s3_with_http_proxy_list(cluster): proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "http", "proxy_list_node") @@ -68,7 +98,3 @@ def test_s3_with_http_remote_proxy(cluster): def test_s3_with_http_env_proxy(cluster): proxy_util.simple_test(cluster, ["proxy1"], "http", "env_node") - - -def test_s3_with_http_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "http", "env_node_no_proxy") diff --git a/tests/integration/test_s3_table_function_with_https_proxy/test.py b/tests/integration/test_s3_table_function_with_https_proxy/test.py index 87285415d85..b1adf2533c8 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_https_proxy/test.py @@ -23,6 +23,18 @@ def cluster(): minio_certs_dir="minio_certs", ) + cluster.add_instance( + "remote_proxy_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_remote.xml", + "configs/config.d/ssl.xml", + ], + env_variables={ + "no_proxy": "not_important_host,, minio1 ,", + }, + with_minio=True, + ) + cluster.add_instance( "proxy_list_node", main_configs=[ @@ -32,6 +44,18 @@ def cluster(): with_minio=True, ) + cluster.add_instance( + "proxy_list_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_list.xml", + "configs/config.d/ssl.xml", + ], + env_variables={ + "no_proxy": "not_important_host,, minio1 ,", + }, + with_minio=True, + ) + cluster.add_instance( "env_node", main_configs=[ @@ -69,17 +93,25 @@ def cluster(): cluster.shutdown() -def test_s3_with_https_proxy_list(cluster): - proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node") +def test_s3_with_https_proxy_list_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node_no_proxy") + +# +# def test_s3_with_https_remote_proxy(cluster): +# proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "remote_proxy_node_no_proxy") +# +# +# def test_s3_with_https_env_no_proxy(cluster): +# proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "env_node_no_proxy") +# +# +# def test_s3_with_https_proxy_list(cluster): +# proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node") +# +# +# def test_s3_with_https_remote_proxy(cluster): +# proxy_util.simple_test(cluster, ["proxy1"], "https", "remote_proxy_node") -def test_s3_with_https_remote_proxy(cluster): - proxy_util.simple_test(cluster, ["proxy1"], "https", "remote_proxy_node") - - -def test_s3_with_https_env_proxy(cluster): - proxy_util.simple_test(cluster, ["proxy1"], "https", "env_node") - - -def test_s3_with_https_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "env_node_no_proxy") +# def test_s3_with_https_env_proxy(cluster): +# proxy_util.simple_test(cluster, ["proxy1"], "https", "env_node") From db21c2cf0a8e267b97675c4f2269f5e30ab163d6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sat, 4 May 2024 11:12:18 -0300 Subject: [PATCH 053/856] fix tests --- .../configs/config.d/proxy_list_no_proxy.xml | 13 ++++++ .../config.d/proxy_remote_no_proxy.xml | 18 ++++++++ .../test.py | 44 ++++++++----------- 3 files changed, 50 insertions(+), 25 deletions(-) create mode 100644 tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list_no_proxy.xml create mode 100644 tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote_no_proxy.xml diff --git a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list_no_proxy.xml b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list_no_proxy.xml new file mode 100644 index 00000000000..0a03986f839 --- /dev/null +++ b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list_no_proxy.xml @@ -0,0 +1,13 @@ + + + not_important_host,, minio1 , + + http://proxy1 + http://proxy2 + + + https://proxy1 + https://proxy2 + + + diff --git a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote_no_proxy.xml b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote_no_proxy.xml new file mode 100644 index 00000000000..943f2b36a34 --- /dev/null +++ b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote_no_proxy.xml @@ -0,0 +1,18 @@ + + + not_important_host,, minio1 , + + + + http://resolver:8080/hostname + https + 443 + 10 + + + + diff --git a/tests/integration/test_s3_table_function_with_https_proxy/test.py b/tests/integration/test_s3_table_function_with_https_proxy/test.py index b1adf2533c8..2cce74565d7 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_https_proxy/test.py @@ -26,12 +26,9 @@ def cluster(): cluster.add_instance( "remote_proxy_node_no_proxy", main_configs=[ - "configs/config.d/proxy_remote.xml", + "configs/config.d/proxy_remote_no_proxy.xml", "configs/config.d/ssl.xml", ], - env_variables={ - "no_proxy": "not_important_host,, minio1 ,", - }, with_minio=True, ) @@ -47,12 +44,9 @@ def cluster(): cluster.add_instance( "proxy_list_node_no_proxy", main_configs=[ - "configs/config.d/proxy_list.xml", + "configs/config.d/proxy_list_no_proxy.xml", "configs/config.d/ssl.xml", ], - env_variables={ - "no_proxy": "not_important_host,, minio1 ,", - }, with_minio=True, ) @@ -96,22 +90,22 @@ def cluster(): def test_s3_with_https_proxy_list_no_proxy(cluster): proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node_no_proxy") -# -# def test_s3_with_https_remote_proxy(cluster): -# proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "remote_proxy_node_no_proxy") -# -# -# def test_s3_with_https_env_no_proxy(cluster): -# proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "env_node_no_proxy") -# -# -# def test_s3_with_https_proxy_list(cluster): -# proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node") -# -# -# def test_s3_with_https_remote_proxy(cluster): -# proxy_util.simple_test(cluster, ["proxy1"], "https", "remote_proxy_node") + +def test_s3_with_https_env_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "env_node_no_proxy") -# def test_s3_with_https_env_proxy(cluster): -# proxy_util.simple_test(cluster, ["proxy1"], "https", "env_node") +def test_s3_with_https_remote_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "remote_proxy_node_no_proxy") + + +def test_s3_with_https_proxy_list(cluster): + proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node") + + +def test_s3_with_https_remote_proxy(cluster): + proxy_util.simple_test(cluster, ["proxy1"], "https", "remote_proxy_node") + + +def test_s3_with_https_env_proxy(cluster): + proxy_util.simple_test(cluster, ["proxy1"], "https", "env_node") From 96b0d878434a3d6ddf0c98d7703b139b979aa0d4 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 6 May 2024 10:38:10 -0300 Subject: [PATCH 054/856] first impl for subdomains --- .../proxyConfigurationToPocoProxyConfig.cpp | 35 +++++++++++++++++-- ...oxy_configuration_to_poco_proxy_config.cpp | 2 +- 2 files changed, 33 insertions(+), 4 deletions(-) diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.cpp b/src/Common/proxyConfigurationToPocoProxyConfig.cpp index ec2a5ba19bd..3e2642d1600 100644 --- a/src/Common/proxyConfigurationToPocoProxyConfig.cpp +++ b/src/Common/proxyConfigurationToPocoProxyConfig.cpp @@ -17,6 +17,25 @@ namespace DB { +namespace +{ + +/* + * Copy `curl` behavior instead of `wget` as it seems to be more flexible. + * `curl` strips leading dot and accepts url gitlab.com as a match for no_proxy .gitlab.com, + * while `wget` does an exact match. + * */ +std::string buildPocoRegexpEntryWithoutLeadingDot(const std::string & host) +{ + std::string_view view_without_leading_dot = host; + if (host[0] == '.') + { + view_without_leading_dot = std::string_view {host.begin() + 1u, host.end()}; + } + + return RE2::QuoteMeta(view_without_leading_dot); +} + /* * Even though there is not an RFC that defines NO_PROXY, it is usually a comma-separated list of domains. * Different tools implement their own versions of `NO_PROXY` support. Some support CIDR blocks, some support wildcard etc. @@ -31,11 +50,15 @@ namespace DB * */ std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts) { + static constexpr auto OR_SEPARATOR = "|"; + static constexpr auto MATCH_ANYTHING = R"((.*?))"; + static constexpr auto MATCH_SUBDOMAINS_REGEX = R"((?:.*\.)?)"; + bool match_any_host = no_proxy_hosts.size() == 1 && no_proxy_hosts[0] == '*'; if (match_any_host) { - return "(.*?)"; + return MATCH_ANYTHING; } std::string host; @@ -52,10 +75,14 @@ std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts) { if (!first) { - result.append("|"); + result.append(OR_SEPARATOR); } - result.append(RE2::QuoteMeta(host)); + auto escaped_host_without_leading_dot = buildPocoRegexpEntryWithoutLeadingDot(host); + + result.append(MATCH_SUBDOMAINS_REGEX); + result.append(escaped_host_without_leading_dot); + first = false; } } @@ -63,6 +90,8 @@ std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts) return result; } +} + Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration) { Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; diff --git a/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp b/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp index 4a8488d4055..db87f23fc65 100644 --- a/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp +++ b/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp @@ -10,7 +10,7 @@ TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuild) auto poco_proxy_configuration = DB::proxyConfigurationToPocoProxyConfig(proxy_configuration); - ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, "localhost|127\\.0\\.0\\.1|some_other_domain\\:8080|sub\\-domain\\.domain\\.com"); + ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, R"((?:.*\.)?localhost|(?:.*\.)?127\.0\.0\.1|(?:.*\.)?some_other_domain\:8080|(?:.*\.)?sub\-domain\.domain\.com)"); } TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildMatchAnything) From c3a4338d3b427ef8638e5ba6033eee4571eeb281 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 6 May 2024 10:43:25 -0300 Subject: [PATCH 055/856] fix tests --- src/Common/tests/gtest_helper_functions.h | 1 + .../tests/gtest_proxy_configuration_resolver_provider.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_helper_functions.h b/src/Common/tests/gtest_helper_functions.h index e3aeea407a1..1e5c2b21d99 100644 --- a/src/Common/tests/gtest_helper_functions.h +++ b/src/Common/tests/gtest_helper_functions.h @@ -101,5 +101,6 @@ struct EnvironmentProxySetter { unsetenv("http_proxy"); // NOLINT(concurrency-mt-unsafe) unsetenv("https_proxy"); // NOLINT(concurrency-mt-unsafe) + unsetenv("no_proxy"); // NOLINT(concurrency-mt-unsafe) } }; diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index 174e9564cee..6e19520ca22 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -208,7 +208,7 @@ TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolC ASSERT_EQ(http_proxy_configuration.host, https_env_proxy_server.getHost()); ASSERT_EQ(http_proxy_configuration.port, https_env_proxy_server.getPort()); ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_env_proxy_server.getScheme())); - ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, no_proxy_hosts); + ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, ""); } // remote resolver is tricky to be tested in unit tests From 7dc4f1778bd8690f62e378ba3c26a013e6ae208b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 6 May 2024 19:44:59 +0000 Subject: [PATCH 056/856] 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 b7096609692098886b050d36e84897792d2872df Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 6 May 2024 16:53:36 -0300 Subject: [PATCH 057/856] update doc and use early return --- .../proxyConfigurationToPocoProxyConfig.cpp | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.cpp b/src/Common/proxyConfigurationToPocoProxyConfig.cpp index 3e2642d1600..f14d586677a 100644 --- a/src/Common/proxyConfigurationToPocoProxyConfig.cpp +++ b/src/Common/proxyConfigurationToPocoProxyConfig.cpp @@ -41,9 +41,10 @@ std::string buildPocoRegexpEntryWithoutLeadingDot(const std::string & host) * Different tools implement their own versions of `NO_PROXY` support. Some support CIDR blocks, some support wildcard etc. * Opting for a simple implementation that covers most use cases: * * Support only single wildcard * (match anything) + * * Match subdomains + * * Strip leading dots * * No regex * * No CIDR blocks - * * No leading dot strip * * No fancy stuff about loopback IPs * https://about.gitlab.com/blog/2021/01/27/we-need-to-talk-no-proxy/ * Open for discussions @@ -71,20 +72,22 @@ std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts) { trim(host); - if (!host.empty()) + if (host.empty()) { - if (!first) - { - result.append(OR_SEPARATOR); - } - - auto escaped_host_without_leading_dot = buildPocoRegexpEntryWithoutLeadingDot(host); - - result.append(MATCH_SUBDOMAINS_REGEX); - result.append(escaped_host_without_leading_dot); - - first = false; + continue; } + + if (!first) + { + result.append(OR_SEPARATOR); + } + + auto escaped_host_without_leading_dot = buildPocoRegexpEntryWithoutLeadingDot(host); + + result.append(MATCH_SUBDOMAINS_REGEX); + result.append(escaped_host_without_leading_dot); + + first = false; } return result; From 9aad50f35dfa325a3968707ec3d7af94e9b5af10 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 6 May 2024 17:02:20 -0300 Subject: [PATCH 058/856] reorder args and adapt tests --- src/Common/ProxyConfigurationResolverProvider.cpp | 8 ++++---- src/Common/ProxyListConfigurationResolver.cpp | 4 ++-- src/Common/ProxyListConfigurationResolver.h | 4 ++-- src/Common/RemoteProxyConfigurationResolver.cpp | 4 ++-- src/Common/RemoteProxyConfigurationResolver.h | 4 ++-- ...gtest_proxy_configuration_resolver_provider.cpp | 2 +- .../gtest_proxy_list_configuration_resolver.cpp | 14 +++++++++++--- 7 files changed, 24 insertions(+), 16 deletions(-) diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index 360db80e360..a486304ce80 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -54,8 +54,8 @@ namespace return std::make_shared( server_configuration, request_protocol, - isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration), - getNoProxyHosts(configuration)); + getNoProxyHosts(configuration), + isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); } auto extractURIList(const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) @@ -96,8 +96,8 @@ namespace : std::make_shared( uris, request_protocol, - isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration), - getNoProxyHosts(configuration)); + getNoProxyHosts(configuration), + isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); } bool hasRemoteResolver(const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) diff --git a/src/Common/ProxyListConfigurationResolver.cpp b/src/Common/ProxyListConfigurationResolver.cpp index 2560b3b8d1d..985bdf001b1 100644 --- a/src/Common/ProxyListConfigurationResolver.cpp +++ b/src/Common/ProxyListConfigurationResolver.cpp @@ -10,8 +10,8 @@ namespace DB ProxyListConfigurationResolver::ProxyListConfigurationResolver( std::vector proxies_, Protocol request_protocol_, - bool disable_tunneling_for_https_requests_over_http_proxy_, - const std::string & no_proxy_hosts_) + const std::string & no_proxy_hosts_, + bool disable_tunneling_for_https_requests_over_http_proxy_) : ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_), proxies(std::move(proxies_)), no_proxy_hosts(no_proxy_hosts_) { diff --git a/src/Common/ProxyListConfigurationResolver.h b/src/Common/ProxyListConfigurationResolver.h index db69e7eb6c9..a87826792d4 100644 --- a/src/Common/ProxyListConfigurationResolver.h +++ b/src/Common/ProxyListConfigurationResolver.h @@ -18,8 +18,8 @@ public: ProxyListConfigurationResolver( std::vector proxies_, Protocol request_protocol_, - bool disable_tunneling_for_https_requests_over_http_proxy_ = false, - const std::string & no_proxy_hosts_ = ""); + const std::string & no_proxy_hosts_, + bool disable_tunneling_for_https_requests_over_http_proxy_ = false); ProxyConfiguration resolve() override; diff --git a/src/Common/RemoteProxyConfigurationResolver.cpp b/src/Common/RemoteProxyConfigurationResolver.cpp index 064840d2e04..06587ab7f53 100644 --- a/src/Common/RemoteProxyConfigurationResolver.cpp +++ b/src/Common/RemoteProxyConfigurationResolver.cpp @@ -19,8 +19,8 @@ namespace ErrorCodes RemoteProxyConfigurationResolver::RemoteProxyConfigurationResolver( const RemoteServerConfiguration & remote_server_configuration_, Protocol request_protocol_, - bool disable_tunneling_for_https_requests_over_http_proxy_, - const std::string & no_proxy_hosts_ + const std::string & no_proxy_hosts_, + bool disable_tunneling_for_https_requests_over_http_proxy_ ) : ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_), remote_server_configuration(remote_server_configuration_), no_proxy_hosts(no_proxy_hosts_) diff --git a/src/Common/RemoteProxyConfigurationResolver.h b/src/Common/RemoteProxyConfigurationResolver.h index 986fd233835..f9add6f0769 100644 --- a/src/Common/RemoteProxyConfigurationResolver.h +++ b/src/Common/RemoteProxyConfigurationResolver.h @@ -28,8 +28,8 @@ public: RemoteProxyConfigurationResolver( const RemoteServerConfiguration & remote_server_configuration_, Protocol request_protocol_, - bool disable_tunneling_for_https_requests_over_http_proxy_ = true, - const std::string & no_proxy_hosts_ = ""); + const std::string & no_proxy_hosts_, + bool disable_tunneling_for_https_requests_over_http_proxy_ = true); ProxyConfiguration resolve() override; diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index 6e19520ca22..d248835699a 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -31,7 +31,7 @@ Poco::URI https_env_proxy_server = Poco::URI("http://https_environment_proxy:312 Poco::URI http_list_proxy_server = Poco::URI("http://http_list_proxy:3128"); Poco::URI https_list_proxy_server = Poco::URI("http://https_list_proxy:3128"); -static std::string no_proxy_hosts = "localhost,,127.0.0.1,some_other_domain,,,, last_domain,"; +static std::string no_proxy_hosts = "localhost,,127.0.0.1,some_other_domain,,,, sub-domain.domain.com,"; TEST_F(ProxyConfigurationResolverProviderTests, EnvironmentResolverShouldBeUsedIfNoSettings) { diff --git a/src/Common/tests/gtest_proxy_list_configuration_resolver.cpp b/src/Common/tests/gtest_proxy_list_configuration_resolver.cpp index 3234fe0ccd1..5d8268eb206 100644 --- a/src/Common/tests/gtest_proxy_list_configuration_resolver.cpp +++ b/src/Common/tests/gtest_proxy_list_configuration_resolver.cpp @@ -10,6 +10,8 @@ namespace { auto proxy_server1 = Poco::URI("http://proxy_server1:3128"); auto proxy_server2 = Poco::URI("http://proxy_server2:3128"); + + std::string no_proxy_hosts = "localhost,,127.0.0.1,some_other_domain,,,, sub-domain.domain.com,"; } TEST(ProxyListConfigurationResolver, SimpleTest) @@ -17,7 +19,8 @@ TEST(ProxyListConfigurationResolver, SimpleTest) ProxyListConfigurationResolver resolver( {proxy_server1, proxy_server2}, - ProxyConfiguration::Protocol::HTTP); + ProxyConfiguration::Protocol::HTTP, + no_proxy_hosts); auto configuration1 = resolver.resolve(); auto configuration2 = resolver.resolve(); @@ -25,10 +28,12 @@ TEST(ProxyListConfigurationResolver, SimpleTest) ASSERT_EQ(configuration1.host, proxy_server1.getHost()); ASSERT_EQ(configuration1.port, proxy_server1.getPort()); ASSERT_EQ(configuration1.protocol, ProxyConfiguration::protocolFromString(proxy_server1.getScheme())); + ASSERT_EQ(configuration1.no_proxy_hosts, no_proxy_hosts); ASSERT_EQ(configuration2.host, proxy_server2.getHost()); ASSERT_EQ(configuration2.port, proxy_server2.getPort()); ASSERT_EQ(configuration2.protocol, ProxyConfiguration::protocolFromString(proxy_server2.getScheme())); + ASSERT_EQ(configuration2.no_proxy_hosts, no_proxy_hosts); } TEST(ProxyListConfigurationResolver, HTTPSRequestsOverHTTPProxyDefault) @@ -36,7 +41,8 @@ TEST(ProxyListConfigurationResolver, HTTPSRequestsOverHTTPProxyDefault) ProxyListConfigurationResolver resolver( {proxy_server1, proxy_server2}, - ProxyConfiguration::Protocol::HTTPS); + ProxyConfiguration::Protocol::HTTPS, + ""); auto configuration1 = resolver.resolve(); auto configuration2 = resolver.resolve(); @@ -45,11 +51,12 @@ TEST(ProxyListConfigurationResolver, HTTPSRequestsOverHTTPProxyDefault) ASSERT_EQ(configuration1.port, proxy_server1.getPort()); ASSERT_EQ(configuration1.protocol, ProxyConfiguration::protocolFromString(proxy_server1.getScheme())); ASSERT_EQ(configuration1.tunneling, true); + ASSERT_EQ(configuration1.no_proxy_hosts, ""); ASSERT_EQ(configuration2.host, proxy_server2.getHost()); ASSERT_EQ(configuration2.port, proxy_server2.getPort()); ASSERT_EQ(configuration2.protocol, ProxyConfiguration::protocolFromString(proxy_server2.getScheme())); - ASSERT_EQ(configuration1.tunneling, true); + ASSERT_EQ(configuration2.no_proxy_hosts, ""); } TEST(ProxyListConfigurationResolver, SimpleTestTunnelingDisabled) @@ -58,6 +65,7 @@ TEST(ProxyListConfigurationResolver, SimpleTestTunnelingDisabled) ProxyListConfigurationResolver resolver( {proxy_server1, proxy_server2}, ProxyConfiguration::Protocol::HTTPS, + "", disable_tunneling_for_https_requests_over_http_proxy); auto configuration1 = resolver.resolve(); From eba4b1d558797153a3d41f3304ca8465712f8065 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 6 May 2024 17:15:51 -0300 Subject: [PATCH 059/856] docs for no_proxy --- .../settings.md | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 28831404a1f..4f8a82805af 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2875,6 +2875,8 @@ Define proxy servers for HTTP and HTTPS requests, currently supported by S3 stor There are three ways to define proxy servers: environment variables, proxy lists, and remote proxy resolvers. +Bypassing proxy servers for specific hosts is also supported with the use of `no_proxy`. + ### Environment variables The `http_proxy` and `https_proxy` environment variables allow you to specify a @@ -2984,6 +2986,29 @@ This also allows a mix of resolver types can be used. By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests over `HTTP` proxy. This setting can be used to disable it. +### no_proxy +By default, all requests will go through the proxy. In order to disable it for specific hosts, the `no_proxy` variable must be set. +It can be set inside the `` clause for list and remote resolvers and as an environment variable for environment resolver. +It supports IP addresses, domains, subdomains and `'*'` wildcard for full bypass. Leading dots are stripped just like curl does. + +Example: + +The below configuration bypasses proxy requests to `clickhouse.cloud` and all of its subdomains (e.g, `auth.clickhouse.cloud`). +The same applies to gitlab, even though it has a leading dot. Both `gitlab.com` and `about.gitlab.com` would bypass the proxy. + +``` xml + + clickhouse.cloud,.gitlab.com + + http://proxy1 + http://proxy2:3128 + + + http://proxy1:3128 + + +``` + ## max_materialized_views_count_for_table {#max_materialized_views_count_for_table} A limit on the number of materialized views attached to a table. From 5eeadb6c0704f5b41407ed6f26deac9d297e720f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 8 May 2024 10:55:26 +0200 Subject: [PATCH 060/856] Support running function for parameterized view value assignment --- src/Interpreters/Context.cpp | 4 ++-- src/Parsers/FunctionParameterValuesVisitor.cpp | 14 +++++++++++--- src/Parsers/FunctionParameterValuesVisitor.h | 3 ++- .../03146_parameterized_view_with_date.reference | 1 + .../03146_parameterized_view_with_date.sql | 12 ++++++++++++ 5 files changed, 28 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03146_parameterized_view_with_date.reference create mode 100644 tests/queries/0_stateless/03146_parameterized_view_with_date.sql diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 44d36e94441..4d84da36f78 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1840,7 +1840,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) { auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); - NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression, getQueryContext()); StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); ASTCreateQuery create; @@ -2054,7 +2054,7 @@ StoragePtr Context::buildParametrizedViewStorage(const ASTPtr & table_expression return nullptr; auto query = original_view->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); - NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression, getQueryContext()); StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); ASTCreateQuery create; diff --git a/src/Parsers/FunctionParameterValuesVisitor.cpp b/src/Parsers/FunctionParameterValuesVisitor.cpp index 3692a4c73e5..e791e07cdfb 100644 --- a/src/Parsers/FunctionParameterValuesVisitor.cpp +++ b/src/Parsers/FunctionParameterValuesVisitor.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -20,8 +21,9 @@ namespace ErrorCodes class FunctionParameterValuesVisitor { public: - explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_) + explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_, ContextPtr context_) : parameter_values(parameter_values_) + ,context(context_) { } @@ -35,6 +37,7 @@ public: private: NameToNameMap & parameter_values; + ContextPtr context; void visitFunction(const ASTFunction & parameter_function) { @@ -64,15 +67,20 @@ private: parameter_values[identifier->name()] = convertFieldToString(cast_literal->value); } } + else + { + ASTPtr res = evaluateConstantExpressionOrIdentifierAsLiteral(expression_list->children[1], context); + parameter_values[identifier->name()] = convertFieldToString(res->as()->value); + } } } } }; -NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast) +NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast, ContextPtr context) { NameToNameMap parameter_values; - FunctionParameterValuesVisitor(parameter_values).visit(ast); + FunctionParameterValuesVisitor(parameter_values, context).visit(ast); return parameter_values; } diff --git a/src/Parsers/FunctionParameterValuesVisitor.h b/src/Parsers/FunctionParameterValuesVisitor.h index e6ce0e42d06..8c2686dcc65 100644 --- a/src/Parsers/FunctionParameterValuesVisitor.h +++ b/src/Parsers/FunctionParameterValuesVisitor.h @@ -2,12 +2,13 @@ #include #include +#include namespace DB { /// Find parameters in a query parameter values and collect them into map. -NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast); +NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast, ContextPtr context); } diff --git a/tests/queries/0_stateless/03146_parameterized_view_with_date.reference b/tests/queries/0_stateless/03146_parameterized_view_with_date.reference new file mode 100644 index 00000000000..1d6227dbbcb --- /dev/null +++ b/tests/queries/0_stateless/03146_parameterized_view_with_date.reference @@ -0,0 +1 @@ +2 2024-04-01 01:00:00 diff --git a/tests/queries/0_stateless/03146_parameterized_view_with_date.sql b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql new file mode 100644 index 00000000000..53022e969ab --- /dev/null +++ b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql @@ -0,0 +1,12 @@ + +drop table if exists table_pv; +create table table_pv (id Int32, timestamp_field DateTime) engine = Memory(); + +insert into table_pv values(1, '2024-03-01 00:00:00'); +insert into table_pv values (2, '2024-04-01 01:00:00'); + +create view pv as select * from table_pv where timestamp_field > {timestamp_param:DateTime}; + +select * from pv (timestamp_param=toDateTime('2024-04-01 00:00:01')); + +drop table table_pv; From 0b939044087f9494cafab57ac1377ed58ed95971 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 8 May 2024 12:16:42 +0000 Subject: [PATCH 061/856] 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 f7d66166ab13dee86bc6f3442707dba2a62bf61f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 8 May 2024 16:13:04 +0200 Subject: [PATCH 062/856] Bump From e1ed0af3d2598f6511a8d804ed52f0822d06b5b5 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 8 May 2024 14:28:09 +0000 Subject: [PATCH 063/856] 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 064/856] 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 065/856] 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 066/856] 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 067/856] 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 068/856] 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 069/856] 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 6a2aa299464d1cbf5ee2fc112813dc560c34c838 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 10 May 2024 17:20:36 -0300 Subject: [PATCH 070/856] don't use stringstream --- .../settings.md | 2 +- .../proxyConfigurationToPocoProxyConfig.cpp | 11 ++++++----- .../helpers/s3_url_proxy_tests_util.py | 19 ++++++++++--------- .../test.py | 12 +++++++++--- .../test.py | 12 +++++++++--- 5 files changed, 35 insertions(+), 21 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 4f8a82805af..5fee3c6db5f 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2994,7 +2994,7 @@ It supports IP addresses, domains, subdomains and `'*'` wildcard for full bypass Example: The below configuration bypasses proxy requests to `clickhouse.cloud` and all of its subdomains (e.g, `auth.clickhouse.cloud`). -The same applies to gitlab, even though it has a leading dot. Both `gitlab.com` and `about.gitlab.com` would bypass the proxy. +The same applies to GitLab, even though it has a leading dot. Both `gitlab.com` and `about.gitlab.com` would bypass the proxy. ``` xml diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.cpp b/src/Common/proxyConfigurationToPocoProxyConfig.cpp index f14d586677a..c221dd394ca 100644 --- a/src/Common/proxyConfigurationToPocoProxyConfig.cpp +++ b/src/Common/proxyConfigurationToPocoProxyConfig.cpp @@ -3,6 +3,7 @@ #include #include +#include #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" @@ -49,26 +50,26 @@ std::string buildPocoRegexpEntryWithoutLeadingDot(const std::string & host) * https://about.gitlab.com/blog/2021/01/27/we-need-to-talk-no-proxy/ * Open for discussions * */ -std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts) +std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts_string) { static constexpr auto OR_SEPARATOR = "|"; static constexpr auto MATCH_ANYTHING = R"((.*?))"; static constexpr auto MATCH_SUBDOMAINS_REGEX = R"((?:.*\.)?)"; - bool match_any_host = no_proxy_hosts.size() == 1 && no_proxy_hosts[0] == '*'; + bool match_any_host = no_proxy_hosts_string.size() == 1 && no_proxy_hosts_string[0] == '*'; if (match_any_host) { return MATCH_ANYTHING; } - std::string host; - std::istringstream no_proxy_stream(no_proxy_hosts); + std::vector no_proxy_hosts; + splitInto<','>(no_proxy_hosts, no_proxy_hosts_string); bool first = true; std::string result; - while (std::getline(no_proxy_stream, host, ',')) + for (auto & host : no_proxy_hosts) { trim(host); diff --git a/tests/integration/helpers/s3_url_proxy_tests_util.py b/tests/integration/helpers/s3_url_proxy_tests_util.py index 16df446b0f7..7d45dcdec33 100644 --- a/tests/integration/helpers/s3_url_proxy_tests_util.py +++ b/tests/integration/helpers/s3_url_proxy_tests_util.py @@ -3,7 +3,7 @@ import time def has_any_proxy_related_logs( -cluster, proxy_instance, protocol, bucket, http_methods={"POST", "PUT", "GET"} + cluster, proxy_instance, protocol, bucket, http_methods={"POST", "PUT", "GET"} ): for i in range(10): logs = cluster.get_container_logs(proxy_instance) @@ -80,7 +80,7 @@ def perform_simple_queries(node, minio_endpoint): def simple_test(cluster, proxies, protocol, bucket): minio_endpoint = build_s3_endpoint(protocol, bucket) - node = cluster.instances[f"{bucket}"] + node = cluster.instances[bucket] perform_simple_queries(node, minio_endpoint) @@ -90,11 +90,12 @@ def simple_test(cluster, proxies, protocol, bucket): def simple_test_assert_no_proxy(cluster, proxies, protocol, bucket): - minio_endpoint = build_s3_endpoint(protocol, bucket) - node = cluster.instances[f"{bucket}"] + minio_endpoint = build_s3_endpoint(protocol, bucket) + node = cluster.instances[bucket] + perform_simple_queries(node, minio_endpoint) - perform_simple_queries(node, minio_endpoint) - - for proxy in proxies: - no_proxy_logs = not has_any_proxy_related_logs(cluster, proxy, protocol, bucket) - assert no_proxy_logs, f"Found proxy logs in {proxy} and it should not have found it" + for proxy in proxies: + no_proxy_logs = not has_any_proxy_related_logs(cluster, proxy, protocol, bucket) + assert ( + no_proxy_logs + ), f"Found proxy logs in {proxy} and it should not have found it" diff --git a/tests/integration/test_s3_table_function_with_http_proxy/test.py b/tests/integration/test_s3_table_function_with_http_proxy/test.py index a935c5290f4..761057ca6c8 100644 --- a/tests/integration/test_s3_table_function_with_http_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_http_proxy/test.py @@ -77,15 +77,21 @@ def cluster(): def test_s3_with_http_proxy_list_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1", "proxy2"], "http", "proxy_list_node_no_proxy") + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1", "proxy2"], "http", "proxy_list_node_no_proxy" + ) def test_s3_with_http_remote_proxy_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "http", "remote_proxy_node_no_proxy") + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1"], "http", "remote_proxy_node_no_proxy" + ) def test_s3_with_http_env_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "http", "env_node_no_proxy") + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1"], "http", "env_node_no_proxy" + ) def test_s3_with_http_proxy_list(cluster): diff --git a/tests/integration/test_s3_table_function_with_https_proxy/test.py b/tests/integration/test_s3_table_function_with_https_proxy/test.py index 2cce74565d7..26656447d87 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_https_proxy/test.py @@ -88,15 +88,21 @@ def cluster(): def test_s3_with_https_proxy_list_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node_no_proxy") + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1", "proxy2"], "https", "proxy_list_node_no_proxy" + ) def test_s3_with_https_env_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "env_node_no_proxy") + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1"], "https", "env_node_no_proxy" + ) def test_s3_with_https_remote_no_proxy(cluster): - proxy_util.simple_test_assert_no_proxy(cluster, ["proxy1"], "https", "remote_proxy_node_no_proxy") + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1"], "https", "remote_proxy_node_no_proxy" + ) def test_s3_with_https_proxy_list(cluster): From 01ef03f8555e232ed4265824262f8c22b289c7a4 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Mon, 13 May 2024 17:11:14 +0300 Subject: [PATCH 071/856] io_uring: improve resumbits visibility --- src/Common/ProfileEvents.cpp | 3 ++- src/Coordination/KeeperConstants.cpp | 3 ++- src/Disks/IO/IOUringReader.cpp | 12 ++++++++---- 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ed0b29c7b44..34abce6b29e 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -631,7 +631,8 @@ The server successfully detected this situation and will download merged part fr \ M(ServerStartupMilliseconds, "Time elapsed from starting server to listening to sockets in milliseconds")\ M(IOUringSQEsSubmitted, "Total number of io_uring SQEs submitted") \ - M(IOUringSQEsResubmits, "Total number of io_uring SQE resubmits performed") \ + M(IOUringSQEsResubmitsAsync, "Total number of asychronous io_uring SQE resubmits performed") \ + M(IOUringSQEsResubmitsSync, "Total number of synchronous io_uring SQE resubmits performed") \ M(IOUringCQEsCompleted, "Total number of successfully completed io_uring CQEs") \ M(IOUringCQEsFailed, "Total number of completed io_uring CQEs with failures") \ \ diff --git a/src/Coordination/KeeperConstants.cpp b/src/Coordination/KeeperConstants.cpp index 8251dca3d1e..51bf037c1c9 100644 --- a/src/Coordination/KeeperConstants.cpp +++ b/src/Coordination/KeeperConstants.cpp @@ -258,7 +258,8 @@ M(KeeperExistsRequest) \ \ M(IOUringSQEsSubmitted) \ - M(IOUringSQEsResubmits) \ + M(IOUringSQEsResubmitsAsync) \ + M(IOUringSQEsResubmitsSync) \ M(IOUringCQEsCompleted) \ M(IOUringCQEsFailed) \ \ diff --git a/src/Disks/IO/IOUringReader.cpp b/src/Disks/IO/IOUringReader.cpp index 90a4d285ecb..ba8c5b94420 100644 --- a/src/Disks/IO/IOUringReader.cpp +++ b/src/Disks/IO/IOUringReader.cpp @@ -22,7 +22,8 @@ namespace ProfileEvents extern const Event AsynchronousReaderIgnoredBytes; extern const Event IOUringSQEsSubmitted; - extern const Event IOUringSQEsResubmits; + extern const Event IOUringSQEsResubmitsAsync; + extern const Event IOUringSQEsResubmitsSync; extern const Event IOUringCQEsCompleted; extern const Event IOUringCQEsFailed; } @@ -149,10 +150,12 @@ int IOUringReader::submitToRing(EnqueuedRequest & enqueued) io_uring_prep_read(sqe, fd, request.buf, static_cast(request.size - enqueued.bytes_read), request.offset + enqueued.bytes_read); int ret = 0; - do + ret = io_uring_submit(&ring); + while (ret == -EINTR || ret == -EAGAIN) { + ProfileEvents::increment(ProfileEvents::IOUringSQEsResubmitsSync); ret = io_uring_submit(&ring); - } while (ret == -EINTR || ret == -EAGAIN); + } if (ret > 0 && !enqueued.resubmitting) { @@ -266,7 +269,7 @@ void IOUringReader::monitorRing() if (cqe->res == -EAGAIN || cqe->res == -EINTR) { enqueued.resubmitting = true; - ProfileEvents::increment(ProfileEvents::IOUringSQEsResubmits); + ProfileEvents::increment(ProfileEvents::IOUringSQEsResubmitsAsync); ret = submitToRing(enqueued); if (ret <= 0) @@ -310,6 +313,7 @@ void IOUringReader::monitorRing() // potential short read, re-submit enqueued.resubmitting = true; enqueued.bytes_read += bytes_read; + ProfileEvents::increment(ProfileEvents::IOUringSQEsResubmitsAsync); ret = submitToRing(enqueued); if (ret <= 0) From 19854af77f8a35de9deea5ca0f6eb50316a213cf Mon Sep 17 00:00:00 2001 From: tomershafir Date: Mon, 13 May 2024 17:49:00 +0300 Subject: [PATCH 072/856] fix typo --- src/Common/ProfileEvents.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 34abce6b29e..87b7c2cce4d 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -631,7 +631,7 @@ The server successfully detected this situation and will download merged part fr \ M(ServerStartupMilliseconds, "Time elapsed from starting server to listening to sockets in milliseconds")\ M(IOUringSQEsSubmitted, "Total number of io_uring SQEs submitted") \ - M(IOUringSQEsResubmitsAsync, "Total number of asychronous io_uring SQE resubmits performed") \ + M(IOUringSQEsResubmitsAsync, "Total number of asynchronous io_uring SQE resubmits performed") \ M(IOUringSQEsResubmitsSync, "Total number of synchronous io_uring SQE resubmits performed") \ M(IOUringCQEsCompleted, "Total number of successfully completed io_uring CQEs") \ M(IOUringCQEsFailed, "Total number of completed io_uring CQEs with failures") \ From 927ba761a31bf1bbe87d99db7b11c52efee18a40 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 13 May 2024 16:33:38 +0000 Subject: [PATCH 073/856] Try to fix flaky s3 tests test_seekable_formats and test_seekable_formats_url --- tests/integration/helpers/test_tools.py | 2 +- tests/integration/test_storage_s3/test.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index 2afbae340be..d605d3ac2d3 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -139,7 +139,7 @@ def assert_logs_contain_with_retry(instance, substring, retry_count=20, sleep_ti def exec_query_with_retry( - instance, query, retry_count=40, sleep_time=0.5, silent=False, settings={} + instance, query, retry_count=40, sleep_time=0.5, silent=False, settings={}, timeout=30 ): exception = None for cnt in range(retry_count): diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index dc929b7db46..6ca11eaa17a 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1133,6 +1133,7 @@ def test_seekable_formats(started_cluster): exec_query_with_retry( instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", + timeout=100, ) result = instance.query(f"SELECT count() FROM {table_function}") @@ -1142,6 +1143,7 @@ def test_seekable_formats(started_cluster): exec_query_with_retry( instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", + timeout=100, ) result = instance.query( @@ -1169,6 +1171,7 @@ def test_seekable_formats_url(started_cluster): exec_query_with_retry( instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", + timeout=100, ) result = instance.query(f"SELECT count() FROM {table_function}") @@ -1178,6 +1181,7 @@ def test_seekable_formats_url(started_cluster): exec_query_with_retry( instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", + timeout=100, ) table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_parquet', 'Parquet', 'a Int32, b String')" From a323dd1a787d5dbf6ad1a35e10cd17966f7005bd Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 13 May 2024 16:41:04 +0000 Subject: [PATCH 074/856] Propagate new timeout properly --- tests/integration/helpers/test_tools.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index d605d3ac2d3..efdf84cbba9 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -144,7 +144,7 @@ def exec_query_with_retry( exception = None for cnt in range(retry_count): try: - res = instance.query(query, timeout=30, settings=settings) + res = instance.query(query, timeout=timeout, settings=settings) if not silent: logging.debug(f"Result of {query} on {cnt} try is {res}") break From d833c9cce05cf508596ef5191a9ee179c59a1c6f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 13 May 2024 16:45:38 +0000 Subject: [PATCH 075/856] 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 076/856] 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 077/856] 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 078/856] 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 95f65cf4220318cc85b94b29588aaa8b04bc0221 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 14 May 2024 16:58:05 +0000 Subject: [PATCH 079/856] Use right comments for system tables (cherry picked from commit 6c5a35d08943c1492cbf7ee861130ce80541879a) --- .../table-engines/integrations/s3queue.md | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 +- .../02117_show_create_table_system.reference | 128 +++++++++--------- .../02118_show_create_table_rocksdb.reference | 2 +- 4 files changed, 67 insertions(+), 67 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 8ebab80423f..deb915f2060 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -268,7 +268,7 @@ For introspection use `system.s3queue` stateless table and `system.s3queue_log` `exception` String ) ENGINE = SystemS3Queue -COMMENT 'SYSTEM TABLE is built on the fly.' │ +COMMENT 'Contains in-memory state of S3Queue metadata and currently processed rows per file.' │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 674e9afa8ac..ac45e5be7dc 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -785,7 +785,7 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromStorage(const String & table_name, cons throw_on_error); create_table_query->set(create_table_query->as()->comment, - std::make_shared("SYSTEM TABLE is built on the fly.")); + std::make_shared(storage->getInMemoryMetadata().comment)); return create_table_query; } diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index d625feb94d3..d15c1545135 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -4,7 +4,7 @@ CREATE TABLE system.aggregate_function_combinators `is_internal` UInt8 ) ENGINE = SystemAggregateFunctionCombinators -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available aggregate function combinators, which could be applied to aggregate functions and change the way they work.' CREATE TABLE system.asynchronous_inserts ( `query` String, @@ -17,7 +17,7 @@ CREATE TABLE system.asynchronous_inserts `entries.bytes` Array(UInt64) ) ENGINE = SystemAsynchronousInserts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about pending asynchronous inserts in queue in server\'s memory.' CREATE TABLE system.asynchronous_metrics ( `metric` String, @@ -25,14 +25,14 @@ CREATE TABLE system.asynchronous_metrics `description` String ) ENGINE = SystemAsynchronousMetrics -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains metrics that are calculated periodically in the background. For example, the amount of RAM in use.' CREATE TABLE system.build_options ( `name` String, `value` String ) ENGINE = SystemBuildOptions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all build flags, compiler options and commit hash for used build.' CREATE TABLE system.clusters ( `cluster` String, @@ -55,14 +55,14 @@ CREATE TABLE system.clusters `name` String ALIAS cluster ) ENGINE = SystemClusters -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about clusters defined in the configuration file or generated by a Replicated database.' CREATE TABLE system.collations ( `name` String, `language` Nullable(String) ) ENGINE = SystemTableCollations -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available collations for alphabetical comparison of strings.' CREATE TABLE system.columns ( `database` String, @@ -88,13 +88,13 @@ CREATE TABLE system.columns `datetime_precision` Nullable(UInt64) ) ENGINE = SystemColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all columns from all tables of the current server.' CREATE TABLE system.contributors ( `name` String ) ENGINE = SystemContributors -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all ClickHouse contributors <3' CREATE TABLE system.current_roles ( `role_name` String, @@ -102,7 +102,7 @@ CREATE TABLE system.current_roles `is_default` UInt8 ) ENGINE = SystemCurrentRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains active roles of a current user. SET ROLE changes the contents of this table.' CREATE TABLE system.data_skipping_indices ( `database` String, @@ -117,7 +117,7 @@ CREATE TABLE system.data_skipping_indices `marks` UInt64 ) ENGINE = SystemDataSkippingIndices -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains all the information about all the data skipping indices in tables, similar to system.columns.' CREATE TABLE system.data_type_families ( `name` String, @@ -125,7 +125,7 @@ CREATE TABLE system.data_type_families `alias_to` String ) ENGINE = SystemTableDataTypeFamilies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available native data types along with all the aliases used for compatibility with other DBMS.' CREATE TABLE system.databases ( `name` String, @@ -138,7 +138,7 @@ CREATE TABLE system.databases `database` String ALIAS name ) ENGINE = SystemDatabases -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all databases of the current server.' CREATE TABLE system.detached_parts ( `database` String, @@ -155,7 +155,7 @@ CREATE TABLE system.detached_parts `level` Nullable(UInt32) ) ENGINE = SystemDetachedParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all parts which are being found in /detached directory along with a reason why it was detached. ClickHouse server doesn\'t use such parts anyhow.' CREATE TABLE system.dictionaries ( `database` String, @@ -185,7 +185,7 @@ CREATE TABLE system.dictionaries `comment` String ) ENGINE = SystemDictionaries -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about dictionaries.' CREATE TABLE system.disks ( `name` String, @@ -205,7 +205,7 @@ CREATE TABLE system.disks `cache_path` String ) ENGINE = SystemDisks -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about disks defined in the server configuration.' CREATE TABLE system.distributed_ddl_queue ( `entry` String, @@ -225,7 +225,7 @@ CREATE TABLE system.distributed_ddl_queue `query_duration_ms` Nullable(UInt64) ) ENGINE = SystemDDLWorkerQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about distributed DDL queries (ON CLUSTER clause) that were executed on a cluster.' CREATE TABLE system.distribution_queue ( `database` String, @@ -241,7 +241,7 @@ CREATE TABLE system.distribution_queue `last_exception_time` DateTime ) ENGINE = SystemDistributionQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about local files that are in the queue to be sent to the shards. These local files contain new parts that are created by inserting new data into the Distributed table in asynchronous mode.' CREATE TABLE system.enabled_roles ( `role_name` String, @@ -250,7 +250,7 @@ CREATE TABLE system.enabled_roles `is_default` UInt8 ) ENGINE = SystemEnabledRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains all active roles at the moment, including current role of the current user and granted roles for current role.' CREATE TABLE system.errors ( `name` String, @@ -262,7 +262,7 @@ CREATE TABLE system.errors `remote` UInt8 ) ENGINE = SystemErrors -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all errors which have ever happened including the error code, last time and message with unsymbolized stacktrace.' CREATE TABLE system.events ( `event` String, @@ -271,7 +271,7 @@ CREATE TABLE system.events `name` String ALIAS event ) ENGINE = SystemEvents -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains profiling events and their current value.' CREATE TABLE system.formats ( `name` String, @@ -281,7 +281,7 @@ CREATE TABLE system.formats `supports_parallel_formatting` UInt8 ) ENGINE = SystemFormats -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all the formats along with flags whether a format is suitable for input/output or whether it supports parallelization.' CREATE TABLE system.functions ( `name` String, @@ -299,7 +299,7 @@ CREATE TABLE system.functions `categories` String ) ENGINE = SystemFunctions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available ordinary and aggregate functions with their descriptions.' CREATE TABLE system.graphite_retentions ( `config_name` String, @@ -314,7 +314,7 @@ CREATE TABLE system.graphite_retentions `Tables.table` Array(String) ) ENGINE = SystemGraphite -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about parameters graphite_rollup which are used in tables with *GraphiteMergeTree engines.' CREATE TABLE system.licenses ( `library_name` String, @@ -323,14 +323,14 @@ CREATE TABLE system.licenses `license_text` String ) ENGINE = SystemLicenses -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains licenses of third-party libraries that are located in the contrib directory of ClickHouse sources.' CREATE TABLE system.macros ( `macro` String, `substitution` String ) ENGINE = SystemMacros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all macros defined in server configuration.' CREATE TABLE system.merge_tree_settings ( `name` String, @@ -344,7 +344,7 @@ CREATE TABLE system.merge_tree_settings `is_obsolete` UInt8 ) ENGINE = SystemMergeTreeSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all MergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query.' CREATE TABLE system.merges ( `database` String, @@ -373,7 +373,7 @@ CREATE TABLE system.merges `merge_algorithm` String ) ENGINE = SystemMerges -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row.' CREATE TABLE system.metrics ( `metric` String, @@ -382,7 +382,7 @@ CREATE TABLE system.metrics `name` String ALIAS metric ) ENGINE = SystemMetrics -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains metrics which can be calculated instantly, or have a current value. For example, the number of simultaneously processed queries or the current replica delay. This table is always up to date.' CREATE TABLE system.moves ( `database` String, @@ -395,7 +395,7 @@ CREATE TABLE system.moves `thread_id` UInt64 ) ENGINE = SystemMoves -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row.' CREATE TABLE system.mutations ( `database` String, @@ -414,25 +414,25 @@ CREATE TABLE system.mutations `latest_fail_reason` String ) ENGINE = SystemMutations -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of mutations and their progress. Each mutation command is represented by a single row.' CREATE TABLE system.numbers ( `number` UInt64 ) ENGINE = SystemNumbers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.' CREATE TABLE system.numbers_mt ( `number` UInt64 ) ENGINE = SystemNumbers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Multithreaded version of `system.numbers`. Numbers order is not guaranteed.' CREATE TABLE system.one ( `dummy` UInt8 ) ENGINE = SystemOne -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.' CREATE TABLE system.part_moves_between_shards ( `database` String, @@ -451,7 +451,7 @@ CREATE TABLE system.part_moves_between_shards `last_exception` String ) ENGINE = SystemShardMoves -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about parts which are currently in a process of moving between shards and their progress.' CREATE TABLE system.parts ( `partition` String, @@ -522,7 +522,7 @@ CREATE TABLE system.parts `part_name` String ALIAS name ) ENGINE = SystemParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently existing (both active and inactive) parts of all *-MergeTree tables. Each part is represented by a single row.' CREATE TABLE system.parts_columns ( `partition` String, @@ -582,7 +582,7 @@ CREATE TABLE system.parts_columns `part_name` String ALIAS name ) ENGINE = SystemPartsColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of columns of all currently existing parts of all MergeTree tables. Each column is represented by a single row.' CREATE TABLE system.processes ( `is_initial_query` UInt8, @@ -630,7 +630,7 @@ CREATE TABLE system.processes `Settings.Values` Array(String) ALIAS mapValues(Settings) ) ENGINE = SystemProcesses -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently executing processes (queries) with their progress.' CREATE TABLE system.projection_parts ( `partition` String, @@ -698,7 +698,7 @@ CREATE TABLE system.projection_parts `part_name` String ALIAS name ) ENGINE = SystemProjectionParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently existing projection parts (a copy of some part containing aggregated data or just sorted in different order) created for all the projections for all tables within a cluster.' CREATE TABLE system.projection_parts_columns ( `partition` String, @@ -754,7 +754,7 @@ CREATE TABLE system.projection_parts_columns `part_name` String ALIAS name ) ENGINE = SystemProjectionPartsColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of columns of all currently existing projection parts of all MergeTree tables. Each column is represented by a single row.' CREATE TABLE system.quota_limits ( `quota_name` String, @@ -773,7 +773,7 @@ CREATE TABLE system.quota_limits `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotaLimits -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about maximums for all intervals of all quotas. Any number of rows or zero can correspond to specific quota.' CREATE TABLE system.quota_usage ( `quota_name` String, @@ -805,7 +805,7 @@ CREATE TABLE system.quota_usage `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotaUsage -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains quota usage by the current user: how much is used and how much is left.' CREATE TABLE system.quotas ( `name` String, @@ -818,7 +818,7 @@ CREATE TABLE system.quotas `apply_to_except` Array(String) ) ENGINE = SystemQuotas -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about quotas.' CREATE TABLE system.quotas_usage ( `quota_name` String, @@ -851,7 +851,7 @@ CREATE TABLE system.quotas_usage `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotasUsage -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains quota usage by all users.' CREATE TABLE system.replicas ( `database` String, @@ -891,7 +891,7 @@ CREATE TABLE system.replicas `replica_is_active` Map(String, UInt8) ) ENGINE = SystemReplicas -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information and status of all table replicas on current server. Each replica is represented by a single row.' CREATE TABLE system.replicated_fetches ( `database` String, @@ -912,7 +912,7 @@ CREATE TABLE system.replicated_fetches `thread_id` UInt64 ) ENGINE = SystemReplicatedFetches -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about currently running background fetches.' CREATE TABLE system.replicated_merge_tree_settings ( `name` String, @@ -926,7 +926,7 @@ CREATE TABLE system.replicated_merge_tree_settings `is_obsolete` UInt8 ) ENGINE = SystemReplicatedMergeTreeSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all ReplicatedMergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query. ' CREATE TABLE system.replication_queue ( `database` String, @@ -952,7 +952,7 @@ CREATE TABLE system.replication_queue `merge_type` String ) ENGINE = SystemReplicationQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about tasks from replication queues stored in ClickHouse Keeper, or ZooKeeper, for each table replica.' CREATE TABLE system.role_grants ( `user_name` Nullable(String), @@ -963,7 +963,7 @@ CREATE TABLE system.role_grants `with_admin_option` UInt8 ) ENGINE = SystemRoleGrants -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains the role grants for users and roles. To add entries to this table, use `GRANT role TO user`. Using this table you may find out which roles are assigned to which users or which roles a user has.' CREATE TABLE system.roles ( `name` String, @@ -971,7 +971,7 @@ CREATE TABLE system.roles `storage` String ) ENGINE = SystemRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all roles created at the server.' CREATE TABLE system.row_policies ( `name` String, @@ -987,7 +987,7 @@ CREATE TABLE system.row_policies `apply_to_except` Array(String) ) ENGINE = SystemRowPolicies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains filters for one particular table, as well as a list of roles and/or users which should use this row policy.' CREATE TABLE system.settings ( `name` String, @@ -1003,7 +1003,7 @@ CREATE TABLE system.settings `is_obsolete` UInt8 ) ENGINE = SystemSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all user-level settings (which can be modified in a scope of query or session), their current and default values along with descriptions.' CREATE TABLE system.settings_profile_elements ( `profile_name` Nullable(String), @@ -1018,7 +1018,7 @@ CREATE TABLE system.settings_profile_elements `inherit_profile` Nullable(String) ) ENGINE = SystemSettingsProfileElements -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Describes the content of each settings profile configured on the server. Including settings constraints, roles and users for which the settings are applied, and parent settings profiles.' CREATE TABLE system.settings_profiles ( `name` String, @@ -1030,7 +1030,7 @@ CREATE TABLE system.settings_profiles `apply_to_except` Array(String) ) ENGINE = SystemSettingsProfiles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains properties of configured setting profiles.' CREATE TABLE system.stack_trace ( `thread_name` String, @@ -1039,7 +1039,7 @@ CREATE TABLE system.stack_trace `trace` Array(UInt64) ) ENGINE = SystemStackTrace -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Allows to obtain an unsymbolized stacktrace from all the threads of the server process.' CREATE TABLE system.storage_policies ( `policy_name` String, @@ -1054,7 +1054,7 @@ CREATE TABLE system.storage_policies `load_balancing` Enum8('ROUND_ROBIN' = 0, 'LEAST_USED' = 1) ) ENGINE = SystemStoragePolicies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about storage policies and volumes defined in the server configuration.' CREATE TABLE system.table_engines ( `name` String, @@ -1068,7 +1068,7 @@ CREATE TABLE system.table_engines `supports_parallel_insert` UInt8 ) ENGINE = SystemTableEngines -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available table engines along with information whether a particular table engine supports some specific features (e.g. settings, skipping indices, projections, replication, TTL, deduplication, parallel insert, etc.)' CREATE TABLE system.table_functions ( `name` String, @@ -1076,7 +1076,7 @@ CREATE TABLE system.table_functions `allow_readonly` UInt8 ) ENGINE = SystemTableFunctions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available table functions with their descriptions.' CREATE TABLE system.tables ( `database` String, @@ -1115,13 +1115,13 @@ CREATE TABLE system.tables `table` String ALIAS name ) ENGINE = SystemTables -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all tables of the current server.' CREATE TABLE system.time_zones ( `time_zone` String ) ENGINE = SystemTimeZones -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of time zones that are supported by the ClickHouse server. This list of timezones might vary depending on the version of ClickHouse.' CREATE TABLE system.user_directories ( `name` String, @@ -1130,7 +1130,7 @@ CREATE TABLE system.user_directories `precedence` UInt64 ) ENGINE = SystemUserDirectories -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains the information about configured user directories - directories on the file system from which ClickHouse server is allowed to read user provided data.' CREATE TABLE system.users ( `name` String, @@ -1151,22 +1151,22 @@ CREATE TABLE system.users `default_database` String ) ENGINE = SystemUsers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all users profiles either configured at the server through a configuration file or created via SQL.' CREATE TABLE system.warnings ( `message` String ) ENGINE = SystemWarnings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains warnings about server configuration to be displayed by clickhouse-client right after it connects to the server.' CREATE TABLE system.zeros ( `zero` UInt8 ) ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Produces unlimited number of non-materialized zeros.' CREATE TABLE system.zeros_mt ( `zero` UInt8 ) ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Multithreaded version of system.zeros.' diff --git a/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference b/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference index 848abb332bb..e1e1ab3f441 100644 --- a/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference +++ b/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference @@ -1 +1 @@ -CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB\nCOMMENT \'Contains a list of metrics exposed from embedded RocksDB.\' From de5ba956394d0b7775eacf4461e501e8985a76e1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 14 May 2024 19:22:49 +0000 Subject: [PATCH 080/856] Rewrite plan for parallel replicas in planner --- .../ClusterProxy/executeQuery.cpp | 39 ++++++- src/Interpreters/ClusterProxy/executeQuery.h | 25 ++++- src/Interpreters/executeQuery.cpp | 3 +- src/Planner/PlannerJoinTree.cpp | 101 ++++++++++++------ src/Processors/QueryPlan/QueryPlan.cpp | 6 +- src/Processors/QueryPlan/QueryPlan.h | 1 - src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 8 files changed, 140 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e4d979d4078..17a678ef9b8 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -22,7 +22,9 @@ #include #include #include - +#include +#include +#include namespace DB { @@ -477,6 +479,41 @@ void executeQueryWithParallelReplicas( query_plan.addStep(std::move(read_from_remote)); } +void executeQueryWithParallelReplicas( + QueryPlan & query_plan, + const StorageID & storage_id, + QueryProcessingStage::Enum processed_stage, + const QueryTreeNodePtr & query_tree, + const PlannerContextPtr & planner_context, + ContextPtr context, + std::shared_ptr storage_limits) +{ + QueryTreeNodePtr modified_query_tree = query_tree->clone(); + rewriteJoinToGlobalJoin(modified_query_tree, context); + modified_query_tree = buildQueryTreeForShard(planner_context, modified_query_tree); + + auto header + = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_tree, context, SelectQueryOptions(processed_stage).analyze()); + auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); + + executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits); +} + +void executeQueryWithParallelReplicas( + QueryPlan & query_plan, + const StorageID & storage_id, + QueryProcessingStage::Enum processed_stage, + const ASTPtr & query_ast, + ContextPtr context, + std::shared_ptr storage_limits) +{ + auto modified_query_ast = ClusterProxy::rewriteSelectQuery( + context, query_ast, storage_id.database_name, storage_id.table_name, /*remote_table_function_ptr*/ nullptr); + auto header = InterpreterSelectQuery(modified_query_ast, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + + executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits); +} + } } diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 582f8d74fd5..c01fdb678ff 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -24,6 +24,12 @@ struct StorageID; struct StorageLimits; using StorageLimitsList = std::list; +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + +class PlannerContext; +using PlannerContextPtr = std::shared_ptr; + namespace ClusterProxy { @@ -65,7 +71,7 @@ void executeQuery( const DistributedSettings & distributed_settings, AdditionalShardFilterGenerator shard_filter_generator); - +/// move to cpp this one void executeQueryWithParallelReplicas( QueryPlan & query_plan, const StorageID & storage_id, @@ -74,6 +80,23 @@ void executeQueryWithParallelReplicas( const ASTPtr & query_ast, ContextPtr context, std::shared_ptr storage_limits); + +void executeQueryWithParallelReplicas( + QueryPlan & query_plan, + const StorageID & storage_id, + QueryProcessingStage::Enum processed_stage, + const ASTPtr & query_ast, + ContextPtr context, + std::shared_ptr storage_limits); + +void executeQueryWithParallelReplicas( + QueryPlan & query_plan, + const StorageID & storage_id, + QueryProcessingStage::Enum processed_stage, + const QueryTreeNodePtr & query_tree, + const PlannerContextPtr & planner_context, + ContextPtr context, + std::shared_ptr storage_limits); } } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 33a4cf2a74c..15d37c381aa 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1184,7 +1184,9 @@ static std::tuple executeQueryImpl( } if (auto * create_interpreter = typeid_cast(&*interpreter)) + { create_interpreter->setIsRestoreFromBackup(flags.distributed_backup_restore); + } { std::unique_ptr span; @@ -1250,7 +1252,6 @@ static std::tuple executeQueryImpl( } } } - } } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1b2a55a50b0..42374cd1cdf 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include @@ -56,6 +57,7 @@ #include #include #include +#include #include #include @@ -767,37 +769,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { if (!select_query_options.only_analyze) { - auto storage_merge_tree = std::dynamic_pointer_cast(storage); - if (storage_merge_tree && query_context->canUseParallelReplicasOnInitiator() - && settings.parallel_replicas_min_number_of_rows_per_replica > 0) - { - UInt64 rows_to_read - = storage_merge_tree->estimateNumberOfRowsToRead(query_context, storage_snapshot, table_expression_query_info); - - if (max_block_size_limited && (max_block_size_limited < rows_to_read)) - rows_to_read = max_block_size_limited; - - size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; - LOG_TRACE( - getLogger("Planner"), - "Estimated {} rows to read. It is enough work for {} parallel replicas", - rows_to_read, - number_of_replicas_to_use); - - if (number_of_replicas_to_use <= 1) - { - planner_context->getMutableQueryContext()->setSetting( - "allow_experimental_parallel_reading_from_replicas", Field(0)); - planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", UInt64{1}); - LOG_DEBUG(getLogger("Planner"), "Disabling parallel replicas because there aren't enough rows to read"); - } - else if (number_of_replicas_to_use < settings.max_parallel_replicas) - { - planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", number_of_replicas_to_use); - LOG_DEBUG(getLogger("Planner"), "Reducing the number of replicas to use to {}", number_of_replicas_to_use); - } - } - auto & prewhere_info = table_expression_query_info.prewhere_info; const auto & prewhere_actions = table_expression_data.getPrewhereFilterActions(); @@ -895,6 +866,74 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres max_block_size, max_streams); + if (storage->isMergeTree() && query_context->canUseParallelReplicasOnInitiator()) + { + // (1) find read step + QueryPlan::Node * node = query_plan.getRootNode(); + ReadFromMergeTree * reading = nullptr; + while (node) + { + IQueryPlanStep * step = typeid_cast(node->step.get()); + if (step) + { + reading = typeid_cast(step); + break; + } + + if (!node->children.empty()) + node = node->children.at(0); + } + + // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read + if (reading && settings.parallel_replicas_min_number_of_rows_per_replica > 0) + { + auto result_ptr = reading->selectRangesToRead(reading->getParts(), reading->getAlterConvertionsForParts()); + + UInt64 rows_to_read = result_ptr->selected_rows; + if (table_expression_query_info.limit > 0 && table_expression_query_info.limit < rows_to_read) + rows_to_read = table_expression_query_info.limit; + + if (max_block_size_limited && (max_block_size_limited < rows_to_read)) + rows_to_read = max_block_size_limited; + + const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; + if (number_of_replicas_to_use > 1) + LOG_TRACE( + getLogger("Planner"), + "Estimated {} rows to read. It is enough work for {} parallel replicas", + rows_to_read, + number_of_replicas_to_use); + + if (number_of_replicas_to_use <= 1) + { + planner_context->getMutableQueryContext()->setSetting( + "allow_experimental_parallel_reading_from_replicas", Field(0)); + planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", UInt64{1}); + LOG_DEBUG(getLogger("Planner"), "Disabling parallel replicas because there aren't enough rows to read"); + } + else if (number_of_replicas_to_use < settings.max_parallel_replicas) + { + planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", number_of_replicas_to_use); + LOG_DEBUG(getLogger("Planner"), "Reducing the number of replicas to use to {}", number_of_replicas_to_use); + } + } + + // (3) if parallel replicas still enabled - replace reading step + if (planner_context->getQueryContext()->canUseParallelReplicasOnInitiator()) + { + QueryPlan query_plan_parallel_replicas; + ClusterProxy::executeQueryWithParallelReplicas( + query_plan_parallel_replicas, + storage->getStorageID(), + from_stage, + table_expression_query_info.query_tree, + table_expression_query_info.planner_context, + query_context, + table_expression_query_info.storage_limits); + query_plan = std::move(query_plan_parallel_replicas); + } + } + const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns) { diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 0fae7e8df4d..a6597b37e4b 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -496,6 +496,8 @@ static void updateDataStreams(QueryPlan::Node & root) void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings) { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "\n{}", StackTrace().toString()); + /// optimization need to be applied before "mergeExpressions" optimization /// it removes redundant sorting steps, but keep underlying expressions, /// so "mergeExpressions" optimization handles them afterwards @@ -520,10 +522,6 @@ void QueryPlan::explainEstimate(MutableColumns & columns) UInt64 parts = 0; UInt64 rows = 0; UInt64 marks = 0; - - EstimateCounters(const std::string & database, const std::string & table) : database_name(database), table_name(table) - { - } }; using CountersPtr = std::shared_ptr; diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index bf135ba3cd6..75c577af24e 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -7,7 +7,6 @@ #include #include -#include #include namespace DB diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b24549df757..b33aabc7bf5 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -213,7 +213,9 @@ void StorageMergeTree::read( size_t max_block_size, size_t num_streams) { - if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) + const auto & settings = local_context->getSettingsRef(); + if (local_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_for_non_replicated_merge_tree + && !settings.allow_experimental_analyzer) { ASTPtr modified_query_ast; Block header; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cc6599f8cd1..d08c3be30d7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5447,7 +5447,7 @@ void StorageReplicatedMergeTree::read( /// For this you have to synchronously go to ZooKeeper. if (settings.select_sequential_consistency) readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); - else if (local_context->canUseParallelReplicasOnInitiator()) + else if (local_context->canUseParallelReplicasOnInitiator() && !settings.allow_experimental_analyzer) readParallelReplicasImpl(query_plan, column_names, query_info, local_context, processed_stage); else readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); From 6faa4dd20c7d9182f21a4f9f1c6cc1e6a83258cd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 15 May 2024 11:19:03 +0000 Subject: [PATCH 081/856] Fix --- src/Planner/PlannerJoinTree.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 42374cd1cdf..158bcd94380 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -873,19 +873,18 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres ReadFromMergeTree * reading = nullptr; while (node) { - IQueryPlanStep * step = typeid_cast(node->step.get()); - if (step) - { - reading = typeid_cast(step); + reading = typeid_cast(node->step.get()); + if (reading) break; - } if (!node->children.empty()) node = node->children.at(0); } + chassert(reading); + // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read - if (reading && settings.parallel_replicas_min_number_of_rows_per_replica > 0) + if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) { auto result_ptr = reading->selectRangesToRead(reading->getParts(), reading->getAlterConvertionsForParts()); From 61a6a27325bc5ded27d3c3fb05010adf5626cd83 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 15 May 2024 14:39:45 +0200 Subject: [PATCH 082/856] Fix check black --- tests/integration/helpers/test_tools.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index efdf84cbba9..1c8c5c33a13 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -139,7 +139,13 @@ def assert_logs_contain_with_retry(instance, substring, retry_count=20, sleep_ti def exec_query_with_retry( - instance, query, retry_count=40, sleep_time=0.5, silent=False, settings={}, timeout=30 + instance, + query, + retry_count=40, + sleep_time=0.5, + silent=False, + settings={}, + timeout=30, ): exception = None for cnt in range(retry_count): From ff01ca80b4ae644adc396c0b94d3b54ac2b8c1e6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 15 May 2024 16:37:42 +0200 Subject: [PATCH 083/856] reduce retires time for queries, increase retries count for backups --- src/Backups/BackupIO_S3.cpp | 6 +++--- src/Core/Settings.h | 1 + src/IO/S3/Client.h | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 15860363615..baa16a269a9 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -54,9 +54,9 @@ namespace S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings.region, context->getRemoteHostFilter(), - static_cast(global_settings.s3_max_redirects), - static_cast(global_settings.s3_retry_attempts), - global_settings.enable_s3_requests_logging, + static_cast(local_settings.s3_max_redirects), + static_cast(local_settings.backup_restore_s3_retry_attempts), + local_settings.enable_s3_requests_logging, /* for_disk_s3 = */ false, request_settings.get_request_throttler, request_settings.put_request_throttler, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a0de354a03..292d2aa72d5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -506,6 +506,7 @@ class IColumn; M(UInt64, backup_restore_keeper_value_max_size, 1048576, "Maximum size of data of a [Zoo]Keeper's node during backup", 0) \ M(UInt64, backup_restore_batch_size_for_keeper_multiread, 10000, "Maximum size of batch for multiread request to [Zoo]Keeper during backup or restore", 0) \ M(UInt64, backup_restore_batch_size_for_keeper_multi, 1000, "Maximum size of batch for multi request to [Zoo]Keeper during backup or restore", 0) \ + M(UInt64, backup_restore_s3_retry_attempts, 1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore.", 0) \ M(UInt64, max_backup_bandwidth, 0, "The maximum read speed in bytes per second for particular backup on server. Zero means unlimited.", 0) \ \ M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \ diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index bd281846343..1e90acb7f7b 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -162,7 +162,7 @@ public: class RetryStrategy : public Aws::Client::RetryStrategy { public: - explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000); + explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 5000); /// NOLINTNEXTLINE(google-runtime-int) bool ShouldRetry(const Aws::Client::AWSError& error, long attemptedRetries) const override; From 0619b0921f195951b8e72c02dcc0ad06094811b2 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 15 May 2024 15:56:24 +0000 Subject: [PATCH 084/856] 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 085/856] 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 5a09efc6da1ca07ea00f686cdd6ee280974e1bce Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 15 May 2024 19:48:51 +0000 Subject: [PATCH 086/856] Fix tests with non-replicated merge tree --- src/Planner/PlannerJoinTree.cpp | 15 ++++++++++++++- src/Storages/IStorage.h | 1 - .../02764_parallel_replicas_plain_merge_tree.sql | 5 +++-- 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 158bcd94380..6ec186cccbb 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -49,6 +49,7 @@ #include #include +#include #include #include @@ -866,7 +867,19 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres max_block_size, max_streams); - if (storage->isMergeTree() && query_context->canUseParallelReplicasOnInitiator()) + auto parallel_replicas_enabled_for_storage = [](const StoragePtr & table, const Settings & query_settings) + { + if (!table->isMergeTree()) + return false; + + if (std::dynamic_pointer_cast(table) + && !query_settings.parallel_replicas_for_non_replicated_merge_tree) + return false; + + return true; + }; + + if (parallel_replicas_enabled_for_storage(storage, settings) && query_context->canUseParallelReplicasOnInitiator()) { // (1) find read step QueryPlan::Node * node = query_plan.getRootNode(); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 87a04c3fcc6..63b59b1049c 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -20,7 +20,6 @@ #include #include -#include namespace DB diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql index 9caa6f76e89..36e811c8ef5 100644 --- a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql +++ b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql @@ -1,4 +1,5 @@ -CREATE TABLE IF NOT EXISTS parallel_replicas_plain (x String) ENGINE=MergeTree() ORDER BY x; +DROP TABLE IF EXISTS parallel_replicas_plain; +CREATE TABLE parallel_replicas_plain (x String) ENGINE=MergeTree() ORDER BY x; INSERT INTO parallel_replicas_plain SELECT toString(number) FROM numbers(10); SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, cluster_for_parallel_replicas='parallel_replicas'; @@ -13,4 +14,4 @@ SET parallel_replicas_for_non_replicated_merge_tree = 1; SELECT x FROM parallel_replicas_plain LIMIT 1 FORMAT Null; SELECT max(length(x)) FROM parallel_replicas_plain FORMAT Null; -DROP TABLE IF EXISTS parallel_replicas_plain; +-- DROP TABLE parallel_replicas_plain; From 5bee42e1f74e38e764c74bd26680a6489b8d699a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 16 May 2024 13:18:02 +0200 Subject: [PATCH 087/856] add new setting to settings changes history --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 5f3e9ffb611..775c740443c 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,6 +93,7 @@ static std::map sett {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, {"output_format_pretty_preserve_border_for_multiline_string", 0, 1, "Applies better rendering for multiline strings."}, + {"backup_restore_s3_retry_attempts", 0, 1000, "A new setting."} }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, From 042c36313da07f77d3f2acae09d2ffb858fbfe37 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 16 May 2024 14:42:26 +0200 Subject: [PATCH 088/856] Updated to use required memory and prevent creating extra buffer --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 127 +++++++++++++++--- .../IO/WriteBufferFromAzureBlobStorage.h | 14 +- 2 files changed, 120 insertions(+), 21 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 2c90e3a9003..ebb3fc4a82d 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -23,7 +23,6 @@ struct WriteBufferFromAzureBlobStorage::PartData { Memory<> memory; size_t data_size = 0; - std::string block_id; }; BufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureObjectStorageSettings & settings) @@ -119,22 +118,32 @@ void WriteBufferFromAzureBlobStorage::preFinalize() // This function should not be run again is_prefinalized = true; + hidePartialData(); + + if (hidden_size > 0) + detachBuffer(); + + setFakeBufferWhenPreFinalized(); + /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (buffer_allocation_policy->getBufferNumber() == 1) + if (detached_part_data.size() == 1) { - size_t data_size = size_t(position() - memory.data()); - if (data_size <= max_single_part_upload_size) + if (detached_part_data.front().data_size <= max_single_part_upload_size) { + auto part_data = std::move(detached_part_data.front()); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(memory.data()), data_size); - execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, data_size); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); + execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); + + detached_part_data.pop_front(); + block_ids.clear(); return; } } - writePart(); + writeMultipartUpload(); } void WriteBufferFromAzureBlobStorage::finalizeImpl() @@ -144,9 +153,13 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() if (!is_prefinalized) preFinalize(); + chassert(offset() == 0); + chassert(hidden_size == 0); + + task_tracker->waitAll(); + if (!block_ids.empty()) { - task_tracker->waitAll(); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); @@ -156,13 +169,60 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() void WriteBufferFromAzureBlobStorage::nextImpl() { task_tracker->waitIfAny(); - writePart(); + + hidePartialData(); + + reallocateFirstBuffer(); + + if (available() > 0) + return; + + detachBuffer(); + + if (detached_part_data.size() > 1) + writeMultipartUpload(); + allocateBuffer(); } +void WriteBufferFromAzureBlobStorage::hidePartialData() +{ + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(offset(), ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + + chassert(memory.size() >= hidden_size + offset()); + + hidden_size += offset(); + chassert(memory.data() + hidden_size == working_buffer.begin() + offset()); + chassert(memory.data() + hidden_size == position()); + + WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); + chassert(offset() == 0); +} + +void WriteBufferFromAzureBlobStorage::reallocateFirstBuffer() +{ + chassert(offset() == 0); + + if (buffer_allocation_policy->getBufferNumber() > 1 || available() > 0) + return; + + const size_t max_first_buffer = buffer_allocation_policy->getBufferSize(); + if (memory.size() == max_first_buffer) + return; + + size_t size = std::min(memory.size() * 2, max_first_buffer); + memory.resize(size); + + WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); + chassert(offset() == 0); +} + void WriteBufferFromAzureBlobStorage::allocateBuffer() { buffer_allocation_policy->nextBuffer(); + chassert(0 == hidden_size); + auto size = buffer_allocation_policy->getBufferSize(); if (buffer_allocation_policy->getBufferNumber() == 1) @@ -172,30 +232,57 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() WriteBuffer::set(memory.data(), memory.size()); } -void WriteBufferFromAzureBlobStorage::writePart() +void WriteBufferFromAzureBlobStorage::detachBuffer() { - auto data_size = size_t(position() - memory.data()); + size_t data_size = size_t(position() - memory.data()); if (data_size == 0) return; - const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); - std::shared_ptr part_data = std::make_shared(std::move(memory), data_size, block_id); - WriteBuffer::set(nullptr, 0); + chassert(data_size == hidden_size); - auto upload_worker = [this, part_data] () + auto buf = std::move(memory); + + WriteBuffer::set(nullptr, 0); + total_size += hidden_size; + hidden_size = 0; + + detached_part_data.push_back({std::move(buf), data_size}); + WriteBuffer::set(nullptr, 0); +} + +void WriteBufferFromAzureBlobStorage::writePart(WriteBufferFromAzureBlobStorage::PartData && part_data) +{ + const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); + auto worker_data = std::make_shared>(block_id, std::move(part_data)); + + auto upload_worker = [this, worker_data] () { + auto & data_size = std::get<1>(*worker_data).data_size; + auto & data_block_id = std::get<0>(*worker_data); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data->memory.data()), part_data->data_size); - execWithRetry([&](){ block_blob_client.StageBlock(part_data->block_id, memory_stream); }, max_unexpected_write_error_retries, part_data->data_size); - - if (write_settings.remote_throttler) - write_settings.remote_throttler->add(part_data->data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(std::get<1>(*worker_data).memory.data()), data_size); + execWithRetry([&](){ block_blob_client.StageBlock(data_block_id, memory_stream); }, max_unexpected_write_error_retries, data_size); }; task_tracker->add(std::move(upload_worker)); } +void WriteBufferFromAzureBlobStorage::setFakeBufferWhenPreFinalized() +{ + WriteBuffer::set(fake_buffer_when_prefinalized, sizeof(fake_buffer_when_prefinalized)); +} + +void WriteBufferFromAzureBlobStorage::writeMultipartUpload() +{ + while (!detached_part_data.empty()) + { + writePart(std::move(detached_part_data.front())); + detached_part_data.pop_front(); + } +} + } #endif + diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 96ba6acefff..25d515df8c8 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -48,8 +48,13 @@ public: private: struct PartData; - void writePart(); + void writeMultipartUpload(); + void writePart(PartData && part_data); + void detachBuffer(); + void reallocateFirstBuffer(); void allocateBuffer(); + void hidePartialData(); + void setFakeBufferWhenPreFinalized(); void finalizeImpl() override; void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); @@ -77,9 +82,16 @@ private: MemoryBufferPtr allocateBuffer() const; + char fake_buffer_when_prefinalized[1] = {}; + bool first_buffer=true; + size_t total_size = 0; + size_t hidden_size = 0; + std::unique_ptr task_tracker; + + std::deque detached_part_data; }; } From 6ecbb7be6e9a28cf57bba05ffe03eb74c2722bc8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 16 May 2024 14:24:46 +0000 Subject: [PATCH 089/856] Fix processing stage with analyzer --- src/Storages/MergeTree/MergeTreeData.cpp | 25 +++++++++++-------- ...02811_parallel_replicas_prewhere_count.sql | 3 +-- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fcbb9bb7208..9dc5ed0251d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6983,19 +6983,22 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( const StorageSnapshotPtr &, SelectQueryInfo &) const { - if (query_context->getClientInfo().collaborate_with_initiator) - return QueryProcessingStage::Enum::FetchColumns; - - /// Parallel replicas - if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState) + if (!query_context->getSettingsRef().allow_experimental_analyzer) { - /// ReplicatedMergeTree - if (supportsReplication()) - return QueryProcessingStage::Enum::WithMergeableState; + if (query_context->getClientInfo().collaborate_with_initiator) + return QueryProcessingStage::Enum::FetchColumns; - /// For non-replicated MergeTree we allow them only if parallel_replicas_for_non_replicated_merge_tree is enabled - if (query_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) - return QueryProcessingStage::Enum::WithMergeableState; + /// Parallel replicas + if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState) + { + /// ReplicatedMergeTree + if (supportsReplication()) + return QueryProcessingStage::Enum::WithMergeableState; + + /// For non-replicated MergeTree we allow them only if parallel_replicas_for_non_replicated_merge_tree is enabled + if (query_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) + return QueryProcessingStage::Enum::WithMergeableState; + } } return QueryProcessingStage::Enum::FetchColumns; diff --git a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql index 14edeecf57e..294c1325ba6 100644 --- a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql +++ b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql @@ -10,7 +10,6 @@ SELECT count() FROM users PREWHERE uid > 2000; -- enable parallel replicas but with high rows threshold SET -skip_unavailable_shards=1, allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='parallel_replicas', @@ -20,4 +19,4 @@ parallel_replicas_min_number_of_rows_per_replica=1000; SELECT '-- count() with parallel replicas -------'; SELECT count() FROM users PREWHERE uid > 2000; -DROP TABLE IF EXISTS users; +DROP TABLE users; From fed6c65858f26e31ad8f3d63a2cb6e9a0b404ff7 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 16 May 2024 17:57:01 +0000 Subject: [PATCH 090/856] 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 79cd8039143f27d24887836ab0e2872ccf832224 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 17 May 2024 11:00:30 +0000 Subject: [PATCH 091/856] Fix: set correct stage after regenerating plan with PR --- src/Planner/PlannerJoinTree.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 6ec186cccbb..caf765e3fdf 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -943,6 +943,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_context, table_expression_query_info.storage_limits); query_plan = std::move(query_plan_parallel_replicas); + from_stage = QueryProcessingStage::WithMergeableState; + + const Block & query_plan_header = query_plan.getCurrentDataStream().header; + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "Parallel replicas query_plan_header:\n{}", query_plan_header.dumpStructure()); } } @@ -1084,6 +1088,11 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { materializeBlockInplace(expected_header); + const Block & query_plan_header = query_plan.getCurrentDataStream().header; + + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "query_plan_header:\n{}", query_plan_header.dumpStructure()); + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "expected_header:\n{}", expected_header.dumpStructure()); + auto rename_actions_dag = ActionsDAG::makeConvertingActions( query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), expected_header.getColumnsWithTypeAndName(), From 1f7198b3d3576de29485cd7b96bbc9bf97d181bb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 12:15:58 +0000 Subject: [PATCH 092/856] 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 38ec80ce92a4cbb18b3959a1a63ee9ee2faeea7c Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 17 May 2024 14:35:58 +0200 Subject: [PATCH 093/856] Update src/Core/SettingsChangesHistory.h --- src/Core/SettingsChangesHistory.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 1ef475ab9db..119e359b29b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,7 +92,6 @@ static std::map sett {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"output_format_pretty_preserve_border_for_multiline_string", 0, 1, "Applies better rendering for multiline strings."}, {"backup_restore_s3_retry_attempts", 0, 1000, "A new setting."}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, From b8d8b76dd58532ecb054addc302f83c2be011419 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 17 May 2024 13:59:39 +0000 Subject: [PATCH 094/856] Fix --- src/Planner/PlannerJoinTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index caf765e3fdf..342983ef9eb 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -933,6 +933,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres // (3) if parallel replicas still enabled - replace reading step if (planner_context->getQueryContext()->canUseParallelReplicasOnInitiator()) { + from_stage = QueryProcessingStage::WithMergeableState; QueryPlan query_plan_parallel_replicas; ClusterProxy::executeQueryWithParallelReplicas( query_plan_parallel_replicas, @@ -943,7 +944,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_context, table_expression_query_info.storage_limits); query_plan = std::move(query_plan_parallel_replicas); - from_stage = QueryProcessingStage::WithMergeableState; const Block & query_plan_header = query_plan.getCurrentDataStream().header; LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "Parallel replicas query_plan_header:\n{}", query_plan_header.dumpStructure()); From f990bb2a210aaefdc23bae870f53d808e8307178 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 17 May 2024 14:44:40 +0000 Subject: [PATCH 095/856] Fix: 02784_parallel_replicas_automatic_decision --- src/Planner/PlannerJoinTree.cpp | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 342983ef9eb..d3465ea8b86 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -899,7 +899,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) { - auto result_ptr = reading->selectRangesToRead(reading->getParts(), reading->getAlterConvertionsForParts()); + auto result_ptr = reading->selectRangesToRead(); UInt64 rows_to_read = result_ptr->selected_rows; if (table_expression_query_info.limit > 0 && table_expression_query_info.limit < rows_to_read) @@ -909,12 +909,11 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres rows_to_read = max_block_size_limited; const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; - if (number_of_replicas_to_use > 1) - LOG_TRACE( - getLogger("Planner"), - "Estimated {} rows to read. It is enough work for {} parallel replicas", - rows_to_read, - number_of_replicas_to_use); + LOG_TRACE( + getLogger("Planner"), + "Estimated {} rows to read. It is enough work for {} parallel replicas", + rows_to_read, + number_of_replicas_to_use); if (number_of_replicas_to_use <= 1) { @@ -1083,16 +1082,12 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres planner.buildQueryPlanIfNeeded(); auto expected_header = planner.getQueryPlan().getCurrentDataStream().header; + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "expected_header:\n{}", expected_header.dumpStructure()); if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, expected_header)) { materializeBlockInplace(expected_header); - const Block & query_plan_header = query_plan.getCurrentDataStream().header; - - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "query_plan_header:\n{}", query_plan_header.dumpStructure()); - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "expected_header:\n{}", expected_header.dumpStructure()); - auto rename_actions_dag = ActionsDAG::makeConvertingActions( query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), expected_header.getColumnsWithTypeAndName(), From d8ddabeb35ba3aaed0dbd7a30f91a00af35f363b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 17 May 2024 14:55:15 +0000 Subject: [PATCH 096/856] Cleanup read() in MT/RMT --- src/Storages/StorageMergeTree.cpp | 28 +------------------ src/Storages/StorageReplicatedMergeTree.cpp | 30 +-------------------- 2 files changed, 2 insertions(+), 56 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b33aabc7bf5..0d867b779d6 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -217,34 +217,8 @@ void StorageMergeTree::read( if (local_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_for_non_replicated_merge_tree && !settings.allow_experimental_analyzer) { - ASTPtr modified_query_ast; - Block header; - if (local_context->getSettingsRef().allow_experimental_analyzer) - { - QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone(); - rewriteJoinToGlobalJoin(modified_query_tree, local_context); - modified_query_tree = buildQueryTreeForShard(query_info.planner_context, modified_query_tree); - header = InterpreterSelectQueryAnalyzer::getSampleBlock( - modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); - } - else - { - const auto table_id = getStorageID(); - modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - header - = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } - ClusterProxy::executeQueryWithParallelReplicas( - query_plan, - getStorageID(), - header, - processed_stage, - modified_query_ast, - local_context, - query_info.storage_limits); + query_plan, getStorageID(), processed_stage, query_info.query, local_context, query_info.storage_limits); } else { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d08c3be30d7..341fa69abbd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5479,36 +5479,8 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( ContextPtr local_context, QueryProcessingStage::Enum processed_stage) { - ASTPtr modified_query_ast; - Block header; - const auto table_id = getStorageID(); - - if (local_context->getSettingsRef().allow_experimental_analyzer) - { - QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone(); - rewriteJoinToGlobalJoin(modified_query_tree, local_context); - modified_query_tree = buildQueryTreeForShard(query_info.planner_context, modified_query_tree); - - header = InterpreterSelectQueryAnalyzer::getSampleBlock( - modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); - } - else - { - modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - header - = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } - ClusterProxy::executeQueryWithParallelReplicas( - query_plan, - table_id, - header, - processed_stage, - modified_query_ast, - local_context, - query_info.storage_limits); + query_plan, getStorageID(), processed_stage, query_info.query, local_context, query_info.storage_limits); } void StorageReplicatedMergeTree::readLocalImpl( From 8d235a4a399b3489ff3a8672134c8905511562a3 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 15:11:21 +0000 Subject: [PATCH 097/856] 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 098/856] 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 099/856] 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 96e19ac5486511d668b451e6541361d8754a4a67 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 17 May 2024 17:10:40 +0000 Subject: [PATCH 100/856] add merge_workload and mutation_workload settings for server and merge tree --- programs/server/Server.cpp | 4 ++ programs/server/config.xml | 8 ++++ src/Core/ServerSettings.h | 2 + src/Interpreters/Context.cpp | 41 +++++++++++++++++++ src/Interpreters/Context.h | 6 +++ .../MergeTree/MergeFromLogEntryTask.cpp | 2 +- .../MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTask.h | 8 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 2 + .../MergeTree/MutateFromLogEntryTask.cpp | 2 +- .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- 11 files changed, 71 insertions(+), 8 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index bcfe3799818..8ad9329c67c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1557,6 +1557,10 @@ try 0, // We don't need any threads one all the parts will be deleted new_server_settings.max_parts_cleaning_thread_pool_size); + + global_context->setMergeWorkload(new_server_settings.merge_workload); + global_context->setMutationWorkload(new_server_settings.mutation_workload); + if (config->has("resources")) { global_context->getResourceManager()->updateConfiguration(*config); diff --git a/programs/server/config.xml b/programs/server/config.xml index 27ed5952fc9..ad920230036 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1396,6 +1396,14 @@ + + + + diff --git a/tests/integration/test_scheduler/configs/resources.xml.default b/tests/integration/test_scheduler/configs/resources.xml.default new file mode 100644 index 00000000000..3b003a17557 --- /dev/null +++ b/tests/integration/test_scheduler/configs/resources.xml.default @@ -0,0 +1,76 @@ + + + + inflight_limit1000000 + priority + fifo0 + fair1 + fifo9 + fifo1 + fair90 + fifo + fifo + fifo9 + fifo9 + fifo9 + fifo9 + + + inflight_limit1000000 + priority + fifo0 + fair1 + fifo9 + fifo1 + fair90 + fifo + fifo + fifo9 + fifo9 + fifo9 + fifo9 + + + + + /prio/admin + /prio/admin + + + /prio/fair/prod + /prio/fair/prod + + + /prio/fair/dev + /prio/fair/dev + + + /prio/fair/dev + /prio/fair/dev + + + /prio/fair/sys/merges + /prio/fair/sys/merges + + + /prio/fair/sys/mutations + /prio/fair/sys/mutations + + + /prio/fair/prod_merges + /prio/fair/prod_merges + + + /prio/fair/prod_mutations + /prio/fair/prod_mutations + + + /prio/fair/dev_merges + /prio/fair/dev_merges + + + /prio/fair/dev_mutations + /prio/fair/dev_mutations + + + diff --git a/tests/integration/test_scheduler/configs/scheduler.xml b/tests/integration/test_scheduler/configs/scheduler.xml deleted file mode 100644 index 523ba1a5a98..00000000000 --- a/tests/integration/test_scheduler/configs/scheduler.xml +++ /dev/null @@ -1,62 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - 33554432 - 10 - 10 - network_read - network_write - - - - - -
- s3 -
-
-
-
-
- - - inflight_limit1000000 - priority - fifo0 - fair1 - fifo9 - fifo1 - - - inflight_limit1000000 - priority - fifo0 - fair1 - fifo9 - fifo1 - - - - - /prio/admin - /prio/admin - - - /prio/fair/prod - /prio/fair/prod - - - /prio/fair/dev - /prio/fair/dev - - - /prio/fair/dev - /prio/fair/dev - - -
diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml new file mode 100644 index 00000000000..823a00a05de --- /dev/null +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -0,0 +1,26 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + 10 + 10 + network_read + network_write + + + + + +
+ s3 +
+
+
+
+
+
diff --git a/tests/integration/test_scheduler/configs/workloads.xml b/tests/integration/test_scheduler/configs/workloads.xml new file mode 100644 index 00000000000..197bf660500 --- /dev/null +++ b/tests/integration/test_scheduler/configs/workloads.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_scheduler/configs/workloads.xml.default b/tests/integration/test_scheduler/configs/workloads.xml.default new file mode 100644 index 00000000000..f010993335d --- /dev/null +++ b/tests/integration/test_scheduler/configs/workloads.xml.default @@ -0,0 +1,4 @@ + + sys_merges + sys_mutations + diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index e6def99c076..e0660c03681 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -13,7 +13,13 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", stay_alive=True, - main_configs=["configs/scheduler.xml"], + main_configs=[ + "configs/storage_configuration.xml", + "configs/resources.xml", + "configs/resources.xml.default", + "configs/workloads.xml", + "configs/workloads.xml.default", + ], with_minio=True, ) @@ -27,6 +33,41 @@ def start_cluster(): cluster.shutdown() +@pytest.fixture(scope="function", autouse=True) +def set_default_configs(): + node.exec_in_container( + [ + "bash", + "-c", + "cp /etc/clickhouse-server/config.d/resources.xml.default /etc/clickhouse-server/config.d/resources.xml", + ] + ) + node.exec_in_container( + [ + "bash", + "-c", + "cp /etc/clickhouse-server/config.d/workloads.xml.default /etc/clickhouse-server/config.d/workloads.xml", + ] + ) + node.query("system reload config") + yield + + +def update_workloads_config(**settings): + xml='' + for name in settings: + xml += f"<{name}>{settings[name]}" + print(xml) + node.exec_in_container( + [ + "bash", + "-c", + f"echo '{xml}' > /etc/clickhouse-server/config.d/workloads.xml", + ] + ) + node.query("system reload config") + + def test_s3_disk(): node.query( f""" @@ -110,3 +151,175 @@ def test_s3_disk(): ) == "1\n" ) + + +def test_merge_workload(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'").strip()) + writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'").strip()) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"insert into data select * from numbers(2e4)") + node.query(f"insert into data select * from numbers(3e4)") + node.query(f"optimize table data final") + + reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'").strip()) + writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'").strip()) + + assert (reads_before < reads_after) + assert (writes_before < writes_after) + + +def test_merge_workload_override(): + node.query( + f""" + drop table if exists prod_data; + drop table if exists dev_data; + create table prod_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', merge_workload='prod_merges'; + create table dev_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', merge_workload='dev_merges'; + """ + ) + + prod_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'").strip()) + prod_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'").strip()) + dev_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'").strip()) + dev_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'").strip()) + + node.query(f"insert into prod_data select * from numbers(1e4)") + node.query(f"insert into prod_data select * from numbers(2e4)") + node.query(f"insert into prod_data select * from numbers(3e4)") + node.query(f"insert into dev_data select * from numbers(1e4)") + node.query(f"insert into dev_data select * from numbers(2e4)") + node.query(f"insert into dev_data select * from numbers(3e4)") + node.query(f"optimize table prod_data final") + node.query(f"optimize table dev_data final") + + prod_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'").strip()) + prod_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'").strip()) + dev_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'").strip()) + dev_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'").strip()) + + assert (prod_reads_before < prod_reads_after) + assert (prod_writes_before < prod_writes_after) + assert (dev_reads_before < dev_reads_after) + assert (dev_writes_before < dev_writes_after) + + +def test_mutate_workload(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"optimize table data final") + + reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'").strip()) + writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'").strip()) + + node.query(f"alter table data update key = 1 where key = 42") + node.query(f"optimize table data final") + + reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'").strip()) + writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'").strip()) + + assert (reads_before < reads_after) + assert (writes_before < writes_after) + + +def test_mutation_workload_override(): + node.query( + f""" + drop table if exists prod_data; + drop table if exists dev_data; + create table prod_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', mutation_workload='prod_mutations'; + create table dev_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', mutation_workload='dev_mutations'; + """ + ) + + node.query(f"insert into prod_data select * from numbers(1e4)") + node.query(f"optimize table prod_data final") + node.query(f"insert into dev_data select * from numbers(1e4)") + node.query(f"optimize table dev_data final") + + prod_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'").strip()) + prod_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'").strip()) + dev_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'").strip()) + dev_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'").strip()) + + node.query(f"alter table prod_data update key = 1 where key = 42") + node.query(f"optimize table prod_data final") + node.query(f"alter table dev_data update key = 1 where key = 42") + node.query(f"optimize table dev_data final") + + prod_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'").strip()) + prod_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'").strip()) + dev_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'").strip()) + dev_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'").strip()) + + assert (prod_reads_before < prod_reads_after) + assert (prod_writes_before < prod_writes_after) + assert (dev_reads_before < dev_reads_after) + assert (dev_writes_before < dev_writes_after) + + +def test_merge_workload_change(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + for env in ['prod', 'dev']: + update_workloads_config(merge_workload=f"{env}_merges") + + reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'").strip()) + writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'").strip()) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"insert into data select * from numbers(2e4)") + node.query(f"insert into data select * from numbers(3e4)") + node.query(f"optimize table data final") + + reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'").strip()) + writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'").strip()) + + assert (reads_before < reads_after) + assert (writes_before < writes_after) + +def test_mutation_workload_change(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + for env in ['prod', 'dev']: + update_workloads_config(mutation_workload=f"{env}_mutations") + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"optimize table data final") + + reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'").strip()) + writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'").strip()) + + node.query(f"alter table data update key = 1 where key = 42") + node.query(f"optimize table data final") + + reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'").strip()) + writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'").strip()) + + breakpoint() + + assert (reads_before < reads_after) + assert (writes_before < writes_after) From 22af95b87151d729b4f9353c8835f32066f978b9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 22 May 2024 14:46:57 +0000 Subject: [PATCH 114/856] Automatic style fix --- tests/integration/test_scheduler/test.py | 231 ++++++++++++++++++----- 1 file changed, 180 insertions(+), 51 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index e0660c03681..d1ae51bc0d7 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -54,7 +54,7 @@ def set_default_configs(): def update_workloads_config(**settings): - xml='' + xml = "" for name in settings: xml += f"<{name}>{settings[name]}" print(xml) @@ -161,19 +161,35 @@ def test_merge_workload(): """ ) - reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'").strip()) - writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'").strip()) + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'" + ).strip() + ) node.query(f"insert into data select * from numbers(1e4)") node.query(f"insert into data select * from numbers(2e4)") node.query(f"insert into data select * from numbers(3e4)") node.query(f"optimize table data final") - reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'").strip()) - writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'").strip()) + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'" + ).strip() + ) - assert (reads_before < reads_after) - assert (writes_before < writes_after) + assert reads_before < reads_after + assert writes_before < writes_after def test_merge_workload_override(): @@ -186,10 +202,26 @@ def test_merge_workload_override(): """ ) - prod_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'").strip()) - prod_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'").strip()) - dev_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'").strip()) - dev_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'").strip()) + prod_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'" + ).strip() + ) + prod_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'" + ).strip() + ) + dev_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'" + ).strip() + ) + dev_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'" + ).strip() + ) node.query(f"insert into prod_data select * from numbers(1e4)") node.query(f"insert into prod_data select * from numbers(2e4)") @@ -200,15 +232,31 @@ def test_merge_workload_override(): node.query(f"optimize table prod_data final") node.query(f"optimize table dev_data final") - prod_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'").strip()) - prod_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'").strip()) - dev_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'").strip()) - dev_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'").strip()) + prod_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'" + ).strip() + ) + prod_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'" + ).strip() + ) + dev_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'" + ).strip() + ) + dev_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'" + ).strip() + ) - assert (prod_reads_before < prod_reads_after) - assert (prod_writes_before < prod_writes_after) - assert (dev_reads_before < dev_reads_after) - assert (dev_writes_before < dev_writes_after) + assert prod_reads_before < prod_reads_after + assert prod_writes_before < prod_writes_after + assert dev_reads_before < dev_reads_after + assert dev_writes_before < dev_writes_after def test_mutate_workload(): @@ -222,17 +270,33 @@ def test_mutate_workload(): node.query(f"insert into data select * from numbers(1e4)") node.query(f"optimize table data final") - reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'").strip()) - writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'").strip()) + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'" + ).strip() + ) node.query(f"alter table data update key = 1 where key = 42") node.query(f"optimize table data final") - reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'").strip()) - writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'").strip()) + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'" + ).strip() + ) - assert (reads_before < reads_after) - assert (writes_before < writes_after) + assert reads_before < reads_after + assert writes_before < writes_after def test_mutation_workload_override(): @@ -250,25 +314,57 @@ def test_mutation_workload_override(): node.query(f"insert into dev_data select * from numbers(1e4)") node.query(f"optimize table dev_data final") - prod_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'").strip()) - prod_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'").strip()) - dev_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'").strip()) - dev_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'").strip()) + prod_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'" + ).strip() + ) + prod_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'" + ).strip() + ) + dev_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'" + ).strip() + ) + dev_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'" + ).strip() + ) node.query(f"alter table prod_data update key = 1 where key = 42") node.query(f"optimize table prod_data final") node.query(f"alter table dev_data update key = 1 where key = 42") node.query(f"optimize table dev_data final") - prod_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'").strip()) - prod_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'").strip()) - dev_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'").strip()) - dev_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'").strip()) + prod_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'" + ).strip() + ) + prod_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'" + ).strip() + ) + dev_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'" + ).strip() + ) + dev_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'" + ).strip() + ) - assert (prod_reads_before < prod_reads_after) - assert (prod_writes_before < prod_writes_after) - assert (dev_reads_before < dev_reads_after) - assert (dev_writes_before < dev_writes_after) + assert prod_reads_before < prod_reads_after + assert prod_writes_before < prod_writes_after + assert dev_reads_before < dev_reads_after + assert dev_writes_before < dev_writes_after def test_merge_workload_change(): @@ -279,22 +375,39 @@ def test_merge_workload_change(): """ ) - for env in ['prod', 'dev']: + for env in ["prod", "dev"]: update_workloads_config(merge_workload=f"{env}_merges") - reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'").strip()) - writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'").strip()) + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'" + ).strip() + ) node.query(f"insert into data select * from numbers(1e4)") node.query(f"insert into data select * from numbers(2e4)") node.query(f"insert into data select * from numbers(3e4)") node.query(f"optimize table data final") - reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'").strip()) - writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'").strip()) + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'" + ).strip() + ) + + assert reads_before < reads_after + assert writes_before < writes_after - assert (reads_before < reads_after) - assert (writes_before < writes_after) def test_mutation_workload_change(): node.query( @@ -304,22 +417,38 @@ def test_mutation_workload_change(): """ ) - for env in ['prod', 'dev']: + for env in ["prod", "dev"]: update_workloads_config(mutation_workload=f"{env}_mutations") node.query(f"insert into data select * from numbers(1e4)") node.query(f"optimize table data final") - reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'").strip()) - writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'").strip()) + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'" + ).strip() + ) node.query(f"alter table data update key = 1 where key = 42") node.query(f"optimize table data final") - reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'").strip()) - writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'").strip()) + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'" + ).strip() + ) breakpoint() - assert (reads_before < reads_after) - assert (writes_before < writes_after) + assert reads_before < reads_after + assert writes_before < writes_after From 319542f85bc5c36bbc7c810c0883b721956250be Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 22 May 2024 19:07:14 +0000 Subject: [PATCH 115/856] 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 116/856] 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 117/856] 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 7e5bf14135604ae6d2b3b5861d93583093993482 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 11 Apr 2024 00:01:48 +0200 Subject: [PATCH 118/856] Add support for LineString WKT --- .../en/sql-reference/functions/geo/polygon.md | 30 ++++++++++++++- src/DataTypes/DataTypeCustomGeo.cpp | 7 ++++ src/DataTypes/DataTypeCustomGeo.h | 6 +++ src/Functions/geometryConverters.h | 38 +++++++++++++++++++ src/Functions/readWkt.cpp | 6 +++ 5 files changed, 86 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index c2572779ada..25a7a1fac8e 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -142,6 +142,34 @@ SELECT readWKTPoint('POINT (1.2 3.4)'); (1.2,3.4) ``` +## readWKTLineString + +Parses a Well-Known Text (WKT) representation of a LineString geometry and returns it in the internal ClickHouse format. + +### Syntax + +```sql +readWKTLineString(wkt_string) +``` + +### Arguments + +- `wkt_string`: The input WKT string representing a LineString geometry. + +### Returned value + +The function returns a ClickHouse internal representation of the linestring geometry. + +### Example + +```sql +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +``` + +```response +[(1,1),(2,2),(3,3),(1,1)] +``` + ## readWKTRing Parses a Well-Known Text (WKT) representation of a Polygon geometry and returns a ring (closed linestring) in the internal ClickHouse format. @@ -163,7 +191,7 @@ The function returns a ClickHouse internal representation of the ring (closed li ### Example ```sql -SELECT readWKTRing('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +SELECT readWKTRing('POLYGON ((1 1, 2 2, 3 3, 1 1))'); ``` ```response diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index f7d05fa3be6..0736d837d46 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -17,6 +17,13 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) std::make_unique(std::make_unique())); }); + // Custom type for simple line which consists from several segments. + factory.registerSimpleDataTypeCustom("LineString", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(Point)"), + std::make_unique(std::make_unique())); + }); + // Custom type for simple polygon without holes stored as Array(Point) factory.registerSimpleDataTypeCustom("Ring", [] { diff --git a/src/DataTypes/DataTypeCustomGeo.h b/src/DataTypes/DataTypeCustomGeo.h index c2a83b3e577..0a1c83e4638 100644 --- a/src/DataTypes/DataTypeCustomGeo.h +++ b/src/DataTypes/DataTypeCustomGeo.h @@ -11,6 +11,12 @@ public: DataTypePointName() : DataTypeCustomFixedName("Point") {} }; +class DataTypeLineStringName : public DataTypeCustomFixedName +{ +public: + DataTypeLineStringName() : DataTypeCustomFixedName("LineString") {} +}; + class DataTypeRingName : public DataTypeCustomFixedName { public: diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 97162fa9dd0..98865a9048f 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -28,6 +28,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +template +using LineString = boost::geometry::model::linestring; + template using Ring = boost::geometry::model::ring; @@ -38,11 +41,13 @@ template using MultiPolygon = boost::geometry::model::multi_polygon>; using CartesianPoint = boost::geometry::model::d2::point_xy; +using CartesianLineString = LineString; using CartesianRing = Ring; using CartesianPolygon = Polygon; using CartesianMultiPolygon = MultiPolygon; using SphericalPoint = boost::geometry::model::point>; +using SphericalLineString = LineString; using SphericalRing = Ring; using SphericalPolygon = Polygon; using SphericalMultiPolygon = MultiPolygon; @@ -208,6 +213,39 @@ private: ColumnFloat64::Container & second_container; }; +/// Serialize Point, LineString as LineString +template +class LineStringSerializer +{ +public: + LineStringSerializer() + : offsets(ColumnUInt64::create()) + {} + + explicit LineStringSerializer(size_t n) + : offsets(ColumnUInt64::create(n)) + {} + + void add(const LineString & ring) + { + size += ring.size(); + offsets->insertValue(size); + for (const auto & point : ring) + point_serializer.add(point); + } + + ColumnPtr finalize() + { + return ColumnArray::create(point_serializer.finalize(), std::move(offsets)); + } + +private: + size_t size = 0; + PointSerializer point_serializer; + ColumnUInt64::MutablePtr offsets; +}; + +/// Almost the same as LineStringSerializer /// Serialize Point, Ring as Ring template class RingSerializer diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index ddc847b1ca5..f4975e85d0d 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -82,6 +82,11 @@ struct ReadWKTPointNameHolder static constexpr const char * name = "readWKTPoint"; }; +struct ReadWKTLineStringNameHolder +{ + static constexpr const char * name = "readWKTLineString"; +}; + struct ReadWKTRingNameHolder { static constexpr const char * name = "readWKTRing"; @@ -102,6 +107,7 @@ struct ReadWKTMultiPolygonNameHolder REGISTER_FUNCTION(ReadWKT) { factory.registerFunction, ReadWKTPointNameHolder>>(); + factory.registerFunction, ReadWKTLineStringNameHolder>>(); factory.registerFunction, ReadWKTRingNameHolder>>(); factory.registerFunction, ReadWKTPolygonNameHolder>>(); factory.registerFunction, ReadWKTMultiPolygonNameHolder>>(); From 40c069dc28c8d08c5477202db9dc98ba53edec2d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 11 Apr 2024 17:26:11 +0000 Subject: [PATCH 119/856] Fix style --- 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 1c601bc200a..31bf406ded1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -485,6 +485,7 @@ Liao LibFuzzer LightHouse LineAsString +LineString Linf LinfDistance LinfNorm @@ -2275,6 +2276,7 @@ randomStringUTF rankCorr rapidjson rawblob +readWKTLineString readWKTMultiPolygon readWKTPoint readWKTPolygon From 54f2549ef59ca72478ef98a5289264e60790dcc1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 12 Apr 2024 12:40:29 +0000 Subject: [PATCH 120/856] Save --- src/Functions/geometryConverters.h | 26 +++++++++++++++++++++++++ src/Functions/polygonsIntersection.cpp | 2 ++ src/Functions/polygonsSymDifference.cpp | 2 ++ src/Functions/polygonsUnion.cpp | 2 ++ src/Functions/polygonsWithin.cpp | 2 ++ 5 files changed, 34 insertions(+) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 98865a9048f..ec74d415824 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -90,6 +90,29 @@ struct ColumnToPointsConverter } }; + +/** + * Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost linestring type. +*/ +template +struct ColumnToLineStringsConverter +{ + static std::vector> convert(ColumnPtr col) + { + const IColumn::Offsets & offsets = typeid_cast(*col).getOffsets(); + size_t prev_offset = 0; + std::vector> answer; + answer.reserve(offsets.size()); + auto tmp = ColumnToPointsConverter::convert(typeid_cast(*col).getDataPtr()); + for (size_t offset : offsets) + { + answer.emplace_back(tmp.begin() + prev_offset, tmp.begin() + offset); + prev_offset = offset; + } + return answer; + } +}; + /** * Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost ring type. */ @@ -382,6 +405,9 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) /// There is no Point type, because for most of geometry functions it is useless. if (factory.get("Point")->equals(*type)) return f(ConverterType>()); + /// There is some ambiguity, we don't take into consideration a name of a custom type. + // else if (factory.get("LineString")->equals(*type)) + // return f(ConverterType>()); else if (factory.get("Ring")->equals(*type)) return f(ConverterType>()); else if (factory.get("Polygon")->equals(*type)) diff --git a/src/Functions/polygonsIntersection.cpp b/src/Functions/polygonsIntersection.cpp index 77484e7e63c..329242e762e 100644 --- a/src/Functions/polygonsIntersection.cpp +++ b/src/Functions/polygonsIntersection.cpp @@ -73,6 +73,8 @@ public: if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsSymDifference.cpp b/src/Functions/polygonsSymDifference.cpp index 194b7f2cfd7..3c219d0facb 100644 --- a/src/Functions/polygonsSymDifference.cpp +++ b/src/Functions/polygonsSymDifference.cpp @@ -71,6 +71,8 @@ public: if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsUnion.cpp b/src/Functions/polygonsUnion.cpp index 37d865af50a..969eb2f78fb 100644 --- a/src/Functions/polygonsUnion.cpp +++ b/src/Functions/polygonsUnion.cpp @@ -71,6 +71,8 @@ public: if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsWithin.cpp b/src/Functions/polygonsWithin.cpp index 35a9e17cdfd..c63ad5ef868 100644 --- a/src/Functions/polygonsWithin.cpp +++ b/src/Functions/polygonsWithin.cpp @@ -75,6 +75,8 @@ public: if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); From 6510f804478303fb5d420d43459201018bea6dc1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 23 May 2024 16:59:07 +0200 Subject: [PATCH 121/856] Updated test to fix unrecognized option error --- tests/queries/0_stateless/02941_variant_type_1.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_2.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_3.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_4.sh | 6 +++--- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02941_variant_type_1.sh b/tests/queries/0_stateless/02941_variant_type_1.sh index 22ca909a26e..723de45eaad 100755 --- a/tests/queries/0_stateless/02941_variant_type_1.sh +++ b/tests/queries/0_stateless/02941_variant_type_1.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test1_insert() { @@ -115,11 +115,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh index 91ba0285bd8..f43cd2bb0d6 100755 --- a/tests/queries/0_stateless/02941_variant_type_2.sh +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test4_insert() { @@ -61,11 +61,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_3.sh b/tests/queries/0_stateless/02941_variant_type_3.sh index 8a039a02d6d..f4b2b304f56 100755 --- a/tests/queries/0_stateless/02941_variant_type_3.sh +++ b/tests/queries/0_stateless/02941_variant_type_3.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test5_insert() { @@ -63,11 +63,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index b003bcdcef1..f9a16847864 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test6_insert() { @@ -58,11 +58,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" From ac6dae516547afe8390c868787219c225a08bdd3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 23 May 2024 15:26:21 +0000 Subject: [PATCH 122/856] Correctly compare custom types --- src/Functions/geometryConverters.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index ec74d415824..1478de14c4d 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -405,10 +405,11 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) /// There is no Point type, because for most of geometry functions it is useless. if (factory.get("Point")->equals(*type)) return f(ConverterType>()); - /// There is some ambiguity, we don't take into consideration a name of a custom type. - // else if (factory.get("LineString")->equals(*type)) - // return f(ConverterType>()); - else if (factory.get("Ring")->equals(*type)) + /// We should take the name into consideration to avoid ambiguity. + /// Because for example both Ring and LineString are resolved to Array. + else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "Ring") + return f(ConverterType>()); + else if (factory.get("Ring")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "Ring") return f(ConverterType>()); else if (factory.get("Polygon")->equals(*type)) return f(ConverterType>()); From f632636f210c34841a7634790e32ba2153633ebf Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 23 May 2024 19:12:02 +0000 Subject: [PATCH 123/856] 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 73f42b0858204f1682269453b0565380afc7a9f4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 May 2024 20:59:10 +0200 Subject: [PATCH 124/856] add clusters with replicas from all replica groups --- src/Databases/DatabaseReplicated.cpp | 67 +++++++++++++++---- src/Databases/DatabaseReplicated.h | 6 +- src/Databases/DatabaseReplicatedWorker.cpp | 2 + src/Interpreters/DDLTask.cpp | 17 ++++- src/Storages/System/StorageSystemClusters.cpp | 4 ++ 5 files changed, 78 insertions(+), 18 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index cc946fc22c4..c9e14790175 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -173,13 +173,40 @@ ClusterPtr DatabaseReplicated::tryGetCluster() const return cluster; } -void DatabaseReplicated::setCluster(ClusterPtr && new_cluster) +ClusterPtr DatabaseReplicated::tryGetAllGroupsCluster() const { std::lock_guard lock{mutex}; - cluster = std::move(new_cluster); + if (replica_group_name.empty()) + return nullptr; + + if (cluster_all_groups) + return cluster_all_groups; + + /// Database is probably not created or not initialized yet, it's ok to return nullptr + if (is_readonly) + return cluster_all_groups; + + try + { + cluster_all_groups = getClusterImpl(/*all_groups*/ true); + } + catch (...) + { + tryLogCurrentException(log); + } + return cluster_all_groups; } -ClusterPtr DatabaseReplicated::getClusterImpl() const +void DatabaseReplicated::setCluster(ClusterPtr && new_cluster, bool all_groups) +{ + std::lock_guard lock{mutex}; + if (all_groups) + cluster_all_groups = std::move(new_cluster); + else + cluster = std::move(new_cluster); +} + +ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const { Strings unfiltered_hosts; Strings hosts; @@ -199,17 +226,24 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const "It's possible if the first replica is not fully created yet " "or if the last replica was just dropped or due to logical error", zookeeper_path); - hosts.clear(); - std::vector paths; - for (const auto & host : unfiltered_hosts) - paths.push_back(zookeeper_path + "/replicas/" + host + "/replica_group"); - - auto replica_groups = zookeeper->tryGet(paths); - - for (size_t i = 0; i < paths.size(); ++i) + if (all_groups) { - if (replica_groups[i].data == replica_group_name) - hosts.push_back(unfiltered_hosts[i]); + hosts = unfiltered_hosts; + } + else + { + hosts.clear(); + std::vector paths; + for (const auto & host : unfiltered_hosts) + paths.push_back(zookeeper_path + "/replicas/" + host + "/replica_group"); + + auto replica_groups = zookeeper->tryGet(paths); + + for (size_t i = 0; i < paths.size(); ++i) + { + if (replica_groups[i].data == replica_group_name) + hosts.push_back(unfiltered_hosts[i]); + } } Int32 cversion = stat.cversion; @@ -274,6 +308,11 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const bool treat_local_as_remote = false; bool treat_local_port_as_remote = getContext()->getApplicationType() == Context::ApplicationType::LOCAL; + + String cluster_name = TSA_SUPPRESS_WARNING_FOR_READ(database_name); /// FIXME + if (all_groups) + cluster_name = "all_groups." + cluster_name; + ClusterConnectionParameters params{ cluster_auth_info.cluster_username, cluster_auth_info.cluster_password, @@ -282,7 +321,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const treat_local_port_as_remote, cluster_auth_info.cluster_secure_connection, Priority{1}, - TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME + cluster_name, cluster_auth_info.cluster_secret}; return std::make_shared(getContext()->getSettingsRef(), shards, params); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 55bcf963d37..f902b45ca86 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -65,6 +65,7 @@ public: /// Returns cluster consisting of database replicas ClusterPtr tryGetCluster() const; + ClusterPtr tryGetAllGroupsCluster() const; void drop(ContextPtr /*context*/) override; @@ -113,8 +114,8 @@ private: ASTPtr parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query); String readMetadataFile(const String & table_name) const; - ClusterPtr getClusterImpl() const; - void setCluster(ClusterPtr && new_cluster); + ClusterPtr getClusterImpl(bool all_groups = false) const; + void setCluster(ClusterPtr && new_cluster, bool all_groups = false); void createEmptyLogEntry(const ZooKeeperPtr & current_zookeeper); @@ -155,6 +156,7 @@ private: UInt64 tables_metadata_digest TSA_GUARDED_BY(metadata_mutex); mutable ClusterPtr cluster; + mutable ClusterPtr cluster_all_groups; LoadTaskPtr startup_replicated_database_task TSA_GUARDED_BY(mutex); }; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 6e19a77c501..31d6f7876a8 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -421,6 +421,8 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na { /// Some replica is added or removed, let's update cached cluster database->setCluster(database->getClusterImpl()); + if (!database->replica_group_name.empty()) + database->setCluster(database->getClusterImpl(/*all_groups*/ true), /*all_groups*/ true); out_reason = fmt::format("Entry {} is a dummy task", entry_name); return {}; } diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index a37b4db029a..06ec9489fc1 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -568,8 +568,21 @@ void ZooKeeperMetadataTransaction::commit() ClusterPtr tryGetReplicatedDatabaseCluster(const String & cluster_name) { - if (const auto * replicated_db = dynamic_cast(DatabaseCatalog::instance().tryGetDatabase(cluster_name).get())) - return replicated_db->tryGetCluster(); + String name = cluster_name; + bool all_groups = false; + if (name.starts_with("all_groups.")) + { + name = name.substr(strlen("all_groups.")); + all_groups = true; + } + + if (const auto * replicated_db = dynamic_cast(DatabaseCatalog::instance().tryGetDatabase(name).get())) + { + if (all_groups) + return replicated_db->tryGetAllGroupsCluster(); + else + return replicated_db->tryGetCluster(); + } return {}; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index cb8d5caa50c..520bd7e7f92 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -54,6 +54,10 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co if (auto database_cluster = replicated->tryGetCluster()) writeCluster(res_columns, {name_and_database.first, database_cluster}, replicated->tryGetAreReplicasActive(database_cluster)); + + if (auto database_cluster = replicated->tryGetAllGroupsCluster()) + writeCluster(res_columns, {"all_groups." + name_and_database.first, database_cluster}, + replicated->tryGetAreReplicasActive(database_cluster)); } } } From aa93ace6d8d92cca4dedcf5a2bebf7c8b2b966d6 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 24 May 2024 10:42:59 +0200 Subject: [PATCH 125/856] Fixed more tests with unrecognized option error --- tests/queries/0_stateless/02941_variant_type_4.sh | 2 +- .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- .../0_stateless/03039_dynamic_all_merge_algorithms_2.sh | 6 +++--- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index b90f9090b07..f9a16847864 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1"g +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test6_insert() { diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 7c1ac41cfdc..0e4c2a0cc5e 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760 --index_granularity 8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index 927ceac72b5..e47b11614af 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760 --index_granularity 8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh index 02362012960..0be0e973d01 100755 --- a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh +++ b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --index_granularity 8128 --merge_max_block_size 8128" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8128" function test() @@ -44,7 +44,7 @@ echo "MergeTree wide + horizontal merge" test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1" echo "MergeTree compact + vertical merge" -test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1" +test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" echo "MergeTree wide + vertical merge" -test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1" +test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" From 61416150076e9e7d4debc4385ecb30d4e1b12565 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 24 May 2024 11:37:52 +0000 Subject: [PATCH 126/856] Add diagnostic --- src/Planner/PlannerJoinTree.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a6e4a8ebcde..68ebf4135c7 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -884,6 +884,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres // (1) find read step QueryPlan::Node * node = query_plan.getRootNode(); ReadFromMergeTree * reading = nullptr; + QueryPlan::Node * last_node = nullptr; while (node) { reading = typeid_cast(node->step.get()); @@ -892,9 +893,18 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!node->children.empty()) node = node->children.at(0); + else + { + last_node = node; + node = nullptr; + } } - chassert(reading); + // chassert(reading); + if (!reading) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading step is expected to be ReadFromMergeTree but it's {}", last_node->step->getName()); + } // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) From d2739d495157026fa3df6e7c140bb2800fad7cea Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 24 May 2024 13:37:57 +0200 Subject: [PATCH 127/856] Fixed merge_max_block_size setting --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 1 + .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- .../0_stateless/03039_dynamic_all_merge_algorithms_2.sh | 6 +++--- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index caa600298ce..b100f96befa 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -24,6 +24,7 @@ function check_refcnt_for_table() local log_file log_file=$(mktemp "$CUR_DIR/clickhouse-tests.XXXXXX.log") local args=( + --allow_repeated_settings --format Null --max_threads 1 --max_block_size 1 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 0e4c2a0cc5e..0c9060d261a 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index e47b11614af..2384350fa85 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh index 0be0e973d01..68cc72faf59 100755 --- a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh +++ b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8128" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() @@ -44,7 +44,7 @@ echo "MergeTree wide + horizontal merge" test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1" echo "MergeTree compact + vertical merge" -test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" +test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128, merge_max_block_size=8128;" echo "MergeTree wide + vertical merge" -test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" +test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128, merge_max_block_size=8128;" From 5c6c378fae33d8fe9294d0031352fd1a82c3bee2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 May 2024 14:34:42 +0200 Subject: [PATCH 128/856] Try to fix GWPAsan --- src/CMakeLists.txt | 2 +- src/Common/Allocator.cpp | 88 +++++++++++++++++++++++++++++- src/Common/AsynchronousMetrics.cpp | 1 + src/Common/PODArray.h | 16 +++++- src/Common/ProfileEvents.cpp | 4 ++ src/Common/clickhouse_malloc.cpp | 5 +- src/Common/memory.cpp | 22 ++++++++ src/Common/memory.h | 45 ++++++++++++++- src/Common/new_delete.cpp | 22 -------- 9 files changed, 177 insertions(+), 28 deletions(-) create mode 100644 src/Common/memory.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4e8946facda..64a09699a54 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -291,7 +291,7 @@ if (TARGET ch_contrib::llvm) endif () if (TARGET ch_contrib::gwp_asan) - target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::gwp_asan) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::gwp_asan) target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::gwp_asan) endif() diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index e80c125c2a0..80f3b204c74 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -10,6 +11,12 @@ #include #include /// MADV_POPULATE_WRITE +namespace ProfileEvents +{ + extern const Event GWPAsanAllocateSuccess; + extern const Event GWPAsanAllocateFailed; + extern const Event GWPAsanFree; +} namespace DB { @@ -59,13 +66,37 @@ void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_) template void * allocNoTrack(size_t size, size_t alignment) { + void * buf; +#if USE_GWP_ASAN + if (unlikely(Memory::GuardedAlloc.shouldSample())) + { + if (void * ptr = Memory::GuardedAlloc.allocate(size, alignment)) + { + if constexpr (clear_memory) + memset(ptr, 0, size); + + if constexpr (populate) + prefaultPages(ptr, size); + + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); + + return ptr; + } + else + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); + } + } +#endif if (alignment <= MALLOC_MIN_ALIGNMENT) { if constexpr (clear_memory) buf = ::calloc(size, 1); else + { buf = ::malloc(size); + } if (nullptr == buf) throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot malloc {}.", ReadableSize(size)); @@ -91,6 +122,15 @@ void * allocNoTrack(size_t size, size_t alignment) void freeNoTrack(void * buf) { +#if USE_GWP_ASAN + if (unlikely(Memory::GuardedAlloc.pointerIsMine(buf))) + { + ProfileEvents::increment(ProfileEvents::GWPAsanFree); + Memory::GuardedAlloc.deallocate(buf); + return; + } +#endif + ::free(buf); } @@ -144,8 +184,54 @@ void * Allocator::realloc(void * buf, size_t old_size, { /// nothing to do. /// BTW, it's not possible to change alignment while doing realloc. + return buf; } - else if (alignment <= MALLOC_MIN_ALIGNMENT) + +#if USE_GWP_ASAN + if (unlikely(Memory::GuardedAlloc.shouldSample())) + { + if (void * ptr = Memory::GuardedAlloc.allocate(new_size, alignment)) + { + auto trace_free = CurrentMemoryTracker::free(old_size); + auto trace_alloc = CurrentMemoryTracker::alloc(new_size); + trace_free.onFree(buf, old_size); + + memcpy(ptr, buf, std::min(old_size, new_size)); + free(buf, old_size); + trace_alloc.onAlloc(buf, new_size); + + if constexpr (clear_memory) + if (new_size > old_size) + memset(reinterpret_cast(ptr) + old_size, 0, new_size - old_size); + + if constexpr (populate) + prefaultPages(ptr, new_size); + + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); + return ptr; + } + else + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); + } + } + + if (unlikely(Memory::GuardedAlloc.pointerIsMine(buf))) + { + /// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods. + void * new_buf = alloc(new_size, alignment); + memcpy(new_buf, buf, std::min(old_size, new_size)); + free(buf, old_size); + buf = new_buf; + + if constexpr (populate) + prefaultPages(buf, new_size); + + return buf; + } +#endif + + if (alignment <= MALLOC_MIN_ALIGNMENT) { /// Resize malloc'd memory region with no special alignment requirement. auto trace_free = CurrentMemoryTracker::free(old_size); diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 4c71b9846c7..ccd65af07f5 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index b4069027ad1..ea3a68eb8bb 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -1,5 +1,7 @@ #pragma once +#include "config.h" + #include #include #include @@ -11,12 +13,16 @@ #include #include #include -#include #ifndef NDEBUG #include #endif +#if USE_GWP_ASAN +# include + +#endif + /** Whether we can use memcpy instead of a loop with assignment to T from U. * It is Ok if types are the same. And if types are integral and of the same size, * example: char, signed char, unsigned char. @@ -112,6 +118,10 @@ protected: template void alloc(size_t bytes, TAllocatorParams &&... allocator_params) { +#if USE_GWP_ASAN + gwp_asan::getThreadLocals()->NextSampleCounter = 1; +#endif + char * allocated = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)); c_start = allocated + pad_left; @@ -141,6 +151,10 @@ protected: return; } +#if USE_GWP_ASAN + gwp_asan::getThreadLocals()->NextSampleCounter = 1; +#endif + unprotect(); ptrdiff_t end_diff = c_end - c_start; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 8c8e2163aad..0e7a7a9d514 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -744,6 +744,10 @@ The server successfully detected this situation and will download merged part fr \ M(ReadWriteBufferFromHTTPRequestsSent, "Number of HTTP requests sent by ReadWriteBufferFromHTTP") \ M(ReadWriteBufferFromHTTPBytes, "Total size of payload bytes received and sent by ReadWriteBufferFromHTTP. Doesn't include HTTP headers.") \ + \ + M(GWPAsanAllocateSuccess, "Number of successful allocations done by GWPAsan") \ + M(GWPAsanAllocateFailed, "Number of failed allocations done by GWPAsan (i.e. filled pool)") \ + M(GWPAsanFree, "Number of free operations done by GWPAsan") \ #ifdef APPLY_FOR_EXTERNAL_EVENTS diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index afdad3c6599..a9961eb4ee4 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -1,13 +1,14 @@ #include #include - /** These functions can be substituted instead of regular ones when memory tracking is needed. */ extern "C" void * clickhouse_malloc(size_t size) { - void * res = malloc(size); + void * res = nullptr; + res = malloc(size); + if (res) { AllocationTrace trace; diff --git a/src/Common/memory.cpp b/src/Common/memory.cpp new file mode 100644 index 00000000000..862af4470f1 --- /dev/null +++ b/src/Common/memory.cpp @@ -0,0 +1,22 @@ +#include +#include + +#if USE_GWP_ASAN +namespace Memory +{ +gwp_asan::GuardedPoolAllocator GuardedAlloc; +static bool guarded_alloc_initialized = [] +{ + gwp_asan::options::initOptions(); + gwp_asan::options::Options &opts = gwp_asan::options::getOptions(); + opts.MaxSimultaneousAllocations = 256; + GuardedAlloc.init(opts); + + ///std::cerr << "GwpAsan is initialized, the options are { Enabled: " << opts.Enabled + /// << ", MaxSimultaneousAllocations: " << opts.MaxSimultaneousAllocations + /// << ", SampleRate: " << opts.SampleRate << " }\n"; + + return true; +}(); +} +#endif diff --git a/src/Common/memory.h b/src/Common/memory.h index a828ba7a38e..427120edc75 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -5,6 +5,7 @@ #include #include +#include #include "config.h" #if USE_JEMALLOC @@ -17,13 +18,24 @@ #if USE_GWP_ASAN # include +# include -static gwp_asan::GuardedPoolAllocator GuardedAlloc; #endif +namespace ProfileEvents +{ + extern const Event GWPAsanAllocateSuccess; + extern const Event GWPAsanAllocateFailed; + extern const Event GWPAsanFree; +} + namespace Memory { +#if USE_GWP_ASAN +extern gwp_asan::GuardedPoolAllocator GuardedAlloc; +#endif + inline ALWAYS_INLINE size_t alignToSizeT(std::align_val_t align) noexcept { return static_cast(align); @@ -39,12 +51,26 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) if constexpr (sizeof...(TAlign) == 1) { if (void * ptr = GuardedAlloc.allocate(size, alignToSizeT(align...))) + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); return ptr; + } + else + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); + } } else { if (void * ptr = GuardedAlloc.allocate(size)) + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); return ptr; + } + else + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); + } } } @@ -69,7 +95,14 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept if (unlikely(GuardedAlloc.shouldSample())) { if (void * ptr = GuardedAlloc.allocate(size)) + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); return ptr; + } + else + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); + } } #endif return malloc(size); @@ -81,7 +114,14 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) if (unlikely(GuardedAlloc.shouldSample())) { if (void * ptr = GuardedAlloc.allocate(size, alignToSizeT(align))) + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); return ptr; + } + else + { + ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); + } } #endif return aligned_alloc(static_cast(align), size); @@ -92,6 +132,7 @@ inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept #if USE_GWP_ASAN if (unlikely(GuardedAlloc.pointerIsMine(ptr))) { + ProfileEvents::increment(ProfileEvents::GWPAsanFree); GuardedAlloc.deallocate(ptr); return; } @@ -111,6 +152,7 @@ inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size, TAlign... al #if USE_GWP_ASAN if (unlikely(GuardedAlloc.pointerIsMine(ptr))) { + ProfileEvents::increment(ProfileEvents::GWPAsanFree); GuardedAlloc.deallocate(ptr); return; } @@ -131,6 +173,7 @@ inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unuse #if USE_GWP_ASAN if (unlikely(GuardedAlloc.pointerIsMine(ptr))) { + ProfileEvents::increment(ProfileEvents::GWPAsanFree); GuardedAlloc.deallocate(ptr); return; } diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index 9e93dca9787..e8151fbe201 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -1,5 +1,4 @@ #include -#include #include #include "config.h" #include @@ -42,27 +41,6 @@ static struct InitializeJemallocZoneAllocatorForOSX } initializeJemallocZoneAllocatorForOSX; #endif -#if USE_GWP_ASAN - -#include - -/// Both clickhouse_new_delete and clickhouse_common_io links gwp_asan, but It should only init once, otherwise it -/// will cause unexpected deadlock. -static struct InitGwpAsan -{ - InitGwpAsan() - { - gwp_asan::options::initOptions(); - gwp_asan::options::Options &opts = gwp_asan::options::getOptions(); - GuardedAlloc.init(opts); - - ///std::cerr << "GwpAsan is initialized, the options are { Enabled: " << opts.Enabled - /// << ", MaxSimultaneousAllocations: " << opts.MaxSimultaneousAllocations - /// << ", SampleRate: " << opts.SampleRate << " }\n"; - } -} init_gwp_asan; -#endif - /// Replace default new/delete with memory tracking versions. /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new /// https://en.cppreference.com/w/cpp/memory/new/operator_delete From 6753a0ad188ce386e7f3059eaae7e293cd4c3087 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 May 2024 16:02:12 +0200 Subject: [PATCH 129/856] Better --- src/CMakeLists.txt | 2 +- src/Common/Allocator.cpp | 2 -- src/Common/AsynchronousMetrics.cpp | 1 - src/Common/clickhouse_malloc.cpp | 5 ++--- src/Common/memory.cpp | 3 ++- src/Daemon/BaseDaemon.cpp | 14 ++++++++++++++ 6 files changed, 19 insertions(+), 8 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 64a09699a54..537cdc7887a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -181,7 +181,7 @@ add_library (clickhouse_new_delete STATIC Common/new_delete.cpp) target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io) if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::jemalloc) - target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::jemalloc) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::jemalloc) target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc) endif() diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 80f3b204c74..8b68ef87298 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -94,9 +94,7 @@ void * allocNoTrack(size_t size, size_t alignment) if constexpr (clear_memory) buf = ::calloc(size, 1); else - { buf = ::malloc(size); - } if (nullptr == buf) throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot malloc {}.", ReadableSize(size)); diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index ccd65af07f5..4c71b9846c7 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index a9961eb4ee4..afdad3c6599 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -1,14 +1,13 @@ #include #include + /** These functions can be substituted instead of regular ones when memory tracking is needed. */ extern "C" void * clickhouse_malloc(size_t size) { - void * res = nullptr; - res = malloc(size); - + void * res = malloc(size); if (res) { AllocationTrace trace; diff --git a/src/Common/memory.cpp b/src/Common/memory.cpp index 862af4470f1..6c17dbe3ba1 100644 --- a/src/Common/memory.cpp +++ b/src/Common/memory.cpp @@ -1,7 +1,8 @@ -#include #include #if USE_GWP_ASAN +#include + namespace Memory { gwp_asan::GuardedPoolAllocator GuardedAlloc; diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index c6c82df2a72..74c37b6123b 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -156,6 +157,19 @@ static void signalHandler(int sig, siginfo_t * info, void * context) const ucontext_t * signal_context = reinterpret_cast(context); const StackTrace stack_trace(*signal_context); + const auto is_gwp_asan = [&] + { + auto state = ::Memory::GuardedAlloc.getAllocatorState(); + if (state->FailureType != gwp_asan::Error::UNKNOWN && state->FailureAddress != 0) + return true; + + auto addr = reinterpret_cast(info->si_addr); + return addr < state->GuardedPagePoolEnd && state->GuardedPagePool <= addr; + }; + + if (is_gwp_asan()) + std::cerr << "GWPAsan caught something!" << std::endl; + writeBinary(sig, out); writePODBinary(*info, out); writePODBinary(signal_context, out); From 33306993c132508d93a9bd2c4d063e73d1d6165d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 24 May 2024 18:02:45 +0000 Subject: [PATCH 130/856] Better --- src/Functions/geometryConverters.h | 10 +++++++--- src/Functions/readWkt.cpp | 25 ++++++++++++++++++++++++- src/Functions/wkt.cpp | 8 ++++++++ 3 files changed, 39 insertions(+), 4 deletions(-) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 1478de14c4d..03831d37e0c 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -405,12 +405,16 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) /// There is no Point type, because for most of geometry functions it is useless. if (factory.get("Point")->equals(*type)) return f(ConverterType>()); + /// We should take the name into consideration to avoid ambiguity. - /// Because for example both Ring and LineString are resolved to Array. - else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "Ring") + /// Because for example both Ring and LineString are resolved to Array(Tuple(Point)). + else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "LineString") return f(ConverterType>()); - else if (factory.get("Ring")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "Ring") + + /// For backward compatibility if we call this function not on a custom type, we will consider Array(Tuple(Point)) as type Ring. + else if (factory.get("Ring")->equals(*type)) return f(ConverterType>()); + else if (factory.get("Polygon")->equals(*type)) return f(ConverterType>()); else if (factory.get("MultiPolygon")->equals(*type)) diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index f4975e85d0d..ccd969af58b 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -107,7 +107,30 @@ struct ReadWKTMultiPolygonNameHolder REGISTER_FUNCTION(ReadWKT) { factory.registerFunction, ReadWKTPointNameHolder>>(); - factory.registerFunction, ReadWKTLineStringNameHolder>>(); + factory.registerFunction, ReadWKTLineStringNameHolder>>(FunctionDocumentation + { + .description=R"( +Parses a Well-Known Text (WKT) representation of a LineString geometry and returns it in the internal ClickHouse format. +)", + .syntax = "readWKTLineString(wkt_string)", + .arguments{ + {"wkt_string", "The input WKT string representing a LineString geometry."} + }, + .returned_value = "The function returns a ClickHouse internal representation of the linestring geometry.", + .examples{ + {"first call", "SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)');", R"( + ┌─readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')─┐ +1. │ [(1,1),(2,2),(3,3),(1,1)] │ + └──────────────────────────────────────────────────────┘ + )"}, + {"second call", "SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'));", R"( + ┌─toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'))─┐ +1. │ LineString │ + └──────────────────────────────────────────────────────────────────┘ + )"}, + }, + .categories{"Unique identifiers"} + }); factory.registerFunction, ReadWKTRingNameHolder>>(); factory.registerFunction, ReadWKTPolygonNameHolder>>(); factory.registerFunction, ReadWKTMultiPolygonNameHolder>>(); diff --git a/src/Functions/wkt.cpp b/src/Functions/wkt.cpp index afcfabd0bf4..678ec02d229 100644 --- a/src/Functions/wkt.cpp +++ b/src/Functions/wkt.cpp @@ -41,6 +41,14 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + /* + * Functions like recursiveRemoveLowCardinality don't pay enough attention to custom types and just erase + * the information about it during type conversions. + * While it is a big problem the quick solution would be just to disable default low cardinality implementation + * because it doesn't make a lot of sense for geo types. + */ + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { auto res_column = ColumnString::create(); From 1fa17f1035012c4f9a215846a35fe74bbb21d3c4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 24 May 2024 19:09:39 +0000 Subject: [PATCH 131/856] Fixed style --- src/Functions/readWkt.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index ccd969af58b..eb262777b0d 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -119,14 +119,14 @@ Parses a Well-Known Text (WKT) representation of a LineString geometry and retur .returned_value = "The function returns a ClickHouse internal representation of the linestring geometry.", .examples{ {"first call", "SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)');", R"( - ┌─readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')─┐ -1. │ [(1,1),(2,2),(3,3),(1,1)] │ - └──────────────────────────────────────────────────────┘ +┌─readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')─┐ +│ [(1,1),(2,2),(3,3),(1,1)] │ +└──────────────────────────────────────────────────────┘ )"}, {"second call", "SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'));", R"( - ┌─toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'))─┐ -1. │ LineString │ - └──────────────────────────────────────────────────────────────────┘ +┌─toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'))─┐ +│ LineString │ +└──────────────────────────────────────────────────────────────────┘ )"}, }, .categories{"Unique identifiers"} From 8f6e4c05ea40b14d9d79646e9180ca5480e609dc Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 24 May 2024 21:34:53 +0200 Subject: [PATCH 132/856] Fix broken link --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index a137eb2bdf2..1c7df32db7f 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -479,7 +479,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. - [input_format_csv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. -- [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. +- [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - allow variable number of columns in CSV format, ignore extra columns and use default values on missing columns. Default value - `false`. - [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. - [input_format_csv_try_infer_numbers_from_strings](/docs/en/operations/settings/settings-formats.md/#input_format_csv_try_infer_numbers_from_strings) - Try to infer numbers from string fields while schema inference. Default value - `false`. From e6f83386665d1c129e5b94c82cdd643215f198c6 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 24 May 2024 20:38:45 +0000 Subject: [PATCH 133/856] 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 6725168b983d197d4e00234785e02373327fd30c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 May 2024 21:34:33 +0200 Subject: [PATCH 134/856] better code, add warning --- src/Databases/DatabaseReplicated.cpp | 9 ++++++++- src/Databases/DatabaseReplicated.h | 2 ++ src/Interpreters/DDLTask.cpp | 4 ++-- src/Storages/System/StorageSystemClusters.cpp | 2 +- 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index c9e14790175..d2a3a5d421c 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -122,6 +122,13 @@ DatabaseReplicated::DatabaseReplicated( fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef()); replica_group_name = context_->getConfigRef().getString("replica_group_name", ""); + + if (!replica_group_name.empty() && database_name.starts_with(DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX)) + { + context_->addWarningMessage(fmt::format("There's a Replicated database with a name starting from '{}', " + "and replica_group_name is configured. It may cause collisions in cluster names.", + ALL_GROUPS_CLUSTER_PREFIX)); + } } String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica) @@ -311,7 +318,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const String cluster_name = TSA_SUPPRESS_WARNING_FOR_READ(database_name); /// FIXME if (all_groups) - cluster_name = "all_groups." + cluster_name; + cluster_name = ALL_GROUPS_CLUSTER_PREFIX + cluster_name; ClusterConnectionParameters params{ cluster_auth_info.cluster_username, diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index f902b45ca86..761d6b4b503 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -20,6 +20,8 @@ using ClusterPtr = std::shared_ptr; class DatabaseReplicated : public DatabaseAtomic { public: + static constexpr auto ALL_GROUPS_CLUSTER_PREFIX = "all_groups."; + DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid, const String & zookeeper_path_, const String & shard_name_, const String & replica_name_, DatabaseReplicatedSettings db_settings_, diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 06ec9489fc1..6c346836ed8 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -570,9 +570,9 @@ ClusterPtr tryGetReplicatedDatabaseCluster(const String & cluster_name) { String name = cluster_name; bool all_groups = false; - if (name.starts_with("all_groups.")) + if (name.starts_with(DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX)) { - name = name.substr(strlen("all_groups.")); + name = name.substr(strlen(DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX)); all_groups = true; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 520bd7e7f92..160c8d6270e 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -56,7 +56,7 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co replicated->tryGetAreReplicasActive(database_cluster)); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {"all_groups." + name_and_database.first, database_cluster}, + writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, replicated->tryGetAreReplicasActive(database_cluster)); } } From b08ecfe6c0ee5b842f30999257daa3ae89ba0916 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Sat, 25 May 2024 10:40:51 +0000 Subject: [PATCH 135/856] 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 44b787c1c906881850593cc5dcf8c9c0989525d1 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 26 May 2024 08:31:01 +0000 Subject: [PATCH 136/856] Remove wrong comment --- src/Common/threadPoolCallbackRunner.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/threadPoolCallbackRunner.h b/src/Common/threadPoolCallbackRunner.h index 5beec660801..afbdcf2df19 100644 --- a/src/Common/threadPoolCallbackRunner.h +++ b/src/Common/threadPoolCallbackRunner.h @@ -54,7 +54,6 @@ ThreadPoolCallbackRunnerUnsafe threadPoolCallbackRunnerUnsafe( auto future = task->get_future(); - /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". /// Note: calling method scheduleOrThrowOnError in intentional, because we don't want to throw exceptions /// in critical places where this callback runner is used (e.g. loading or deletion of parts) my_pool->scheduleOrThrowOnError([my_task = std::move(task)]{ (*my_task)(); }, priority); @@ -163,7 +162,6 @@ public: task->future = task_func->get_future(); - /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". /// Note: calling method scheduleOrThrowOnError in intentional, because we don't want to throw exceptions /// in critical places where this callback runner is used (e.g. loading or deletion of parts) pool.scheduleOrThrowOnError([my_task = std::move(task_func)]{ (*my_task)(); }, priority); From cea82aab5970eeddad04cbabc27407c0c1dc0ff9 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 26 May 2024 20:43:49 +0000 Subject: [PATCH 137/856] add dynamic untracked memory limits for more precise memory tracking --- src/Common/CurrentMemoryTracker.cpp | 9 +++++++++ src/Common/CurrentMemoryTracker.h | 2 ++ src/Common/ThreadStatus.h | 12 ++++++++++++ src/Core/Settings.h | 1 + src/Interpreters/ThreadStatusExt.cpp | 10 +++++++--- 5 files changed, 31 insertions(+), 3 deletions(-) diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 02c7dc6e224..6166119eccf 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -57,6 +57,7 @@ AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory { auto res = memory_tracker->allocImpl(will_be, throw_if_memory_exceeded); current_thread->untracked_memory = 0; + current_thread->updateUntrackedMemoryLimit(memory_tracker->get()); return res; } else @@ -84,6 +85,13 @@ void CurrentMemoryTracker::check() std::ignore = memory_tracker->allocImpl(0, true); } +Int64 CurrentMemoryTracker::get() +{ + if (auto * memory_tracker = getMemoryTracker()) + return memory_tracker->get(); + return 0; +} + AllocationTrace CurrentMemoryTracker::alloc(Int64 size) { bool throw_if_memory_exceeded = true; @@ -107,6 +115,7 @@ AllocationTrace CurrentMemoryTracker::free(Int64 size) { Int64 untracked_memory = current_thread->untracked_memory; current_thread->untracked_memory = 0; + current_thread->updateUntrackedMemoryLimit(memory_tracker->get() + untracked_memory); return memory_tracker->free(-untracked_memory); } } diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index 18a1e3f49b1..401eeed93dd 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -12,7 +12,9 @@ struct CurrentMemoryTracker /// This function should be called after memory deallocation. [[nodiscard]] static AllocationTrace free(Int64 size); + static void check(); + [[nodiscard]] static Int64 get(); /// Throws MEMORY_LIMIT_EXCEEDED (if it's allowed to throw exceptions) static void injectFault(); diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 0c02ab8fdb0..04fb568540b 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -183,6 +183,12 @@ public: Int64 untracked_memory = 0; /// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters. Int64 untracked_memory_limit = 4 * 1024 * 1024; + /// To keep total untracked memory limited to `untracked_memory_ratio * RSS` we have to account threads with small and large memory footprint differently. + /// For this purpose we dynamically change `untracked_memory_limit` after every tracking event using a simple formula: + /// untracked_memory_limit = clamp(untracked_memory_ratio * cur_memory_bytes, min_untracked_memory, max_untracked_memory) + /// Note that this values are updated when thread is attached to a group + Int64 min_untracked_memory = 4 * 1024 * 1024; + Int64 max_untracked_memory = 4 * 1024; /// Statistics of read and write rows/bytes Progress progress_in; @@ -309,6 +315,12 @@ public: void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period); + void updateUntrackedMemoryLimit(Int64 current) + { + constexpr Int64 untracked_memory_ratio_bits = 4; // untracked_memory_ratio = 1.0 / (1 << untracked_memory_ratio_bits) = 1.0 / 16 = 6.25% + untracked_memory_limit = std::clamp(current >> untracked_memory_ratio_bits, min_untracked_memory, max_untracked_memory); + } + private: void applyGlobalSettings(); void applyQuerySettings(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f0389e7e2d5..28b068b9e37 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -489,6 +489,7 @@ class IColumn; M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ + M(UInt64, min_untracked_memory, (4 * 1024), "Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread_memory_usage/16 and clamped between min_untracked_memory and max_untracked_memory for every thread.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 9ca521a4ab3..981c7d45d8e 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -210,9 +211,12 @@ void ThreadStatus::applyQuerySettings() query_id_from_query_context = query_context_ptr->getCurrentQueryId(); initQueryProfiler(); - untracked_memory_limit = settings.max_untracked_memory; - if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast(untracked_memory_limit)) - untracked_memory_limit = settings.memory_profiler_step; + max_untracked_memory = settings.max_untracked_memory; + if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast(max_untracked_memory)) + max_untracked_memory = settings.memory_profiler_step; + min_untracked_memory = std::min(settings.min_untracked_memory, max_untracked_memory); + + updateUntrackedMemoryLimit(CurrentMemoryTracker::get()); #if defined(OS_LINUX) /// Set "nice" value if required. From b30d11f046c3bb80612e4510f863c78200a98b93 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Mon, 27 May 2024 16:02:15 +0800 Subject: [PATCH 138/856] adapting parquet reader output block rows --- src/Core/Settings.h | 3 ++- src/Core/SettingsChangesHistory.h | 2 ++ src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 3 ++- .../Formats/Impl/ParquetBlockInputFormat.cpp | 19 ++++++++++++++++++- .../Formats/Impl/ParquetBlockInputFormat.h | 2 ++ 6 files changed, 27 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f0389e7e2d5..c9efd1e4a97 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1053,7 +1053,8 @@ class IColumn; M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ - M(UInt64, input_format_parquet_max_block_size, 8192, "Max block size for parquet reader.", 0) \ + M(UInt64, input_format_parquet_max_block_size, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader.", 0) \ + M(UInt64, input_format_parquet_prefer_block_bytes, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader", 0) \ M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format Protobuf", 0) \ M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format CapnProto", 0) \ M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format ORC", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 16f28d94640..be031592c12 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,6 +92,8 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader."}, + {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a7883919c4c..e90986f2236 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -161,6 +161,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.parquet.output_string_as_string = settings.output_format_parquet_string_as_string; format_settings.parquet.output_fixed_string_as_fixed_byte_array = settings.output_format_parquet_fixed_string_as_fixed_byte_array; format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; + format_settings.parquet.prefer_block_bytes = settings.input_format_parquet_prefer_block_bytes; format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; format_settings.parquet.output_compliant_nested_types = settings.output_format_parquet_compliant_nested_types; format_settings.parquet.use_custom_encoder = settings.output_format_parquet_use_custom_encoder; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index b296928e4d4..337aafbbe9c 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -265,7 +265,8 @@ struct FormatSettings bool preserve_order = false; bool use_custom_encoder = true; bool parallel_encoding = true; - UInt64 max_block_size = 8192; + UInt64 max_block_size = DEFAULT_BLOCK_SIZE; + size_t prefer_block_bytes = DEFAULT_BLOCK_SIZE * 256; ParquetVersion output_version; ParquetCompression output_compression_method = ParquetCompression::SNAPPY; bool output_compliant_nested_types = true; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 7fc7b9c3cab..fd2e7f88fb2 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -418,6 +418,21 @@ void ParquetBlockInputFormat::initializeIfNeeded() int num_row_groups = metadata->num_row_groups(); row_group_batches.reserve(num_row_groups); + auto adative_chunk_size = [&](int row_group_idx) -> size_t + { + size_t total_size = 0; + auto row_group_meta = metadata->RowGroup(row_group_idx); + for (int column_index : column_indices) + { + total_size += row_group_meta->ColumnChunk(column_index)->total_uncompressed_size(); + } + if (!total_size || !format_settings.parquet.prefer_block_bytes) return 0; + auto average_row_bytes = total_size / row_group_meta->num_rows(); + /// max_block_bytes >= num_rows >= 128 + auto num_rows = std::min(format_settings.parquet.prefer_block_bytes/average_row_bytes, format_settings.parquet.max_block_size); + return std::max(num_rows, 128UL); + }; + for (int row_group = 0; row_group < num_row_groups; ++row_group) { if (skip_row_groups.contains(row_group)) @@ -437,6 +452,8 @@ void ParquetBlockInputFormat::initializeIfNeeded() row_group_batches.back().row_groups_idxs.push_back(row_group); row_group_batches.back().total_rows += metadata->RowGroup(row_group)->num_rows(); row_group_batches.back().total_bytes_compressed += metadata->RowGroup(row_group)->total_compressed_size(); + auto rows = adative_chunk_size(row_group); + row_group_batches.back().adaptive_chunk_size = rows ? format_settings.parquet.max_block_size :rows; } } @@ -446,7 +463,7 @@ void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_bat parquet::ArrowReaderProperties properties; properties.set_use_threads(false); - properties.set_batch_size(format_settings.parquet.max_block_size); + properties.set_batch_size(row_group_batch.adaptive_chunk_size); // When reading a row group, arrow will: // 1. Look at `metadata` to get all byte ranges it'll need to read from the file (typically one diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index d6591f5c0a3..24735ee4371 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -208,6 +208,8 @@ private: size_t total_rows = 0; size_t total_bytes_compressed = 0; + size_t adaptive_chunk_size = 0; + std::vector row_groups_idxs; // These are only used by the decoding thread, so don't require locking the mutex. From 8484decf4891c7c8939688e3261ea3b5e4c53584 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 27 May 2024 10:32:17 +0200 Subject: [PATCH 139/856] Fix test fails --- .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 0c9060d261a..887b2ed94d7 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index 2384350fa85..371ae87c2ef 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" From 74c18bcdb4bb380cecee77c724df3c75a401c170 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Mon, 27 May 2024 17:54:34 +0800 Subject: [PATCH 140/856] fix comment error --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index fd2e7f88fb2..1f423d41327 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -428,7 +428,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() } if (!total_size || !format_settings.parquet.prefer_block_bytes) return 0; auto average_row_bytes = total_size / row_group_meta->num_rows(); - /// max_block_bytes >= num_rows >= 128 + /// max_block_size >= num_rows >= 128 auto num_rows = std::min(format_settings.parquet.prefer_block_bytes/average_row_bytes, format_settings.parquet.max_block_size); return std::max(num_rows, 128UL); }; From 15fc35699fe3a9cbfbb024deb587cafbf665f781 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Mon, 27 May 2024 18:00:16 +0800 Subject: [PATCH 141/856] fix --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 1f423d41327..95da938f4e6 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -453,7 +453,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() row_group_batches.back().total_rows += metadata->RowGroup(row_group)->num_rows(); row_group_batches.back().total_bytes_compressed += metadata->RowGroup(row_group)->total_compressed_size(); auto rows = adative_chunk_size(row_group); - row_group_batches.back().adaptive_chunk_size = rows ? format_settings.parquet.max_block_size :rows; + row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_size; } } From 80f195d2b983c2db2feb2d5924d06588a7382d9c Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 May 2024 18:46:07 +0200 Subject: [PATCH 142/856] Refactor s3 settings --- src/Backups/BackupIO_S3.cpp | 6 +- src/Backups/BackupIO_S3.h | 2 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 7 +- src/Coordination/Standalone/Context.cpp | 2 +- src/Core/Settings.h | 23 +- .../ObjectStorages/ObjectStorageFactory.cpp | 23 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 23 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 6 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 109 +++-- src/Disks/ObjectStorages/S3/diskSettings.h | 17 +- src/IO/ReadBufferFromS3.cpp | 4 +- src/IO/ReadBufferFromS3.h | 6 +- src/IO/S3/Credentials.h | 8 +- src/IO/S3/copyS3File.cpp | 18 +- src/IO/S3/copyS3File.h | 6 +- src/IO/S3/getObjectInfo.cpp | 16 +- src/IO/S3/getObjectInfo.h | 8 +- src/IO/S3/tests/gtest_aws_s3_client.cpp | 6 +- src/IO/S3Common.cpp | 379 ++++++++++++++++-- src/IO/S3Common.h | 86 +++- src/IO/S3Defines.h | 32 ++ src/IO/S3Settings.cpp | 62 +++ src/IO/S3Settings.h | 45 +++ src/IO/WriteBufferFromS3.cpp | 10 +- src/IO/WriteBufferFromS3.h | 8 +- src/IO/tests/gtest_writebuffer_s3.cpp | 4 +- src/Interpreters/Context.cpp | 10 +- src/Interpreters/Context.h | 4 +- .../ExternalDataSourceConfiguration.h | 2 +- .../ObjectStorage/S3/Configuration.cpp | 8 +- src/Storages/ObjectStorage/S3/Configuration.h | 4 +- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- src/Storages/StorageS3Settings.cpp | 315 --------------- src/Storages/StorageS3Settings.h | 122 ------ 34 files changed, 717 insertions(+), 666 deletions(-) create mode 100644 src/IO/S3Defines.h create mode 100644 src/IO/S3Settings.cpp create mode 100644 src/IO/S3Settings.h delete mode 100644 src/Storages/StorageS3Settings.cpp delete mode 100644 src/Storages/StorageS3Settings.h diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 92e208ba464..cbf18e2bff9 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -134,7 +134,7 @@ BackupReaderS3::BackupReaderS3( , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup).value_or(S3Settings{})) { auto & request_settings = s3_settings.request_settings; - request_settings.updateFromSettingsIfChanged(context_->getSettingsRef()); + request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint request_settings.allow_native_copy = allow_s3_native_copy; client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); @@ -225,10 +225,10 @@ BackupWriterS3::BackupWriterS3( , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup).value_or(S3Settings{})) { auto & request_settings = s3_settings.request_settings; - request_settings.updateFromSettingsIfChanged(context_->getSettingsRef()); + request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint request_settings.allow_native_copy = allow_s3_native_copy; - request_settings.setStorageClassName(storage_class_name); + request_settings.upload_settings.storage_class_name = storage_class_name; client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); if (auto blob_storage_system_log = context_->getBlobStorageLog()) { diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index f81eb975df3..327f06363c5 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index b984b8ad18e..acf8faa9edd 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -64,7 +65,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo return; } - auto auth_settings = S3::AuthSettings::loadFromConfig(config_prefix, config); + auto auth_settings = S3::AuthSettings::loadFromConfig(config, config_prefix, Context::getGlobalContextInstance()->getSettingsRef()); String endpoint = macros->expand(config.getString(config_prefix + ".endpoint")); auto new_uri = S3::URI{endpoint}; @@ -154,7 +155,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapsh if (s3_client == nullptr) return; - S3Settings::RequestSettings request_settings_1; + S3::RequestSettings request_settings_1; const auto create_writer = [&](const auto & key) { @@ -197,7 +198,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapsh lock_writer.finalize(); // We read back the written UUID, if it's the same we can upload the file - S3Settings::RequestSettings request_settings_2; + S3::RequestSettings request_settings_2; request_settings_2.max_single_read_retries = 1; ReadBufferFromS3 lock_reader { diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 4b14b038852..2802d51ae26 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f0389e7e2d5..a7a19702282 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -81,17 +82,19 @@ class IColumn; M(UInt64, s3_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to S3 (some implementations does not supports variable size parts).", 0) \ M(UInt64, azure_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to Azure blob storage.", 0) \ M(UInt64, azure_max_blocks_in_multipart_upload, 50000, "Maximum number of blocks in multipart upload for Azure.", 0) \ - M(UInt64, s3_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ - M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \ + M(UInt64, s3_min_upload_part_size, S3::DEFAULT_MIN_UPLOAD_PART_SIZE, "The minimum size of part to upload during multipart upload to S3.", 0) \ + M(UInt64, s3_max_upload_part_size, S3::DEFAULT_MAX_UPLOAD_PART_SIZE, "The maximum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, azure_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage.", 0) \ M(UInt64, azure_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage.", 0) \ - M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \ - M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3, s3_min_upload_part_size is multiplied by s3_upload_part_size_multiply_factor.", 0) \ + M(UInt64, s3_upload_part_size_multiply_factor, S3::DEFAULT_UPLOAD_PART_SIZE_MULTIPLY_FACTOR, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \ + M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, S3::DEFAULT_UPLOAD_PART_SIZE_MULTIPLY_PARTS_COUNT_THRESHOLD, "Each time this number of parts was uploaded to S3, s3_min_upload_part_size is multiplied by s3_upload_part_size_multiply_factor.", 0) \ + M(UInt64, s3_max_part_number, S3::DEFAULT_MAX_PART_NUMBER, "Maximum part number number for s3 upload part.", 0) \ + M(UInt64, s3_max_single_operation_copy_size, S3::DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE, "Maximum size for a single copy operation in s3", 0) \ M(UInt64, azure_upload_part_size_multiply_factor, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage.", 0) \ M(UInt64, azure_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor.", 0) \ - M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ + M(UInt64, s3_max_inflight_parts_for_one_file, S3::DEFAULT_MAX_INFLIGHT_PARTS_FOR_ONE_FILE, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ M(UInt64, azure_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ - M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ + M(UInt64, s3_max_single_part_upload_size, S3::DEFAULT_MAX_SINGLE_PART_UPLOAD_SIZE, "The maximum size of object to upload using singlepart upload to S3.", 0) \ M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ M(UInt64, azure_max_single_part_copy_size, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage.", 0) \ M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ @@ -99,13 +102,13 @@ class IColumn; M(UInt64, azure_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write", 0) \ M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ - M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ + M(UInt64, s3_max_connections, S3::DEFAULT_MAX_CONNECTIONS, "The maximum number of connections per server.", 0) \ M(UInt64, s3_max_get_rps, 0, "Limit on S3 GET request per second rate before throttling. Zero means unlimited.", 0) \ M(UInt64, s3_max_get_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_get_rps`", 0) \ M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate before throttling. Zero means unlimited.", 0) \ M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \ M(UInt64, s3_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ - M(Bool, s3_use_adaptive_timeouts, true, "When adaptive timeouts are enabled first two attempts are made with low receive and send timeout", 0) \ + M(Bool, s3_use_adaptive_timeouts, S3::DEFAULT_USE_ADAPTIVE_TIMEOUTS, "When adaptive timeouts are enabled first two attempts are made with low receive and send timeout", 0) \ M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ @@ -124,8 +127,8 @@ class IColumn; M(Bool, s3_validate_request_settings, true, "Validate S3 request settings", 0) \ M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ - M(UInt64, s3_request_timeout_ms, 30000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ - M(UInt64, s3_connect_timeout_ms, 1000, "Connection timeout for host from s3 disks.", 0) \ + M(UInt64, s3_request_timeout_ms, S3::DEFAULT_REQUEST_TIMEOUT_MS, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ + M(UInt64, s3_connect_timeout_ms, S3::DEFAULT_CONNECT_TIMEOUT_MS, "Connection timeout for host from s3 disks.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(String, s3queue_default_zookeeper_path, "/clickhouse/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \ M(Bool, s3queue_enable_logging_to_s3queue_log, false, "Enable writing to system.s3queue_log. The value can be overwritten per table with table settings", 0) \ diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index d7884c2911b..81de22811fe 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -169,6 +169,14 @@ void checkS3Capabilities( } } +static std::string getEndpoint( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const ContextPtr & context) +{ + return context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); +} + void registerS3ObjectStorage(ObjectStorageFactory & factory) { static constexpr auto disk_type = "s3"; @@ -182,8 +190,9 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) { auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); - auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings, true); + auto settings = getSettings(config, config_prefix, context, /* for_disk_s3 */true, /* validate_settings */true); + auto endpoint = getEndpoint(config, config_prefix, context); + auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); auto object_storage = createObjectStorage( @@ -218,8 +227,9 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); - auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings, true); + auto settings = getSettings(config, config_prefix, context, /* for_disk_s3 */true, /* validate_settings */true); + auto endpoint = getEndpoint(config, config_prefix, context); + auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); auto object_storage = std::make_shared>( @@ -252,8 +262,9 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); - auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings, true); + auto settings = getSettings(config, config_prefix, context, /* for_disk_s3 */true, /* validate_settings */true); + auto endpoint = getEndpoint(config, config_prefix, context); + auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); auto object_storage = std::make_shared>( diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index c07313b52db..416eebc7493 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -168,7 +168,7 @@ private: bool S3ObjectStorage::exists(const StoredObject & object) const { auto settings_ptr = s3_settings.get(); - return S3::objectExists(*client.get(), uri.bucket, object.remote_path, {}, settings_ptr->request_settings); + return S3::objectExists(*client.get(), uri.bucket, object.remote_path, {}); } std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT @@ -258,10 +258,10 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (mode != WriteMode::Rewrite) throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); - S3Settings::RequestSettings request_settings = s3_settings.get()->request_settings; + S3::RequestSettings request_settings = s3_settings.get()->request_settings; if (auto query_context = CurrentThread::getQueryContext()) { - request_settings.updateFromSettingsIfChanged(query_context->getSettingsRef()); + request_settings.updateFromSettings(query_context->getSettingsRef(), /* if_changed */true); } ThreadPoolCallbackRunnerUnsafe scheduler; @@ -440,8 +440,7 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s { auto settings_ptr = s3_settings.get(); auto object_info = S3::getObjectInfo( - *client.get(), uri.bucket, path, {}, settings_ptr->request_settings, - /* with_metadata= */ true, /* throw_on_error= */ false); + *client.get(), uri.bucket, path, {}, /* with_metadata= */ true, /* throw_on_error= */ false); if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) return {}; @@ -460,7 +459,7 @@ ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) cons S3::ObjectInfo object_info; try { - object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true); + object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, /* with_metadata= */ true); } catch (DB::Exception & e) { @@ -489,7 +488,7 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT { auto current_client = dest_s3->client.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*current_client, uri.bucket, object_from.remote_path, {}, settings_ptr->request_settings); + auto size = S3::getObjectSize(*current_client, uri.bucket, object_from.remote_path, {}); auto scheduler = threadPoolCallbackRunnerUnsafe(getThreadPoolWriter(), "S3ObjStor_copy"); try @@ -532,7 +531,7 @@ void S3ObjectStorage::copyObject( // NOLINT { auto current_client = client.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*current_client, uri.bucket, object_from.remote_path, {}, settings_ptr->request_settings); + auto size = S3::getObjectSize(*current_client, uri.bucket, object_from.remote_path, {}); auto scheduler = threadPoolCallbackRunnerUnsafe(getThreadPoolWriter(), "S3ObjStor_copy"); copyS3File(current_client, @@ -575,7 +574,7 @@ void S3ObjectStorage::applyNewSettings( ContextPtr context, const ApplyNewSettingsOptions & options) { - auto new_s3_settings = getSettings(config, config_prefix, context, context->getSettingsRef().s3_validate_request_settings); + auto new_s3_settings = getSettings(config, config_prefix, context, for_disk_s3, context->getSettingsRef().s3_validate_request_settings); if (!static_headers.empty()) { new_s3_settings->auth_settings.headers.insert( @@ -589,7 +588,7 @@ void S3ObjectStorage::applyNewSettings( auto current_s3_settings = s3_settings.get(); if (options.allow_client_change && (current_s3_settings->auth_settings.hasUpdates(new_s3_settings->auth_settings) || for_disk_s3)) { - auto new_client = getClient(config, config_prefix, context, *new_s3_settings, for_disk_s3, &uri); + auto new_client = getClient(uri, *new_s3_settings, context, for_disk_s3); client.set(std::move(new_client)); } s3_settings.set(std::move(new_s3_settings)); @@ -601,8 +600,8 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( const std::string & config_prefix, ContextPtr context) { - auto new_s3_settings = getSettings(config, config_prefix, context); - auto new_client = getClient(config, config_prefix, context, *new_s3_settings, true); + auto new_s3_settings = getSettings(config, config_prefix, context, for_disk_s3, true); + auto new_client = getClient(uri, *new_s3_settings, context, for_disk_s3); auto new_uri{uri}; new_uri.bucket = new_namespace; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 1fff6d67e23..b7b7ba6ee8d 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include @@ -20,7 +20,7 @@ struct S3ObjectStorageSettings S3ObjectStorageSettings() = default; S3ObjectStorageSettings( - const S3Settings::RequestSettings & request_settings_, + const S3::RequestSettings & request_settings_, const S3::AuthSettings & auth_settings_, uint64_t min_bytes_for_seek_, int32_t list_object_keys_size_, @@ -34,7 +34,7 @@ struct S3ObjectStorageSettings , read_only(read_only_) {} - S3Settings::RequestSettings request_settings; + S3::RequestSettings request_settings; S3::AuthSettings auth_settings; uint64_t min_bytes_for_seek; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 139472a8b01..f66af556ce1 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include #include @@ -18,18 +19,12 @@ #include #include -#include +#include #include #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace ErrorCodes { extern const int NO_ELEMENTS_IN_CONFIG; @@ -39,11 +34,14 @@ std::unique_ptr getSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, + bool for_disk_s3, bool validate_settings) { - const Settings & settings = context->getSettingsRef(); - auto request_settings = S3Settings::RequestSettings(config, config_prefix, settings, "s3_", validate_settings); - auto auth_settings = S3::AuthSettings::loadFromConfig(config_prefix, config); + const auto & settings = context->getSettingsRef(); + const std::string setting_name_prefix = for_disk_s3 ? "s3_" : ""; + + auto auth_settings = S3::AuthSettings::loadFromConfig(config, config_prefix, settings); + auto request_settings = S3::RequestSettings::loadFromConfig(config, config_prefix, settings, validate_settings, setting_name_prefix); return std::make_unique( request_settings, @@ -55,38 +53,33 @@ std::unique_ptr getSettings( } std::unique_ptr getClient( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - ContextPtr context, + const std::string & endpoint, const S3ObjectStorageSettings & settings, - bool for_disk_s3, - const S3::URI * url_) + ContextPtr context, + bool for_disk_s3) +{ + auto url = S3::URI(endpoint); + if (!url.key.ends_with('/')) + url.key.push_back('/'); + return getClient(url, settings, context, for_disk_s3); +} + +std::unique_ptr getClient( + const S3::URI & url, + const S3ObjectStorageSettings & settings, + ContextPtr context, + bool for_disk_s3) { const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); - const Settings & local_settings = context->getSettingsRef(); - const auto & auth_settings = settings.auth_settings; const auto & request_settings = settings.request_settings; - S3::URI url; - if (for_disk_s3) - { - String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); - url = S3::URI(endpoint); - if (!url.key.ends_with('/')) - url.key.push_back('/'); - } - else - { - if (!url_) - throw Exception(ErrorCodes::LOGICAL_ERROR, "URL not passed"); - url = *url_; - } const bool is_s3_express_bucket = S3::isS3ExpressEndpoint(url.endpoint); - if (is_s3_express_bucket && !config.has(config_prefix + ".region")) + if (is_s3_express_bucket && auth_settings.region.empty()) { throw Exception( - ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Region should be explicitly specified for directory buckets ({})", config_prefix); + ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "Region should be explicitly specified for directory buckets"); } S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( @@ -96,49 +89,47 @@ std::unique_ptr getClient( static_cast(global_settings.s3_retry_attempts), global_settings.enable_s3_requests_logging, for_disk_s3, - settings.request_settings.get_request_throttler, - settings.request_settings.put_request_throttler, + request_settings.get_request_throttler, + request_settings.put_request_throttler, url.uri.getScheme()); - client_configuration.connectTimeoutMs = config.getUInt64(config_prefix + ".connect_timeout_ms", local_settings.s3_connect_timeout_ms.value); - client_configuration.requestTimeoutMs = config.getUInt64(config_prefix + ".request_timeout_ms", local_settings.s3_request_timeout_ms.value); - client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", static_cast(request_settings.max_connections)); - client_configuration.http_keep_alive_timeout = config.getUInt(config_prefix + ".http_keep_alive_timeout", S3::DEFAULT_KEEP_ALIVE_TIMEOUT); - client_configuration.http_keep_alive_max_requests = config.getUInt(config_prefix + ".http_keep_alive_max_requests", S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS); + client_configuration.connectTimeoutMs = auth_settings.connect_timeout_ms.value_or(S3::DEFAULT_CONNECT_TIMEOUT_MS); + client_configuration.requestTimeoutMs = auth_settings.request_timeout_ms.value_or(S3::DEFAULT_REQUEST_TIMEOUT_MS); + client_configuration.maxConnections = static_cast(auth_settings.max_connections.value_or(S3::DEFAULT_MAX_CONNECTIONS)); + client_configuration.http_keep_alive_timeout = auth_settings.http_keep_alive_timeout.value_or(S3::DEFAULT_KEEP_ALIVE_TIMEOUT); + client_configuration.http_keep_alive_max_requests = auth_settings.http_keep_alive_max_requests.value_or(S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS); client_configuration.endpointOverride = url.endpoint; - client_configuration.s3_use_adaptive_timeouts = config.getBool( - config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); + client_configuration.s3_use_adaptive_timeouts = auth_settings.use_adaptive_timeouts.value_or(S3::DEFAULT_USE_ADAPTIVE_TIMEOUTS); if (for_disk_s3) { + /// TODO: move to S3Common auth settings parsing /* * Override proxy configuration for backwards compatibility with old configuration format. * */ - if (auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( - ProxyConfiguration::protocolFromString(url.uri.getScheme()), config_prefix, config)) - { - client_configuration.per_request_configuration - = [proxy_config]() { return proxy_config->resolve(); }; - client_configuration.error_report - = [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); }; - } + // if (auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + // ProxyConfiguration::protocolFromString(url.uri.getScheme()), config_prefix, config)) + // { + // client_configuration.per_request_configuration + // = [proxy_config]() { return proxy_config->resolve(); }; + // client_configuration.error_report + // = [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); }; + // } } - S3::ServerSideEncryptionKMSConfig sse_kms_config = S3::getSSEKMSConfig(config_prefix, config); S3::ClientSettings client_settings{ .use_virtual_addressing = url.is_virtual_hosted_style, - .disable_checksum = local_settings.s3_disable_checksum, - .gcs_issue_compose_request = config.getBool("s3.gcs_issue_compose_request", false), - .is_s3express_bucket = is_s3_express_bucket, + .disable_checksum = auth_settings.disable_checksum.value_or(S3::DEFAULT_DISABLE_CHECKSUM), + .gcs_issue_compose_request = auth_settings.gcs_issue_compose_request.value_or(false), }; auto credentials_configuration = S3::CredentialsConfiguration { - auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), - auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - auth_settings.expiration_window_seconds.value_or(context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), - auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), + auth_settings.use_environment_credentials.value_or(S3::DEFAULT_USE_ENVIRONMENT_CREDENTIALS), + auth_settings.use_insecure_imds_request.value_or(false), + auth_settings.expiration_window_seconds.value_or(S3::DEFAULT_EXPIRATION_WINDOW_SECONDS), + auth_settings.no_sign_request.value_or(S3::DEFAULT_NO_SIGN_REQUEST), }; return S3::ClientFactory::instance().create( @@ -147,7 +138,7 @@ std::unique_ptr getClient( auth_settings.access_key_id, auth_settings.secret_access_key, auth_settings.server_side_encryption_customer_key_base64, - std::move(sse_kms_config), + auth_settings.server_side_encryption_kms_config, auth_settings.headers, credentials_configuration, auth_settings.session_token); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 11ac64ce913..5e7a18152d1 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -18,15 +18,20 @@ std::unique_ptr getSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, - bool validate_settings = true); + bool for_disk_s3, + bool validate_settings); std::unique_ptr getClient( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - ContextPtr context, + const std::string & endpoint, const S3ObjectStorageSettings & settings, - bool for_disk_s3, - const S3::URI * url_ = nullptr); + ContextPtr context, + bool for_disk_s3); + +std::unique_ptr getClient( + const S3::URI & url_, + const S3ObjectStorageSettings & settings, + ContextPtr context, + bool for_disk_s3); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 8823af55936..9e001232e65 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -51,7 +51,7 @@ ReadBufferFromS3::ReadBufferFromS3( const String & bucket_, const String & key_, const String & version_id_, - const S3Settings::RequestSettings & request_settings_, + const S3::RequestSettings & request_settings_, const ReadSettings & settings_, bool use_external_buffer_, size_t offset_, @@ -318,7 +318,7 @@ size_t ReadBufferFromS3::getFileSize() if (file_size) return *file_size; - auto object_size = S3::getObjectSize(*client_ptr, bucket, key, version_id, request_settings); + auto object_size = S3::getObjectSize(*client_ptr, bucket, key, version_id); file_size = object_size; return *file_size; diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 003c88df7d2..c6625c2d632 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include "config.h" #if USE_AWS_S3 @@ -28,7 +28,7 @@ private: String bucket; String key; String version_id; - const S3Settings::RequestSettings request_settings; + const S3::RequestSettings request_settings; /// These variables are atomic because they can be used for `logging only` /// (where it is not important to get consistent result) @@ -47,7 +47,7 @@ public: const String & bucket_, const String & key_, const String & version_id_, - const S3Settings::RequestSettings & request_settings_, + const S3::RequestSettings & request_settings_, const ReadSettings & settings_, bool use_external_buffer = false, size_t offset_ = 0, diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 8d586223035..b8698d9b302 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -13,18 +13,12 @@ # include # include +# include namespace DB::S3 { -inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; -inline static constexpr uint64_t DEFAULT_CONNECT_TIMEOUT_MS = 1000; -inline static constexpr uint64_t DEFAULT_REQUEST_TIMEOUT_MS = 30000; -inline static constexpr uint64_t DEFAULT_MAX_CONNECTIONS = 100; -inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_TIMEOUT = 5; -inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_MAX_REQUESTS = 100; - /// In GCP metadata service can be accessed via DNS regardless of IPv4 or IPv6. static inline constexpr char GCP_METADATA_SERVICE_ENDPOINT[] = "http://metadata.google.internal"; diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index cff6fa5ad21..471c4a687a6 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -56,7 +56,7 @@ namespace const std::shared_ptr & client_ptr_, const String & dest_bucket_, const String & dest_key_, - const S3Settings::RequestSettings & request_settings_, + const S3::RequestSettings & request_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunnerUnsafe schedule_, bool for_disk_s3_, @@ -66,7 +66,7 @@ namespace , dest_bucket(dest_bucket_) , dest_key(dest_key_) , request_settings(request_settings_) - , upload_settings(request_settings.getUploadSettings()) + , upload_settings(request_settings.upload_settings) , object_metadata(object_metadata_) , schedule(schedule_) , for_disk_s3(for_disk_s3_) @@ -81,8 +81,8 @@ namespace std::shared_ptr client_ptr; const String & dest_bucket; const String & dest_key; - const S3Settings::RequestSettings & request_settings; - const S3Settings::RequestSettings::PartUploadSettings & upload_settings; + const S3::RequestSettings & request_settings; + const S3::RequestSettings::PartUploadSettings & upload_settings; const std::optional> & object_metadata; ThreadPoolCallbackRunnerUnsafe schedule; bool for_disk_s3; @@ -239,7 +239,7 @@ namespace void checkObjectAfterUpload() { LOG_TRACE(log, "Checking object {} exists after upload", dest_key); - S3::checkObjectExists(*client_ptr, dest_bucket, dest_key, {}, request_settings, "Immediately after upload"); + S3::checkObjectExists(*client_ptr, dest_bucket, dest_key, {}, "Immediately after upload"); LOG_TRACE(log, "Object {} exists after upload", dest_key); } @@ -465,7 +465,7 @@ namespace const std::shared_ptr & client_ptr_, const String & dest_bucket_, const String & dest_key_, - const S3Settings::RequestSettings & request_settings_, + const S3::RequestSettings & request_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunnerUnsafe schedule_, bool for_disk_s3_, @@ -647,7 +647,7 @@ namespace size_t src_size_, const String & dest_bucket_, const String & dest_key_, - const S3Settings::RequestSettings & request_settings_, + const S3::RequestSettings & request_settings_, const ReadSettings & read_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunnerUnsafe schedule_, @@ -853,7 +853,7 @@ void copyDataToS3File( const std::shared_ptr & dest_s3_client, const String & dest_bucket, const String & dest_key, - const S3Settings::RequestSettings & settings, + const S3::RequestSettings & settings, BlobStorageLogWriterPtr blob_storage_log, const std::optional> & object_metadata, ThreadPoolCallbackRunnerUnsafe schedule, @@ -872,7 +872,7 @@ void copyS3File( size_t src_size, const String & dest_bucket, const String & dest_key, - const S3Settings::RequestSettings & settings, + const S3::RequestSettings & settings, const ReadSettings & read_settings, BlobStorageLogWriterPtr blob_storage_log, const std::optional> & object_metadata, diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index d5da4d260b1..c2cd8735188 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -4,7 +4,7 @@ #if USE_AWS_S3 -#include +#include #include #include #include @@ -38,7 +38,7 @@ void copyS3File( size_t src_size, const String & dest_bucket, const String & dest_key, - const S3Settings::RequestSettings & settings, + const S3::RequestSettings & settings, const ReadSettings & read_settings, BlobStorageLogWriterPtr blob_storage_log, const std::optional> & object_metadata = std::nullopt, @@ -57,7 +57,7 @@ void copyDataToS3File( const std::shared_ptr & dest_s3_client, const String & dest_bucket, const String & dest_key, - const S3Settings::RequestSettings & settings, + const S3::RequestSettings & settings, BlobStorageLogWriterPtr blob_storage_log, const std::optional> & object_metadata = std::nullopt, ThreadPoolCallbackRunnerUnsafe schedule_ = {}, diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index 78efda4ae57..9271ad820e4 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -44,7 +44,7 @@ namespace /// Performs a request to get the size and last modification time of an object. std::pair, Aws::S3::S3Error> tryGetObjectInfo( const S3::Client & client, const String & bucket, const String & key, const String & version_id, - const S3Settings::RequestSettings & /*request_settings*/, bool with_metadata) + bool with_metadata) { auto outcome = headObject(client, bucket, key, version_id); if (!outcome.IsSuccess()) @@ -73,11 +73,10 @@ ObjectInfo getObjectInfo( const String & bucket, const String & key, const String & version_id, - const S3Settings::RequestSettings & request_settings, bool with_metadata, bool throw_on_error) { - auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, with_metadata); + auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, with_metadata); if (object_info) { return *object_info; @@ -96,20 +95,18 @@ size_t getObjectSize( const String & bucket, const String & key, const String & version_id, - const S3Settings::RequestSettings & request_settings, bool throw_on_error) { - return getObjectInfo(client, bucket, key, version_id, request_settings, {}, throw_on_error).size; + return getObjectInfo(client, bucket, key, version_id, {}, throw_on_error).size; } bool objectExists( const S3::Client & client, const String & bucket, const String & key, - const String & version_id, - const S3Settings::RequestSettings & request_settings) + const String & version_id) { - auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, {}); + auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, {}); if (object_info) return true; @@ -126,10 +123,9 @@ void checkObjectExists( const String & bucket, const String & key, const String & version_id, - const S3Settings::RequestSettings & request_settings, std::string_view description) { - auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, {}); + auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, {}); if (object_info) return; throw S3Exception(error.GetErrorType(), "{}Object {} in bucket {} suddenly disappeared: {}", diff --git a/src/IO/S3/getObjectInfo.h b/src/IO/S3/getObjectInfo.h index ac8072a4338..32f34f74069 100644 --- a/src/IO/S3/getObjectInfo.h +++ b/src/IO/S3/getObjectInfo.h @@ -3,7 +3,7 @@ #include "config.h" #if USE_AWS_S3 -#include +#include #include #include @@ -24,7 +24,6 @@ ObjectInfo getObjectInfo( const String & bucket, const String & key, const String & version_id = {}, - const S3Settings::RequestSettings & request_settings = {}, bool with_metadata = false, bool throw_on_error = true); @@ -33,15 +32,13 @@ size_t getObjectSize( const String & bucket, const String & key, const String & version_id = {}, - const S3Settings::RequestSettings & request_settings = {}, bool throw_on_error = true); bool objectExists( const S3::Client & client, const String & bucket, const String & key, - const String & version_id = {}, - const S3Settings::RequestSettings & request_settings = {}); + const String & version_id = {}); /// Throws an exception if a specified object doesn't exist. `description` is used as a part of the error message. void checkObjectExists( @@ -49,7 +46,6 @@ void checkObjectExists( const String & bucket, const String & key, const String & version_id = {}, - const S3Settings::RequestSettings & request_settings = {}, std::string_view description = {}); bool isNotFoundError(Aws::S3::S3Errors error); diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 0a28c578f69..5ee9648a44e 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -25,7 +25,7 @@ #include #include #include -#include +#include #include #include "TestPocoHTTPServer.h" @@ -69,7 +69,7 @@ void doReadRequest(std::shared_ptr client, const DB::S3::U UInt64 max_single_read_retries = 1; DB::ReadSettings read_settings; - DB::S3Settings::RequestSettings request_settings; + DB::S3::RequestSettings request_settings; request_settings.max_single_read_retries = max_single_read_retries; DB::ReadBufferFromS3 read_buffer( client, @@ -88,7 +88,7 @@ void doWriteRequest(std::shared_ptr client, const DB::S3:: { UInt64 max_unexpected_write_error_retries = 1; - DB::S3Settings::RequestSettings request_settings; + DB::S3::RequestSettings request_settings; request_settings.max_unexpected_write_error_retries = max_unexpected_write_error_retries; DB::WriteBufferFromS3 write_buffer( client, diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 78c51fcb29c..8a01f6ca33a 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -8,11 +8,13 @@ #if USE_AWS_S3 -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include namespace ProfileEvents @@ -48,6 +50,7 @@ bool S3Exception::isRetryableError() const namespace DB::ErrorCodes { extern const int S3_ERROR; + extern const int INVALID_SETTING_VALUE; } #endif @@ -98,61 +101,90 @@ ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, c return sse_kms_config; } -AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config) +AuthSettings AuthSettings::loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings, + const std::string & setting_name_prefix) { - auto access_key_id = config.getString(config_elem + ".access_key_id", ""); - auto secret_access_key = config.getString(config_elem + ".secret_access_key", ""); - auto session_token = config.getString(config_elem + ".session_token", ""); + auto auth_settings = AuthSettings::loadFromSettings(settings); - auto region = config.getString(config_elem + ".region", ""); - auto server_side_encryption_customer_key_base64 = config.getString(config_elem + ".server_side_encryption_customer_key_base64", ""); + const std::string prefix = config_prefix + "." + setting_name_prefix; + auto has = [&](const std::string & key) -> bool { return config.has(prefix + key); }; + auto get_uint = [&](const std::string & key) -> size_t { return config.getUInt64(prefix + key); }; + auto get_bool = [&](const std::string & key) -> bool { return config.getBool(prefix + key); }; + auto get_string = [&](const std::string & key) -> std::string { return config.getString(prefix + key); }; - std::optional use_environment_credentials; - if (config.has(config_elem + ".use_environment_credentials")) - use_environment_credentials = config.getBool(config_elem + ".use_environment_credentials"); + if (has("access_key_id")) + auth_settings.access_key_id = get_string("access_key_id"); + if (has("secret_access_key")) + auth_settings.secret_access_key = get_string("secret_access_key"); + if (has("session_token")) + auth_settings.secret_access_key = get_string("session_token"); - std::optional use_insecure_imds_request; - if (config.has(config_elem + ".use_insecure_imds_request")) - use_insecure_imds_request = config.getBool(config_elem + ".use_insecure_imds_request"); + if (has("region")) + auth_settings.region = get_string("region"); + if (has("server_side_encryption_customer_key_base64")) + auth_settings.region = get_string("server_side_encryption_customer_key_base64"); - std::optional expiration_window_seconds; - if (config.has(config_elem + ".expiration_window_seconds")) - expiration_window_seconds = config.getUInt64(config_elem + ".expiration_window_seconds"); + if (has("connect_timeout_ms")) + auth_settings.connect_timeout_ms = get_uint("connect_timeout_ms"); + if (has("request_timeout_ms")) + auth_settings.request_timeout_ms = get_uint("request_timeout_ms"); + if (has("max_connections")) + auth_settings.max_connections = get_uint("max_connections"); - std::optional no_sign_request; - if (config.has(config_elem + ".no_sign_request")) - no_sign_request = config.getBool(config_elem + ".no_sign_request"); + if (has("http_keep_alive_timeout")) + auth_settings.http_keep_alive_timeout = get_uint("http_keep_alive_timeout"); + if (has("http_keep_alive_max_requests")) + auth_settings.http_keep_alive_max_requests = get_uint("http_keep_alive_max_requests"); - HTTPHeaderEntries headers = getHTTPHeaders(config_elem, config); - ServerSideEncryptionKMSConfig sse_kms_config = getSSEKMSConfig(config_elem, config); + if (has("use_environment_credentials")) + auth_settings.use_environment_credentials = get_bool("use_environment_credentials"); + if (has("use_adaptive_timeouts")) + auth_settings.use_adaptive_timeouts = get_bool("use_adaptive_timeouts"); + if (has("no_sing_request")) + auth_settings.no_sign_request = get_bool("no_sing_request"); + if (has("expiration_window_seconds")) + auth_settings.expiration_window_seconds = get_uint("expiration_window_seconds"); + if (has("gcs_issue_compose_request")) + auth_settings.gcs_issue_compose_request = get_bool("gcs_issue_compose_request"); + if (has("use_insecure_imds_request")) + auth_settings.use_insecure_imds_request = get_bool("use_insecure_imds_request"); + + auth_settings.headers = getHTTPHeaders(config_prefix, config); + auth_settings.server_side_encryption_kms_config = getSSEKMSConfig(config_prefix, config); - std::unordered_set users; Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_elem, keys); + config.keys(config_prefix, keys); for (const auto & key : keys) { if (startsWith(key, "user")) - users.insert(config.getString(config_elem + "." + key)); + auth_settings.users.insert(config.getString(config_prefix + "." + key)); } - return AuthSettings - { - std::move(access_key_id), std::move(secret_access_key), std::move(session_token), - std::move(region), - std::move(server_side_encryption_customer_key_base64), - std::move(sse_kms_config), - std::move(headers), - use_environment_credentials, - use_insecure_imds_request, - expiration_window_seconds, - no_sign_request, - std::move(users) - }; + return auth_settings; } -bool AuthSettings::canBeUsedByUser(const String & user) const +AuthSettings AuthSettings::loadFromSettings(const DB::Settings & settings) { - return users.empty() || users.contains(user); + AuthSettings auth_settings{}; + auth_settings.updateFromSettings(settings, /* if_changed */false); + return auth_settings; +} + +void AuthSettings::updateFromSettings(const DB::Settings & settings, bool if_changed) +{ + if (!if_changed || settings.s3_connect_timeout_ms.changed) + connect_timeout_ms = settings.s3_connect_timeout_ms; + if (!if_changed || settings.s3_request_timeout_ms.changed) + request_timeout_ms = settings.s3_request_timeout_ms; + if (!if_changed || settings.s3_max_connections.changed) + max_connections = settings.s3_max_connections; + if (!if_changed || settings.s3_use_adaptive_timeouts.changed) + use_adaptive_timeouts = settings.s3_use_adaptive_timeouts; + if (!if_changed || settings.s3_disable_checksum.changed) + disable_checksum = settings.s3_disable_checksum; } bool AuthSettings::hasUpdates(const AuthSettings & other) const @@ -183,7 +215,7 @@ void AuthSettings::updateFrom(const AuthSettings & from) server_side_encryption_kms_config = from.server_side_encryption_kms_config; if (from.use_environment_credentials.has_value()) - use_environment_credentials = from.use_environment_credentials; + use_environment_credentials = from.use_environment_credentials; if (from.use_insecure_imds_request.has_value()) use_insecure_imds_request = from.use_insecure_imds_request; @@ -197,5 +229,264 @@ void AuthSettings::updateFrom(const AuthSettings & from) users.insert(from.users.begin(), from.users.end()); } +RequestSettings RequestSettings::loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings, + bool validate_settings, + const std::string & setting_name_prefix) +{ + auto request_settings = RequestSettings::loadFromSettings(settings, validate_settings); + + String prefix = config_prefix + "." + setting_name_prefix; + auto has = [&](const std::string & key) -> bool { return config.has(prefix + key); }; + auto get_uint = [&](const std::string & key) -> size_t { return config.getUInt64(prefix + key); }; + auto get_string = [&](const std::string & key) -> std::string { return config.getString(prefix + key); }; + auto get_bool = [&](const std::string & key) -> bool { return config.getBool(prefix + key); }; + + if (has("strict_upload_part_size")) + request_settings.upload_settings.strict_upload_part_size = get_uint("strict_upload_part_size"); + if (has("min_upload_part_size")) + request_settings.upload_settings.min_upload_part_size = get_uint("min_upload_part_size"); + if (has("max_upload_part_size")) + request_settings.upload_settings.max_upload_part_size = get_uint("max_upload_part_size"); + if (has("upload_part_size_multiply_factor")) + request_settings.upload_settings.upload_part_size_multiply_factor = get_uint("upload_part_size_multiply_factor"); + if (has("upload_part_size_multiply_parts_count_threshold")) + request_settings.upload_settings.upload_part_size_multiply_parts_count_threshold = get_uint("upload_part_size_multiply_parts_count_threshold"); + if (has("max_inflight_parts_for_one_file")) + request_settings.upload_settings.max_inflight_parts_for_one_file = get_uint("max_inflight_parts_for_one_file"); + if (has("max_part_number")) + request_settings.upload_settings.max_part_number = get_uint("max_part_number"); + if (has("max_single_part_upload_size")) + request_settings.upload_settings.max_single_part_upload_size = get_uint("max_single_part_upload_size"); + if (has("max_single_operation_copy_size")) + request_settings.upload_settings.max_single_operation_copy_size = get_uint("max_single_operation_copy_size"); + if (has("s3_storage_class")) + request_settings.upload_settings.storage_class_name = get_string("s3_storage_class"); + + request_settings.upload_settings.storage_class_name = Poco::toUpperInPlace(request_settings.upload_settings.storage_class_name); + if (validate_settings) + request_settings.upload_settings.validate(); + + if (has("max_single_read_retries")) + request_settings.max_single_read_retries = get_uint("max_single_read_retries"); + if (has("check_objects_after_upload")) + request_settings.check_objects_after_upload = get_bool("check_objects_after_upload"); + if (has("list_object_keys_size")) + request_settings.list_object_keys_size = get_uint("list_object_keys_size"); + if (has("allow_native_copy")) + request_settings.allow_native_copy = get_bool("allow_native_copy"); + if (has("throw_on_zero_files_match")) + request_settings.throw_on_zero_files_match = get_bool("throw_on_zero_files_match"); + if (has("request_timeout_ms")) + request_settings.request_timeout_ms = get_uint("request_timeout_ms"); + + /// NOTE: it would be better to reuse old throttlers + /// to avoid losing token bucket state on every config reload, + /// which could lead to exceeding limit for short time. + /// But it is good enough unless very high `burst` values are used. + if (UInt64 max_get_rps = has("max_get_rps") ? get_uint("max_get_rps") : settings.s3_max_get_rps) + { + size_t default_max_get_burst = settings.s3_max_get_burst + ? settings.s3_max_get_burst + : (Throttler::default_burst_seconds * max_get_rps); + size_t max_get_burst = has("max_get_burst") ? get_uint("max_get_burst") : default_max_get_burst; + request_settings.get_request_throttler = std::make_shared(max_get_rps, max_get_burst); + } + if (UInt64 max_put_rps = has("max_put_rps") ? get_uint("max_put_rps") : settings.s3_max_put_rps) + { + size_t default_max_put_burst = settings.s3_max_put_burst + ? settings.s3_max_put_burst + : (Throttler::default_burst_seconds * max_put_rps); + size_t max_put_burst = has("max_put_burst") ? get_uint("max_put_burst") : default_max_put_burst; + request_settings.put_request_throttler = std::make_shared(max_put_rps, max_put_burst); + } + return request_settings; } + +RequestSettings RequestSettings::loadFromNamedCollection(const NamedCollection & collection, bool validate_settings) +{ + RequestSettings settings{}; + + if (collection.has("strict_upload_part_size")) + settings.upload_settings.strict_upload_part_size = collection.get("strict_upload_part_size"); + if (collection.has("min_upload_part_size")) + settings.upload_settings.min_upload_part_size = collection.get("min_upload_part_size"); + if (collection.has("max_upload_part_size")) + settings.upload_settings.min_upload_part_size = collection.get("max_upload_part_size"); + if (collection.has("upload_part_size_multiply_factor")) + settings.upload_settings.upload_part_size_multiply_factor = collection.get("upload_part_size_multiply_factor"); + if (collection.has("upload_part_size_multiply_parts_count_threshold")) + settings.upload_settings.upload_part_size_multiply_parts_count_threshold = collection.get("upload_part_size_multiply_parts_count_threshold"); + if (collection.has("max_inflight_parts_for_one_file")) + settings.upload_settings.max_inflight_parts_for_one_file = collection.get("max_inflight_parts_for_one_file"); + if (collection.has("max_part_number")) + settings.upload_settings.max_single_part_upload_size = collection.get("max_part_number"); + if (collection.has("max_single_part_upload_size")) + settings.upload_settings.max_single_part_upload_size = collection.get("max_single_part_upload_size"); + if (collection.has("max_single_operation_copy_size")) + settings.upload_settings.max_single_part_upload_size = collection.get("max_single_operation_copy_size"); + if (collection.has("s3_storage_class")) + settings.upload_settings.storage_class_name = collection.get("s3_storage_class"); + + settings.upload_settings.storage_class_name = Poco::toUpperInPlace(settings.upload_settings.storage_class_name); + if (validate_settings) + settings.upload_settings.validate(); + + if (collection.has("max_single_read_retries")) + settings.max_single_read_retries = collection.get("max_single_read_retries"); + if (collection.has("list_object_keys_size")) + settings.list_object_keys_size = collection.get("list_object_keys_size"); + if (collection.has("allow_native_copy")) + settings.allow_native_copy = collection.get("allow_native_copy"); + if (collection.has("throw_on_zero_files_match")) + settings.throw_on_zero_files_match = collection.get("throw_on_zero_files_match"); + + return settings; +} + +RequestSettings RequestSettings::loadFromSettings(const DB::Settings & settings, bool validate_settings) +{ + RequestSettings request_settings{}; + request_settings.updateFromSettings(settings, /* if_changed */false, validate_settings); + return request_settings; +} + +void RequestSettings::updateFromSettings(const DB::Settings & settings, bool if_changed, bool validate_settings) +{ + if (!if_changed || settings.s3_strict_upload_part_size.changed) + upload_settings.strict_upload_part_size = settings.s3_strict_upload_part_size; + if (!if_changed || settings.s3_min_upload_part_size.changed) + upload_settings.min_upload_part_size = settings.s3_min_upload_part_size; + if (!if_changed || settings.s3_max_upload_part_size.changed) + upload_settings.max_upload_part_size = settings.s3_max_upload_part_size; + if (!if_changed || settings.s3_upload_part_size_multiply_factor.changed) + upload_settings.upload_part_size_multiply_factor = settings.s3_upload_part_size_multiply_factor; + if (!if_changed || settings.s3_upload_part_size_multiply_parts_count_threshold.changed) + upload_settings.upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; + if (!if_changed || settings.s3_max_inflight_parts_for_one_file.changed) + upload_settings.max_inflight_parts_for_one_file = settings.s3_max_inflight_parts_for_one_file; + if (!if_changed || settings.s3_max_part_number.changed) + upload_settings.max_part_number = settings.s3_max_part_number; + if (!if_changed || settings.s3_max_single_part_upload_size.changed) + upload_settings.max_single_part_upload_size = settings.s3_max_single_part_upload_size; + if (!if_changed || settings.s3_max_single_operation_copy_size.changed) + upload_settings.max_part_number = settings.s3_max_single_operation_copy_size; + + if (validate_settings) + upload_settings.validate(); + + if (!if_changed || settings.s3_max_single_read_retries.changed) + max_single_read_retries = settings.s3_max_single_read_retries; + if (!if_changed || settings.s3_check_objects_after_upload.changed) + check_objects_after_upload = settings.s3_check_objects_after_upload; + if (!if_changed || settings.s3_max_unexpected_write_error_retries.changed) + max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; + if (!if_changed || settings.s3_list_object_keys_size.changed) + list_object_keys_size = settings.s3_list_object_keys_size; + if (!if_changed || settings.s3_throw_on_zero_files_match.changed) + throw_on_zero_files_match = settings.s3_throw_on_zero_files_match; + if (!if_changed || settings.s3_request_timeout_ms.changed) + request_timeout_ms = settings.s3_request_timeout_ms; + + if ((!if_changed || settings.s3_max_get_rps.changed || settings.s3_max_get_burst.changed) && settings.s3_max_get_rps) + { + size_t max_get_burst = settings.s3_max_get_burst + ? settings.s3_max_get_burst + : Throttler::default_burst_seconds * settings.s3_max_get_rps; + get_request_throttler = std::make_shared(settings.s3_max_get_rps, max_get_burst); + } + if ((!if_changed || settings.s3_max_put_rps.changed || settings.s3_max_put_burst.changed) && settings.s3_max_put_rps) + { + size_t max_put_burst = settings.s3_max_put_burst + ? settings.s3_max_put_burst + : Throttler::default_burst_seconds * settings.s3_max_put_rps; + put_request_throttler = std::make_shared(settings.s3_max_put_rps, max_put_burst); + } +} + +void RequestSettings::PartUploadSettings::validate() +{ + static constexpr size_t min_upload_part_size_limit = 5 * 1024 * 1024; + if (strict_upload_part_size && strict_upload_part_size < min_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting strict_upload_part_size has invalid value {} which is less than the s3 API limit {}", + ReadableSize(strict_upload_part_size), ReadableSize(min_upload_part_size_limit)); + + if (min_upload_part_size < min_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting min_upload_part_size has invalid value {} which is less than the s3 API limit {}", + ReadableSize(min_upload_part_size), ReadableSize(min_upload_part_size_limit)); + + static constexpr size_t max_upload_part_size_limit = 5ull * 1024 * 1024 * 1024; + if (max_upload_part_size > max_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_upload_part_size has invalid value {} which is greater than the s3 API limit {}", + ReadableSize(max_upload_part_size), ReadableSize(max_upload_part_size_limit)); + + if (max_single_part_upload_size > max_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_single_part_upload_size has invalid value {} which is grater than the s3 API limit {}", + ReadableSize(max_single_part_upload_size), ReadableSize(max_upload_part_size_limit)); + + if (max_single_operation_copy_size > max_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_single_operation_copy_size has invalid value {} which is grater than the s3 API limit {}", + ReadableSize(max_single_operation_copy_size), ReadableSize(max_upload_part_size_limit)); + + if (max_upload_part_size < min_upload_part_size) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_upload_part_size ({}) can't be less than setting min_upload_part_size {}", + ReadableSize(max_upload_part_size), ReadableSize(min_upload_part_size)); + + if (!upload_part_size_multiply_factor) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting upload_part_size_multiply_factor cannot be zero"); + + if (!upload_part_size_multiply_parts_count_threshold) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting upload_part_size_multiply_parts_count_threshold cannot be zero"); + + if (!max_part_number) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_part_number cannot be zero"); + + static constexpr size_t max_part_number_limit = 10000; + if (max_part_number > max_part_number_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_part_number has invalid value {} which is grater than the s3 API limit {}", + ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); + + size_t maybe_overflow; + if (common::mulOverflow(max_upload_part_size, upload_part_size_multiply_factor, maybe_overflow)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting upload_part_size_multiply_factor is too big ({}). " + "Multiplication to max_upload_part_size ({}) will cause integer overflow", + ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); + + std::unordered_set storage_class_names {"STANDARD", "INTELLIGENT_TIERING"}; + if (!storage_class_name.empty() && !storage_class_names.contains(storage_class_name)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting storage_class has invalid value {} which only supports STANDARD and INTELLIGENT_TIERING", + storage_class_name); + + /// TODO: it's possible to set too small limits. + /// We can check that max possible object size is not too small. +} + +} + } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index b3e01bd6132..01a7ca56779 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -4,10 +4,6 @@ #include #include -#include -#include -#include - #include "config.h" #if USE_AWS_S3 @@ -15,12 +11,15 @@ #include #include #include +#include +#include #include #include #include #include +#include namespace DB { @@ -31,6 +30,7 @@ namespace ErrorCodes } class RemoteHostFilter; +class NamedCollection; class S3Exception : public Exception { @@ -77,31 +77,93 @@ ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, c struct AuthSettings { - static AuthSettings loadFromConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); - std::string access_key_id; std::string secret_access_key; std::string session_token; std::string region; std::string server_side_encryption_customer_key_base64; - ServerSideEncryptionKMSConfig server_side_encryption_kms_config; HTTPHeaderEntries headers; + std::unordered_set users; + ServerSideEncryptionKMSConfig server_side_encryption_kms_config; + + std::optional connect_timeout_ms; + std::optional request_timeout_ms; + std::optional max_connections; + std::optional http_keep_alive_timeout; + std::optional http_keep_alive_max_requests; + std::optional expiration_window_seconds; std::optional use_environment_credentials; - std::optional use_insecure_imds_request; - std::optional expiration_window_seconds; std::optional no_sign_request; - - std::unordered_set users; + std::optional use_adaptive_timeouts; + std::optional use_insecure_imds_request; + std::optional is_virtual_hosted_style; + std::optional disable_checksum; + std::optional gcs_issue_compose_request; bool hasUpdates(const AuthSettings & other) const; void updateFrom(const AuthSettings & from); - bool canBeUsedByUser(const String & user) const; + bool canBeUsedByUser(const String & user) const { return users.empty() || users.contains(user); } + + static AuthSettings loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings, + const std::string & setting_name_prefix = ""); + + static AuthSettings loadFromSettings(const DB::Settings & settings); + + static AuthSettings loadFromNamedCollection(const NamedCollection & collection); + + void updateFromSettings(const DB::Settings & settings, bool if_changed); private: bool operator==(const AuthSettings & other) const = default; }; +struct RequestSettings +{ + size_t max_single_read_retries = 4; + size_t request_timeout_ms = 30000; + size_t max_unexpected_write_error_retries = 4; + size_t list_object_keys_size = 1000; + + bool allow_native_copy = true; + bool check_objects_after_upload = false; + bool throw_on_zero_files_match = false; + + struct PartUploadSettings + { + size_t strict_upload_part_size = 0; + size_t min_upload_part_size = 16 * 1024 * 1024; + size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; + size_t upload_part_size_multiply_factor = 2; + size_t upload_part_size_multiply_parts_count_threshold = 500; + size_t max_inflight_parts_for_one_file = 20; + size_t max_part_number = 10000; + size_t max_single_part_upload_size = 32 * 1024 * 1024; + size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; + String storage_class_name; + + void validate(); + }; + + PartUploadSettings upload_settings; + ThrottlerPtr get_request_throttler; + ThrottlerPtr put_request_throttler; + + static RequestSettings loadFromSettings(const DB::Settings & settings, bool validate_settings = true); + static RequestSettings loadFromNamedCollection(const NamedCollection & collection, bool validate_settings = true); + static RequestSettings loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings, + bool validate_settings = true, + const std::string & setting_name_prefix = ""); + + void updateFromSettings(const DB::Settings & settings, bool if_changed, bool validate_settings = true); +}; + } diff --git a/src/IO/S3Defines.h b/src/IO/S3Defines.h new file mode 100644 index 00000000000..ad01920adda --- /dev/null +++ b/src/IO/S3Defines.h @@ -0,0 +1,32 @@ +#pragma once +#include + +namespace DB::S3 +{ + +/// Client settings. +inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; +inline static constexpr uint64_t DEFAULT_CONNECT_TIMEOUT_MS = 1000; +inline static constexpr uint64_t DEFAULT_REQUEST_TIMEOUT_MS = 30000; +inline static constexpr uint64_t DEFAULT_MAX_CONNECTIONS = 1024; +inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_TIMEOUT = 5; +inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_MAX_REQUESTS = 100; + +inline static constexpr bool DEFAULT_USE_ENVIRONMENT_CREDENTIALS = true; +inline static constexpr bool DEFAULT_NO_SIGN_REQUEST = false; +inline static constexpr bool DEFAULT_DISABLE_CHECKSUM = false; +inline static constexpr bool DEFAULT_USE_ADAPTIVE_TIMEOUTS = true; + +/// Upload settings. +inline static constexpr uint64_t DEFAULT_MIN_UPLOAD_PART_SIZE = 16 * 1024 * 1024; +inline static constexpr uint64_t DEFAULT_MAX_UPLOAD_PART_SIZE = 5ull * 1024 * 1024 * 1024; +inline static constexpr uint64_t DEFAULT_MAX_SINGLE_PART_UPLOAD_SIZE = 32 * 1024 * 1024; +inline static constexpr uint64_t DEFAULT_UPLOAD_PART_SIZE_MULTIPLY_FACTOR = 2; +inline static constexpr uint64_t DEFAULT_UPLOAD_PART_SIZE_MULTIPLY_PARTS_COUNT_THRESHOLD = 500; +inline static constexpr uint64_t DEFAULT_MAX_PART_NUMBER = 10000; + +/// Other settings. +inline static constexpr uint64_t DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE = 32 * 1024 * 1024; +inline static constexpr uint64_t DEFAULT_MAX_INFLIGHT_PARTS_FOR_ONE_FILE = 20; + +} diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp new file mode 100644 index 00000000000..6a7b2ea5627 --- /dev/null +++ b/src/IO/S3Settings.cpp @@ -0,0 +1,62 @@ +#include + +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +void S3SettingsByEndpoint::loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings) +{ + std::lock_guard lock(mutex); + s3_settings.clear(); + if (!config.has(config_prefix)) + return; + + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(config_prefix, config_keys); + + for (const String & key : config_keys) + { + const auto key_path = config_prefix + "." + key; + const auto endpoint_path = key_path + ".endpoint"; + if (config.has(endpoint_path)) + { + auto endpoint = config.getString(endpoint_path); + auto auth_settings = S3::AuthSettings::loadFromConfig(config, key_path, settings); + auto request_settings = S3::RequestSettings::loadFromConfig(config, key_path, settings); + s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); + } + } +} + +std::optional S3SettingsByEndpoint::getSettings( + const String & endpoint, + const String & user, + bool ignore_user) const +{ + std::lock_guard lock(mutex); + auto next_prefix_setting = s3_settings.upper_bound(endpoint); + + /// Linear time algorithm may be replaced with logarithmic with prefix tree map. + for (auto possible_prefix_setting = next_prefix_setting; possible_prefix_setting != s3_settings.begin();) + { + std::advance(possible_prefix_setting, -1); + const auto & [endpoint_prefix, settings] = *possible_prefix_setting; + if (endpoint.starts_with(endpoint_prefix) && (ignore_user || settings.auth_settings.canBeUsedByUser(user))) + return possible_prefix_setting->second; + } + + return {}; +} + +} diff --git a/src/IO/S3Settings.h b/src/IO/S3Settings.h new file mode 100644 index 00000000000..58e12e48002 --- /dev/null +++ b/src/IO/S3Settings.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace Poco::Util { class AbstractConfiguration; } + +namespace DB +{ + +struct Settings; + +struct S3Settings +{ + S3::AuthSettings auth_settings; + S3::RequestSettings request_settings; +}; + +class S3SettingsByEndpoint +{ +public: + void loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings); + + std::optional getSettings( + const std::string & endpoint, + const std::string & user, + bool ignore_user = false) const; + +private: + mutable std::mutex mutex; + std::map s3_settings; +}; + + +} diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index ff18a77f09f..b83ca174820 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -72,7 +72,7 @@ struct WriteBufferFromS3::PartData } }; -BufferAllocationPolicyPtr createBufferAllocationPolicy(const S3Settings::RequestSettings::PartUploadSettings & settings) +BufferAllocationPolicyPtr createBufferAllocationPolicy(const S3::RequestSettings::PartUploadSettings & settings) { BufferAllocationPolicy::Settings allocation_settings; allocation_settings.strict_size = settings.strict_upload_part_size; @@ -91,7 +91,7 @@ WriteBufferFromS3::WriteBufferFromS3( const String & bucket_, const String & key_, size_t buf_size_, - const S3Settings::RequestSettings & request_settings_, + const S3::RequestSettings & request_settings_, BlobStorageLogWriterPtr blob_log_, std::optional> object_metadata_, ThreadPoolCallbackRunnerUnsafe schedule_, @@ -100,7 +100,7 @@ WriteBufferFromS3::WriteBufferFromS3( , bucket(bucket_) , key(key_) , request_settings(request_settings_) - , upload_settings(request_settings.getUploadSettings()) + , upload_settings(request_settings.upload_settings) , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) @@ -214,9 +214,9 @@ void WriteBufferFromS3::finalizeImpl() if (request_settings.check_objects_after_upload) { - S3::checkObjectExists(*client_ptr, bucket, key, {}, request_settings, "Immediately after upload"); + S3::checkObjectExists(*client_ptr, bucket, key, {}, "Immediately after upload"); - size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}, request_settings); + size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}); if (actual_size != total_size) throw Exception( ErrorCodes::S3_ERROR, diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index fbfec3588fa..d4e25ea1733 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -38,7 +38,7 @@ public: const String & bucket_, const String & key_, size_t buf_size_, - const S3Settings::RequestSettings & request_settings_, + const S3::RequestSettings & request_settings_, BlobStorageLogWriterPtr blob_log_, std::optional> object_metadata_ = std::nullopt, ThreadPoolCallbackRunnerUnsafe schedule_ = {}, @@ -78,8 +78,8 @@ private: const String bucket; const String key; - const S3Settings::RequestSettings request_settings; - const S3Settings::RequestSettings::PartUploadSettings & upload_settings; + const S3::RequestSettings request_settings; + const S3::RequestSettings::PartUploadSettings & upload_settings; const WriteSettings write_settings; const std::shared_ptr client_ptr; const std::optional> object_metadata; diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 4a4d7cc0fc2..1e1fdc02060 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -546,8 +546,8 @@ public: std::unique_ptr getWriteBuffer(String file_name = "file") { - S3Settings::RequestSettings request_settings; - request_settings.updateFromSettingsIfChanged(settings); + S3::RequestSettings request_settings; + request_settings.updateFromSettings(settings, /* if_changed */true); client->resetCounters(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e1d82a8f604..738f4e9280e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -31,7 +31,7 @@ #include #include #include -#include +#include #include #include #include @@ -370,7 +370,7 @@ struct ContextSharedPart : boost::noncopyable ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers OnceFlag system_logs_initialized; std::unique_ptr system_logs TSA_GUARDED_BY(mutex); /// Used to log queries and operations on parts - std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage + std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage std::vector warnings TSA_GUARDED_BY(mutex); /// Store warning messages about server configuration. /// Background executors for *MergeTree tables @@ -4264,7 +4264,7 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration { std::lock_guard lock(shared->mutex); if (shared->storage_s3_settings) - shared->storage_s3_settings->loadFromConfig("s3", config, getSettingsRef()); + shared->storage_s3_settings->loadFromConfig(config, /* config_prefix */"s3", getSettingsRef()); } } @@ -4316,14 +4316,14 @@ const DistributedSettings & Context::getDistributedSettings() const return *shared->distributed_settings; } -const StorageS3Settings & Context::getStorageS3Settings() const +const S3SettingsByEndpoint & Context::getStorageS3Settings() const { std::lock_guard lock(shared->mutex); if (!shared->storage_s3_settings) { const auto & config = shared->getConfigRefWithLock(lock); - shared->storage_s3_settings.emplace().loadFromConfig("s3", config, getSettingsRef()); + shared->storage_s3_settings.emplace().loadFromConfig(config, "s3", getSettingsRef()); } return *shared->storage_s3_settings; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 814534f7035..0de24883e42 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -117,7 +117,7 @@ struct DistributedSettings; struct InitialAllRangesAnnouncement; struct ParallelReadRequest; struct ParallelReadResponse; -class StorageS3Settings; +class S3SettingsByEndpoint; class IDatabase; class DDLWorker; class ITableFunction; @@ -1107,7 +1107,7 @@ public: const MergeTreeSettings & getMergeTreeSettings() const; const MergeTreeSettings & getReplicatedMergeTreeSettings() const; const DistributedSettings & getDistributedSettings() const; - const StorageS3Settings & getStorageS3Settings() const; + const S3SettingsByEndpoint & getStorageS3Settings() const; /// Prevents DROP TABLE if its size is greater than max_size (50GB by default, max_size=0 turn off this check) void setMaxTableSizeToDrop(size_t max_size); diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index d4e737a7de1..c703c9ce999 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 6b6cde0c431..45d54b62cbe 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -108,9 +108,9 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, const auto & settings = context->getSettingsRef(); const std::string config_prefix = "s3."; - auto s3_settings = getSettings(config, config_prefix, context, settings.s3_validate_request_settings); + auto s3_settings = getSettings(config, config_prefix, context, /* for_disk_s3 */false, settings.s3_validate_request_settings); - request_settings.updateFromSettingsIfChanged(settings); + request_settings.updateFromSettings(settings, /* if_changed */true); auth_settings.updateFrom(s3_settings->auth_settings); s3_settings->auth_settings = auth_settings; @@ -126,7 +126,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); - auto client = getClient(config, config_prefix, context, *s3_settings, false, &url); + auto client = getClient(url, *s3_settings, context, /* for_disk_s3 */false); auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); auto s3_capabilities = S3Capabilities { @@ -159,7 +159,7 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); structure = collection.getOrDefault("structure", "auto"); - request_settings = S3Settings::RequestSettings(collection); + request_settings = S3::RequestSettings::loadFromNamedCollection(collection, /* validate_settings */true); static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 906d10a1a9a..5a952497851 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -3,7 +3,7 @@ #include "config.h" #if USE_AWS_S3 -#include +#include #include namespace DB @@ -58,7 +58,7 @@ private: std::vector keys; S3::AuthSettings auth_settings; - S3Settings::RequestSettings request_settings; + S3::RequestSettings request_settings; HTTPHeaderEntries headers_from_ast; /// Headers from ast is a part of static configuration. /// If s3 configuration was passed from ast, then it is static. /// If from config - it can be changed with config reload. diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index e1583b8329c..378fa941d09 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp deleted file mode 100644 index b767805f637..00000000000 --- a/src/Storages/StorageS3Settings.cpp +++ /dev/null @@ -1,315 +0,0 @@ -#include - -#include - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INVALID_SETTING_VALUE; -} - -S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const Settings & settings, bool validate_settings) -{ - updateFromSettings(settings, false); - if (validate_settings) - validate(); -} - -S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Settings & settings, - String setting_name_prefix, - bool validate_settings) - : PartUploadSettings(settings, validate_settings) -{ - String key = config_prefix + "." + setting_name_prefix; - strict_upload_part_size = config.getUInt64(key + "strict_upload_part_size", strict_upload_part_size); - min_upload_part_size = config.getUInt64(key + "min_upload_part_size", min_upload_part_size); - max_upload_part_size = config.getUInt64(key + "max_upload_part_size", max_upload_part_size); - upload_part_size_multiply_factor = config.getUInt64(key + "upload_part_size_multiply_factor", upload_part_size_multiply_factor); - upload_part_size_multiply_parts_count_threshold = config.getUInt64(key + "upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); - max_inflight_parts_for_one_file = config.getUInt64(key + "max_inflight_parts_for_one_file", max_inflight_parts_for_one_file); - max_part_number = config.getUInt64(key + "max_part_number", max_part_number); - max_single_part_upload_size = config.getUInt64(key + "max_single_part_upload_size", max_single_part_upload_size); - max_single_operation_copy_size = config.getUInt64(key + "max_single_operation_copy_size", max_single_operation_copy_size); - - /// This configuration is only applicable to s3. Other types of object storage are not applicable or have different meanings. - storage_class_name = config.getString(config_prefix + ".s3_storage_class", storage_class_name); - storage_class_name = Poco::toUpperInPlace(storage_class_name); - - if (validate_settings) - validate(); -} - -S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedCollection & collection) -{ - strict_upload_part_size = collection.getOrDefault("strict_upload_part_size", strict_upload_part_size); - min_upload_part_size = collection.getOrDefault("min_upload_part_size", min_upload_part_size); - max_single_part_upload_size = collection.getOrDefault("max_single_part_upload_size", max_single_part_upload_size); - upload_part_size_multiply_factor = collection.getOrDefault("upload_part_size_multiply_factor", upload_part_size_multiply_factor); - upload_part_size_multiply_parts_count_threshold = collection.getOrDefault("upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); - max_inflight_parts_for_one_file = collection.getOrDefault("max_inflight_parts_for_one_file", max_inflight_parts_for_one_file); - - /// This configuration is only applicable to s3. Other types of object storage are not applicable or have different meanings. - storage_class_name = collection.getOrDefault("s3_storage_class", storage_class_name); - storage_class_name = Poco::toUpperInPlace(storage_class_name); - - validate(); -} - -void S3Settings::RequestSettings::PartUploadSettings::updateFromSettings(const Settings & settings, bool if_changed) -{ - if (!if_changed || settings.s3_strict_upload_part_size.changed) - strict_upload_part_size = settings.s3_strict_upload_part_size; - - if (!if_changed || settings.s3_min_upload_part_size.changed) - min_upload_part_size = settings.s3_min_upload_part_size; - - if (!if_changed || settings.s3_max_upload_part_size.changed) - max_upload_part_size = settings.s3_max_upload_part_size; - - if (!if_changed || settings.s3_upload_part_size_multiply_factor.changed) - upload_part_size_multiply_factor = settings.s3_upload_part_size_multiply_factor; - - if (!if_changed || settings.s3_upload_part_size_multiply_parts_count_threshold.changed) - upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; - - if (!if_changed || settings.s3_max_inflight_parts_for_one_file.changed) - max_inflight_parts_for_one_file = settings.s3_max_inflight_parts_for_one_file; - - if (!if_changed || settings.s3_max_single_part_upload_size.changed) - max_single_part_upload_size = settings.s3_max_single_part_upload_size; -} - -void S3Settings::RequestSettings::PartUploadSettings::validate() -{ - static constexpr size_t min_upload_part_size_limit = 5 * 1024 * 1024; - if (strict_upload_part_size && strict_upload_part_size < min_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting strict_upload_part_size has invalid value {} which is less than the s3 API limit {}", - ReadableSize(strict_upload_part_size), ReadableSize(min_upload_part_size_limit)); - - if (min_upload_part_size < min_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting min_upload_part_size has invalid value {} which is less than the s3 API limit {}", - ReadableSize(min_upload_part_size), ReadableSize(min_upload_part_size_limit)); - - static constexpr size_t max_upload_part_size_limit = 5ull * 1024 * 1024 * 1024; - if (max_upload_part_size > max_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_upload_part_size has invalid value {} which is greater than the s3 API limit {}", - ReadableSize(max_upload_part_size), ReadableSize(max_upload_part_size_limit)); - - if (max_single_part_upload_size > max_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_single_part_upload_size has invalid value {} which is grater than the s3 API limit {}", - ReadableSize(max_single_part_upload_size), ReadableSize(max_upload_part_size_limit)); - - if (max_single_operation_copy_size > max_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_single_operation_copy_size has invalid value {} which is grater than the s3 API limit {}", - ReadableSize(max_single_operation_copy_size), ReadableSize(max_upload_part_size_limit)); - - if (max_upload_part_size < min_upload_part_size) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_upload_part_size ({}) can't be less than setting min_upload_part_size {}", - ReadableSize(max_upload_part_size), ReadableSize(min_upload_part_size)); - - if (!upload_part_size_multiply_factor) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting upload_part_size_multiply_factor cannot be zero"); - - if (!upload_part_size_multiply_parts_count_threshold) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting upload_part_size_multiply_parts_count_threshold cannot be zero"); - - if (!max_part_number) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_part_number cannot be zero"); - - static constexpr size_t max_part_number_limit = 10000; - if (max_part_number > max_part_number_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_part_number has invalid value {} which is grater than the s3 API limit {}", - ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); - - size_t maybe_overflow; - if (common::mulOverflow(max_upload_part_size, upload_part_size_multiply_factor, maybe_overflow)) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting upload_part_size_multiply_factor is too big ({}). " - "Multiplication to max_upload_part_size ({}) will cause integer overflow", - ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); - - std::unordered_set storage_class_names {"STANDARD", "INTELLIGENT_TIERING"}; - if (!storage_class_name.empty() && !storage_class_names.contains(storage_class_name)) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting storage_class has invalid value {} which only supports STANDARD and INTELLIGENT_TIERING", - storage_class_name); - - /// TODO: it's possible to set too small limits. We can check that max possible object size is not too small. -} - - -S3Settings::RequestSettings::RequestSettings(const Settings & settings, bool validate_settings) - : upload_settings(settings, validate_settings) -{ - updateFromSettingsImpl(settings, false); -} - -S3Settings::RequestSettings::RequestSettings(const NamedCollection & collection) - : upload_settings(collection) -{ - max_single_read_retries = collection.getOrDefault("max_single_read_retries", max_single_read_retries); - max_connections = collection.getOrDefault("max_connections", max_connections); - list_object_keys_size = collection.getOrDefault("list_object_keys_size", list_object_keys_size); - allow_native_copy = collection.getOrDefault("allow_native_copy", allow_native_copy); - throw_on_zero_files_match = collection.getOrDefault("throw_on_zero_files_match", throw_on_zero_files_match); -} - -S3Settings::RequestSettings::RequestSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Settings & settings, - String setting_name_prefix, - bool validate_settings) - : upload_settings(config, config_prefix, settings, setting_name_prefix, validate_settings) -{ - String key = config_prefix + "." + setting_name_prefix; - max_single_read_retries = config.getUInt64(key + "max_single_read_retries", settings.s3_max_single_read_retries); - max_connections = config.getUInt64(key + "max_connections", settings.s3_max_connections); - check_objects_after_upload = config.getBool(key + "check_objects_after_upload", settings.s3_check_objects_after_upload); - list_object_keys_size = config.getUInt64(key + "list_object_keys_size", settings.s3_list_object_keys_size); - allow_native_copy = config.getBool(key + "allow_native_copy", allow_native_copy); - throw_on_zero_files_match = config.getBool(key + "throw_on_zero_files_match", settings.s3_throw_on_zero_files_match); - retry_attempts = config.getUInt64(key + "retry_attempts", settings.s3_retry_attempts); - request_timeout_ms = config.getUInt64(key + "request_timeout_ms", settings.s3_request_timeout_ms); - - /// NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, - /// which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = config.getUInt64(key + "max_get_rps", settings.s3_max_get_rps)) - { - size_t default_max_get_burst = settings.s3_max_get_burst - ? settings.s3_max_get_burst - : (Throttler::default_burst_seconds * max_get_rps); - - size_t max_get_burst = config.getUInt64(key + "max_get_burst", default_max_get_burst); - - get_request_throttler = std::make_shared(max_get_rps, max_get_burst); - } - if (UInt64 max_put_rps = config.getUInt64(key + "max_put_rps", settings.s3_max_put_rps)) - { - size_t default_max_put_burst = settings.s3_max_put_burst - ? settings.s3_max_put_burst - : (Throttler::default_burst_seconds * max_put_rps); - - size_t max_put_burst = config.getUInt64(key + "max_put_burst", default_max_put_burst); - - put_request_throttler = std::make_shared(max_put_rps, max_put_burst); - } -} - -void S3Settings::RequestSettings::updateFromSettingsImpl(const Settings & settings, bool if_changed) -{ - if (!if_changed || settings.s3_max_single_read_retries.changed) - max_single_read_retries = settings.s3_max_single_read_retries; - - if (!if_changed || settings.s3_max_connections.changed) - max_connections = settings.s3_max_connections; - - if (!if_changed || settings.s3_check_objects_after_upload.changed) - check_objects_after_upload = settings.s3_check_objects_after_upload; - - if (!if_changed || settings.s3_max_unexpected_write_error_retries.changed) - max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; - - if (!if_changed || settings.s3_list_object_keys_size.changed) - list_object_keys_size = settings.s3_list_object_keys_size; - - if ((!if_changed || settings.s3_max_get_rps.changed || settings.s3_max_get_burst.changed) && settings.s3_max_get_rps) - get_request_throttler = std::make_shared( - settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); - - if ((!if_changed || settings.s3_max_put_rps.changed || settings.s3_max_put_burst.changed) && settings.s3_max_put_rps) - put_request_throttler = std::make_shared( - settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); - - if (!if_changed || settings.s3_throw_on_zero_files_match.changed) - throw_on_zero_files_match = settings.s3_throw_on_zero_files_match; - - if (!if_changed || settings.s3_retry_attempts.changed) - retry_attempts = settings.s3_retry_attempts; - - if (!if_changed || settings.s3_request_timeout_ms.changed) - request_timeout_ms = settings.s3_request_timeout_ms; -} - -void S3Settings::RequestSettings::updateFromSettingsIfChanged(const Settings & settings) -{ - updateFromSettingsImpl(settings, true); - upload_settings.updateFromSettings(settings, true); -} - -void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings) -{ - std::lock_guard lock(mutex); - s3_settings.clear(); - if (!config.has(config_elem)) - return; - - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(config_elem, config_keys); - - for (const String & key : config_keys) - { - if (config.has(config_elem + "." + key + ".endpoint")) - { - auto endpoint = config.getString(config_elem + "." + key + ".endpoint"); - auto auth_settings = S3::AuthSettings::loadFromConfig(config_elem + "." + key, config); - S3Settings::RequestSettings request_settings(config, config_elem + "." + key, settings); - - s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); - } - } -} - -std::optional StorageS3Settings::getSettings(const String & endpoint, const String & user, bool ignore_user) const -{ - std::lock_guard lock(mutex); - auto next_prefix_setting = s3_settings.upper_bound(endpoint); - - /// Linear time algorithm may be replaced with logarithmic with prefix tree map. - for (auto possible_prefix_setting = next_prefix_setting; possible_prefix_setting != s3_settings.begin();) - { - std::advance(possible_prefix_setting, -1); - const auto & [endpoint_prefix, settings] = *possible_prefix_setting; - if (endpoint.starts_with(endpoint_prefix) && (ignore_user || settings.auth_settings.canBeUsedByUser(user))) - return possible_prefix_setting->second; - } - - return {}; -} - -} diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h deleted file mode 100644 index c3bc8aa6ed6..00000000000 --- a/src/Storages/StorageS3Settings.h +++ /dev/null @@ -1,122 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace Poco::Util -{ -class AbstractConfiguration; -} - -namespace DB -{ - -struct Settings; -class NamedCollection; - -struct S3Settings -{ - struct RequestSettings - { - struct PartUploadSettings - { - size_t strict_upload_part_size = 0; - size_t min_upload_part_size = 16 * 1024 * 1024; - size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; - size_t upload_part_size_multiply_factor = 2; - size_t upload_part_size_multiply_parts_count_threshold = 500; - size_t max_inflight_parts_for_one_file = 20; - size_t max_part_number = 10000; - size_t max_single_part_upload_size = 32 * 1024 * 1024; - size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; - String storage_class_name; - - void updateFromSettings(const Settings & settings, bool if_changed); - void validate(); - - private: - PartUploadSettings() = default; - explicit PartUploadSettings(const Settings & settings, bool validate_settings = true); - explicit PartUploadSettings(const NamedCollection & collection); - PartUploadSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Settings & settings, - String setting_name_prefix = {}, - bool validate_settings = true); - - friend struct RequestSettings; - }; - - private: - PartUploadSettings upload_settings = {}; - - public: - size_t max_single_read_retries = 4; - size_t max_connections = 1024; - bool check_objects_after_upload = false; - size_t max_unexpected_write_error_retries = 4; - size_t list_object_keys_size = 1000; - ThrottlerPtr get_request_throttler; - ThrottlerPtr put_request_throttler; - size_t retry_attempts = 10; - size_t request_timeout_ms = 30000; - bool allow_native_copy = true; - - bool throw_on_zero_files_match = false; - - const PartUploadSettings & getUploadSettings() const { return upload_settings; } - PartUploadSettings & getUploadSettings() { return upload_settings; } - - void setStorageClassName(const String & storage_class_name) { upload_settings.storage_class_name = storage_class_name; } - - RequestSettings() = default; - explicit RequestSettings(const Settings & settings, bool validate_settings = true); - explicit RequestSettings(const NamedCollection & collection); - - /// What's the setting_name_prefix, and why do we need it? - /// There are (at least) two config sections where s3 settings can be specified: - /// * settings for s3 disk (clickhouse/storage_configuration/disks) - /// * settings for s3 storage (clickhouse/s3), which are also used for backups - /// Even though settings are the same, in case of s3 disk they are prefixed with "s3_" - /// ("s3_max_single_part_upload_size"), but in case of s3 storage they are not - /// ( "max_single_part_upload_size"). Why this happened is a complete mystery to me. - RequestSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Settings & settings, - String setting_name_prefix = {}, - bool validate_settings = true); - - void updateFromSettingsIfChanged(const Settings & settings); - - private: - void updateFromSettingsImpl(const Settings & settings, bool if_changed); - }; - - S3::AuthSettings auth_settings; - RequestSettings request_settings; -}; - -/// Settings for the StorageS3. -class StorageS3Settings -{ -public: - void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings); - - std::optional getSettings(const String & endpoint, const String & user, bool ignore_user = false) const; - -private: - mutable std::mutex mutex; - std::map s3_settings; -}; - -} From 804b1ad1e02825826f593f80d589ea9015b6a5d6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 May 2024 10:34:16 +0000 Subject: [PATCH 143/856] Better --- src/CMakeLists.txt | 2 +- src/Common/Allocator.cpp | 1 - src/Common/GWPAsan.cpp | 196 ++++++++++++++++++++++++++++++++++++++ src/Common/GWPAsan.h | 20 ++++ src/Common/memory.cpp | 23 ----- src/Common/memory.h | 11 +-- src/Daemon/BaseDaemon.cpp | 19 ++-- 7 files changed, 224 insertions(+), 48 deletions(-) create mode 100644 src/Common/GWPAsan.cpp create mode 100644 src/Common/GWPAsan.h delete mode 100644 src/Common/memory.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7ae20b21889..7b887391df0 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -183,7 +183,7 @@ add_library (clickhouse_new_delete STATIC Common/new_delete.cpp) target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io) if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::jemalloc) - target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::jemalloc) + target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::jemalloc) target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc) endif() diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 8b68ef87298..67ef98cf221 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -66,7 +66,6 @@ void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_) template void * allocNoTrack(size_t size, size_t alignment) { - void * buf; #if USE_GWP_ASAN if (unlikely(Memory::GuardedAlloc.shouldSample())) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp new file mode 100644 index 00000000000..6f57af9e982 --- /dev/null +++ b/src/Common/GWPAsan.cpp @@ -0,0 +1,196 @@ +#include +#include + +#if USE_GWP_ASAN +# include +# include +# include +# include +# include +# include +# include + +namespace Memory +{ + +namespace +{ + size_t getBackTrace(uintptr_t * trace_buffer, size_t buffer_size) + { + StackTrace stacktrace; + auto trace_size = std::min(buffer_size, stacktrace.getSize()); + const auto & frame_pointers = stacktrace.getFramePointers(); + memcpy(trace_buffer, frame_pointers.data(), std::min(trace_size, buffer_size) * sizeof(uintptr_t)); + return trace_size; + } +} + +gwp_asan::GuardedPoolAllocator GuardedAlloc; +static bool guarded_alloc_initialized = [] +{ + gwp_asan::options::initOptions(); + gwp_asan::options::Options &opts = gwp_asan::options::getOptions(); + opts.MaxSimultaneousAllocations = 1024; + opts.Backtrace = getBackTrace; + GuardedAlloc.init(opts); + + ///std::cerr << "GwpAsan is initialized, the options are { Enabled: " << opts.Enabled + /// << ", MaxSimultaneousAllocations: " << opts.MaxSimultaneousAllocations + /// << ", SampleRate: " << opts.SampleRate << " }\n"; + + return true; +}(); + +bool isGWPAsanError(uintptr_t fault_address) +{ + const auto * state = GuardedAlloc.getAllocatorState(); + if (state->FailureType != gwp_asan::Error::UNKNOWN && state->FailureAddress != 0) + return true; + + return fault_address < state->GuardedPagePoolEnd && state->GuardedPagePool <= fault_address; +} + +namespace +{ + +struct ScopedEndOfReportDecorator { + explicit ScopedEndOfReportDecorator(Poco::LoggerPtr log_) : log(std::move(log_)) {} + ~ScopedEndOfReportDecorator() { LOG_FATAL(log, "*** End GWP-ASan report ***"); } + Poco::LoggerPtr log; +}; + +constexpr std::string_view unknown_crash_text = + "GWP-ASan cannot provide any more information about this error. This may " + "occur due to a wild memory access into the GWP-ASan pool, or an " + "overflow/underflow that is > 512B in length.\n"; + + +// Prints the provided error and metadata information. +void printHeader(gwp_asan::Error error, uintptr_t fault_address, const gwp_asan::AllocationMetadata * allocation_meta, Poco::LoggerPtr log) +{ + bool access_was_in_bounds = false; + std::string description; + if (error != gwp_asan::Error::UNKNOWN && allocation_meta != nullptr) + { + uintptr_t address = __gwp_asan_get_allocation_address(allocation_meta); + size_t size = __gwp_asan_get_allocation_size(allocation_meta); + if (fault_address < address) + { + description = fmt::format( + "({} byte{} to the left of a {}-byte allocation at 0x{}) ", + address - fault_address, + (address - fault_address == 1) ? "" : "s", + size, + address); + } + else if (fault_address > address) + { + description = fmt::format( + "({} byte{} to the right of a {}-byte allocation at 0x{}) ", + fault_address - address, + (fault_address - address == 1) ? "" : "s", + size, + address); + } + else if (error == gwp_asan::Error::DOUBLE_FREE) + { + description = fmt::format("(a {}-byte allocation) ", size); + } + else + { + access_was_in_bounds = true; + description = fmt::format( + "({} byte{} into a {}-byte allocation at 0x{}) ", + fault_address - address, + (fault_address - address == 1) ? "" : "s", + size, + address); + } + } + + // Possible number of digits of a 64-bit number: ceil(log10(2^64)) == 20. Add + // a null terminator, and round to the nearest 8-byte boundary. + uint64_t thread_id = gwp_asan::getThreadID(); + std::string thread_id_string = thread_id == gwp_asan::kInvalidThreadID ? " trace; + + // Maybe print the deallocation trace. + if (__gwp_asan_is_deallocated(allocation_meta)) + { + uint64_t thread_id = __gwp_asan_get_deallocation_thread_id(allocation_meta); + if (thread_id == gwp_asan::kInvalidThreadID) + LOG_FATAL(logger, "0x{} was deallocated by thread here:", fault_address); + else + LOG_FATAL(logger, "0x{} was deallocated by thread {} here:", fault_address, thread_id); + const auto trace_length = __gwp_asan_get_deallocation_trace(allocation_meta, trace.data(), maximum_stack_frames); + StackTrace::toStringEveryLine( + reinterpret_cast(trace.data()), 0, trace_length, [&](const auto line) { LOG_FATAL(logger, fmt::runtime(line)); }); + } + + // Print the allocation trace. + uint64_t thread_id = __gwp_asan_get_allocation_thread_id(allocation_meta); + if (thread_id == gwp_asan::kInvalidThreadID) + LOG_FATAL(logger, "0x{} was allocated by thread here:", fault_address); + else + LOG_FATAL(logger, "0x{} was allocated by thread {} here:", fault_address, thread_id); + const auto trace_length = __gwp_asan_get_allocation_trace(allocation_meta, trace.data(), maximum_stack_frames); + StackTrace::toStringEveryLine( + reinterpret_cast(trace.data()), 0, trace_length, [&](const auto line) { LOG_FATAL(logger, fmt::runtime(line)); }); +} + +} +#endif diff --git a/src/Common/GWPAsan.h b/src/Common/GWPAsan.h new file mode 100644 index 00000000000..164c6ee0221 --- /dev/null +++ b/src/Common/GWPAsan.h @@ -0,0 +1,20 @@ +#pragma once + +#include "config.h" + +#if USE_GWP_ASAN + +#include + +namespace Memory +{ + +extern gwp_asan::GuardedPoolAllocator GuardedAlloc; + +bool isGWPAsanError(uintptr_t fault_address); + +void printGWPAsanReport(uintptr_t fault_address); + +} + +#endif diff --git a/src/Common/memory.cpp b/src/Common/memory.cpp deleted file mode 100644 index 6c17dbe3ba1..00000000000 --- a/src/Common/memory.cpp +++ /dev/null @@ -1,23 +0,0 @@ -#include - -#if USE_GWP_ASAN -#include - -namespace Memory -{ -gwp_asan::GuardedPoolAllocator GuardedAlloc; -static bool guarded_alloc_initialized = [] -{ - gwp_asan::options::initOptions(); - gwp_asan::options::Options &opts = gwp_asan::options::getOptions(); - opts.MaxSimultaneousAllocations = 256; - GuardedAlloc.init(opts); - - ///std::cerr << "GwpAsan is initialized, the options are { Enabled: " << opts.Enabled - /// << ", MaxSimultaneousAllocations: " << opts.MaxSimultaneousAllocations - /// << ", SampleRate: " << opts.SampleRate << " }\n"; - - return true; -}(); -} -#endif diff --git a/src/Common/memory.h b/src/Common/memory.h index 427120edc75..633994a83e2 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -6,6 +6,7 @@ #include #include #include +#include #include "config.h" #if USE_JEMALLOC @@ -16,12 +17,6 @@ # include #endif -#if USE_GWP_ASAN -# include -# include - -#endif - namespace ProfileEvents { extern const Event GWPAsanAllocateSuccess; @@ -32,10 +27,6 @@ namespace ProfileEvents namespace Memory { -#if USE_GWP_ASAN -extern gwp_asan::GuardedPoolAllocator GuardedAlloc; -#endif - inline ALWAYS_INLINE size_t alignToSizeT(std::align_val_t align) noexcept { return static_cast(align); diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 74c37b6123b..3f7ad8d7126 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include @@ -157,18 +157,11 @@ static void signalHandler(int sig, siginfo_t * info, void * context) const ucontext_t * signal_context = reinterpret_cast(context); const StackTrace stack_trace(*signal_context); - const auto is_gwp_asan = [&] - { - auto state = ::Memory::GuardedAlloc.getAllocatorState(); - if (state->FailureType != gwp_asan::Error::UNKNOWN && state->FailureAddress != 0) - return true; - - auto addr = reinterpret_cast(info->si_addr); - return addr < state->GuardedPagePoolEnd && state->GuardedPagePool <= addr; - }; - - if (is_gwp_asan()) - std::cerr << "GWPAsan caught something!" << std::endl; +#if USE_GWP_ASAN + if (const auto fault_address = reinterpret_cast(info->si_addr); + ::Memory::isGWPAsanError(fault_address)) + ::Memory::printGWPAsanReport(fault_address); +#endif writeBinary(sig, out); writePODBinary(*info, out); From e0c8ae8f4baf9a9571aaa02e7d8a06610cf91d9e Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 27 May 2024 10:44:14 +0000 Subject: [PATCH 144/856] fix tests --- src/Core/SettingsChangesHistory.h | 1 + tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 16f28d94640..4c087060179 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -106,6 +106,7 @@ static std::map sett {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, + {"min_untracked_memory", 4_MiB, 4_KiB, "A new setting."}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, diff --git a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql index 69bd15e3f54..68472a93c9c 100644 --- a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql +++ b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql @@ -7,7 +7,8 @@ -- sizeof(HLL) is (2^K * 6 / 8) -- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400 -SET use_uncompressed_cache = 0; +SET use_uncompressed_cache = 0; +SET min_untracked_memory = 4194304; -- 4MiB -- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements SELECT 'UInt32'; From 4f3e2cae11aa1fccd14eff9bc6978269611caf10 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 27 May 2024 11:04:52 +0000 Subject: [PATCH 145/856] start keeper global thread pool with sufficient amount of threads Signed-off-by: Duc Canh Le --- programs/keeper/Keeper.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index dba5c2b7d2a..0d3c1f10894 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -361,9 +361,10 @@ try } GlobalThreadPool::initialize( - config().getUInt("max_thread_pool_size", 100), - config().getUInt("max_thread_pool_free_size", 1000), - config().getUInt("thread_pool_queue_size", 10000) + /// We need to have sufficient amount of threads for connections + nuraft workers + keeper workers, 1000 is an estimation + std::min(1000U, config().getUInt("max_thread_pool_size", 1000)), + config().getUInt("max_thread_pool_free_size", 100), + config().getUInt("thread_pool_queue_size", 1000) ); /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ From 01a16fd8e2d7ba65ee9feddcfb1cf7609be02947 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 27 May 2024 11:33:01 +0000 Subject: [PATCH 146/856] 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 2a433112ee4e695d59ee79445c96a87f1d0caea3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 27 May 2024 14:01:52 +0000 Subject: [PATCH 147/856] Add test --- .../0_stateless/03164_linestring_geometry.reference | 11 +++++++++++ .../queries/0_stateless/03164_linestring_geometry.sql | 8 ++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/03164_linestring_geometry.reference create mode 100644 tests/queries/0_stateless/03164_linestring_geometry.sql diff --git a/tests/queries/0_stateless/03164_linestring_geometry.reference b/tests/queries/0_stateless/03164_linestring_geometry.reference new file mode 100644 index 00000000000..1f68df04614 --- /dev/null +++ b/tests/queries/0_stateless/03164_linestring_geometry.reference @@ -0,0 +1,11 @@ +-- { echoOn } +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +[(1,1),(2,2),(3,3),(1,1)] +SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +LineString +SELECT wkt(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +LINESTRING(1 1,2 2,3 3,1 1) +-- Native Array(Tuple(Float64, Float64)) is threated as Ring, not as LineString. +WITH wkt(CAST([(1, 1), (2, 2), (3, 3)], 'Array(Tuple(Float64, Float64))')) as x +SELECT x, toTypeName(x), readWKTRing(x) as y, toTypeName(y); +POLYGON((1 1,2 2,3 3)) String [(1,1),(2,2),(3,3)] Ring diff --git a/tests/queries/0_stateless/03164_linestring_geometry.sql b/tests/queries/0_stateless/03164_linestring_geometry.sql new file mode 100644 index 00000000000..e4f1d1295e7 --- /dev/null +++ b/tests/queries/0_stateless/03164_linestring_geometry.sql @@ -0,0 +1,8 @@ +-- { echoOn } +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +SELECT wkt(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); + +-- Native Array(Tuple(Float64, Float64)) is threated as Ring, not as LineString. +WITH wkt(CAST([(1, 1), (2, 2), (3, 3)], 'Array(Tuple(Float64, Float64))')) as x +SELECT x, toTypeName(x), readWKTRing(x) as y, toTypeName(y); From 5abebeca63d02f12464fdd4d3067d988f0005104 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 May 2024 16:08:55 +0200 Subject: [PATCH 148/856] Use BaseSettings --- src/Backups/BackupIO_S3.cpp | 2 +- src/Core/BaseSettings.h | 44 +++- .../ObjectStorages/S3/S3ObjectStorage.cpp | 3 +- src/IO/S3/copyS3File.cpp | 30 ++- src/IO/S3Common.cpp | 234 +++++++----------- src/IO/S3Common.h | 61 +++-- src/IO/WriteBufferFromS3.cpp | 29 ++- src/IO/WriteBufferFromS3.h | 1 - 8 files changed, 197 insertions(+), 207 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index cbf18e2bff9..8c16aa0c291 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -228,7 +228,7 @@ BackupWriterS3::BackupWriterS3( request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint request_settings.allow_native_copy = allow_s3_native_copy; - request_settings.upload_settings.storage_class_name = storage_class_name; + request_settings.storage_class_name = storage_class_name; client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); if (auto blob_storage_system_log = context_->getBlobStorageLog()) { diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index adf7a41193c..6242d78aee7 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -108,6 +108,7 @@ public: public: const String & getName() const; Field getValue() const; + void setValue(const Field & value); Field getDefaultValue() const; String getValueString() const; String getDefaultValueString() const; @@ -122,10 +123,10 @@ public: private: friend class BaseSettings; - const BaseSettings * settings; + BaseSettings * settings; const typename Traits::Accessor * accessor; size_t index; - std::conditional_t custom_setting; + std::conditional_t custom_setting; }; enum SkipFlags @@ -144,35 +145,50 @@ public: Iterator & operator++(); Iterator operator++(int); /// NOLINT const SettingFieldRef & operator *() const { return field_ref; } + SettingFieldRef & operator *() { return field_ref; } bool operator ==(const Iterator & other) const; bool operator !=(const Iterator & other) const { return !(*this == other); } private: friend class BaseSettings; - Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_); + Iterator(BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_); void doSkip(); void setPointerToCustomSetting(); SettingFieldRef field_ref; - std::conditional_t custom_settings_iterator; + std::conditional_t custom_settings_iterator; SkipFlags skip_flags; }; class Range { public: - Range(const BaseSettings & settings_, SkipFlags skip_flags_) : settings(settings_), accessor(Traits::Accessor::instance()), skip_flags(skip_flags_) {} + Range(BaseSettings & settings_, SkipFlags skip_flags_) : settings(settings_), accessor(Traits::Accessor::instance()), skip_flags(skip_flags_) {} Iterator begin() const { return Iterator(settings, accessor, skip_flags); } Iterator end() const { return Iterator(settings, accessor, SKIP_ALL); } private: - const BaseSettings & settings; + BaseSettings & settings; const typename Traits::Accessor & accessor; SkipFlags skip_flags; }; - Range all(SkipFlags skip_flags = SKIP_NONE) const { return Range{*this, skip_flags}; } + class MutableRange + { + public: + MutableRange(BaseSettings & settings_, SkipFlags skip_flags_) : settings(settings_), accessor(Traits::Accessor::instance()), skip_flags(skip_flags_) {} + Iterator begin() { return Iterator(settings, accessor, skip_flags); } + Iterator end() { return Iterator(settings, accessor, SKIP_ALL); } + + private: + BaseSettings & settings; + const typename Traits::Accessor & accessor; + SkipFlags skip_flags; + }; + + Range all(SkipFlags skip_flags = SKIP_NONE) const { return Range{const_cast &>(*this), skip_flags}; } + MutableRange allMutable(SkipFlags skip_flags = SKIP_NONE) { return MutableRange{*this, skip_flags}; } Range allChanged() const { return all(SKIP_UNCHANGED); } Range allUnchanged() const { return all(SKIP_CHANGED); } Range allBuiltin() const { return all(SKIP_CUSTOM); } @@ -608,7 +624,7 @@ const SettingFieldCustom * BaseSettings::tryGetCustomSetting(std::strin } template -BaseSettings::Iterator::Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_) +BaseSettings::Iterator::Iterator(BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_) : skip_flags(skip_flags_) { field_ref.settings = &settings_; @@ -741,6 +757,18 @@ Field BaseSettings::SettingFieldRef::getValue() const return accessor->getValue(*settings, index); } +template +void BaseSettings::SettingFieldRef::setValue(const Field & value) +{ + if constexpr (Traits::allow_custom_settings) + { + if (custom_setting) + custom_setting->second = value; + } + else + accessor->setValue(*settings, index, value); +} + template Field BaseSettings::SettingFieldRef::getDefaultValue() const { diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 013b44a3c7b..00a1216e52a 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -577,6 +577,7 @@ void S3ObjectStorage::applyNewSettings( auto settings_from_config = getSettings(config, config_prefix, context, for_disk_s3, context->getSettingsRef().s3_validate_request_settings); auto modified_settings = std::make_unique(*s3_settings.get()); modified_settings->auth_settings.updateFrom(settings_from_config->auth_settings); + modified_settings->request_settings.updateIfChanged(settings_from_config->request_settings); if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString(), context->getUserName())) modified_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); @@ -585,7 +586,7 @@ void S3ObjectStorage::applyNewSettings( if (options.allow_client_change && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3)) { - auto new_client = getClient(uri, *new_s3_settings, context, for_disk_s3); + auto new_client = getClient(uri, *settings_from_config, context, for_disk_s3); client.set(std::move(new_client)); } s3_settings.set(std::move(modified_settings)); diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 471c4a687a6..35b75a5cc90 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -66,7 +66,6 @@ namespace , dest_bucket(dest_bucket_) , dest_key(dest_key_) , request_settings(request_settings_) - , upload_settings(request_settings.upload_settings) , object_metadata(object_metadata_) , schedule(schedule_) , for_disk_s3(for_disk_s3_) @@ -82,7 +81,6 @@ namespace const String & dest_bucket; const String & dest_key; const S3::RequestSettings & request_settings; - const S3::RequestSettings::PartUploadSettings & upload_settings; const std::optional> & object_metadata; ThreadPoolCallbackRunnerUnsafe schedule; bool for_disk_s3; @@ -127,8 +125,8 @@ namespace if (object_metadata.has_value()) request.SetMetadata(object_metadata.value()); - const auto & storage_class_name = upload_settings.storage_class_name; - if (!storage_class_name.empty()) + const auto & storage_class_name = request_settings.storage_class_name; + if (!storage_class_name.value.empty()) request.SetStorageClass(Aws::S3::Model::StorageClassMapper::GetStorageClassForName(storage_class_name)); client_ptr->setKMSHeaders(request); @@ -185,7 +183,7 @@ namespace request.SetMultipartUpload(multipart_upload); - size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); @@ -290,9 +288,9 @@ namespace if (!total_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chosen multipart upload for an empty file. This must not happen"); - auto max_part_number = upload_settings.max_part_number; - auto min_upload_part_size = upload_settings.min_upload_part_size; - auto max_upload_part_size = upload_settings.max_upload_part_size; + auto max_part_number = request_settings.max_part_number; + auto min_upload_part_size = request_settings.min_upload_part_size; + auto max_upload_part_size = request_settings.max_upload_part_size; if (!max_part_number) throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_part_number must not be 0"); @@ -479,7 +477,7 @@ namespace void performCopy() { - if (size <= upload_settings.max_single_part_upload_size) + if (size <= request_settings.max_single_part_upload_size) performSinglepartUpload(); else performMultipartUpload(); @@ -512,8 +510,8 @@ namespace if (object_metadata.has_value()) request.SetMetadata(object_metadata.value()); - const auto & storage_class_name = upload_settings.storage_class_name; - if (!storage_class_name.empty()) + const auto & storage_class_name = request_settings.storage_class_name; + if (!storage_class_name.value.empty()) request.SetStorageClass(Aws::S3::Model::StorageClassMapper::GetStorageClassForName(storage_class_name)); /// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840 @@ -524,7 +522,7 @@ namespace void processPutRequest(S3::PutObjectRequest & request) { - size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3PutObject); @@ -666,7 +664,7 @@ namespace void performCopy() { LOG_TEST(log, "Copy object {} to {} using native copy", src_key, dest_key); - if (!supports_multipart_copy || size <= upload_settings.max_single_operation_copy_size) + if (!supports_multipart_copy || size <= request_settings.max_single_operation_copy_size) performSingleOperationCopy(); else performMultipartUploadCopy(); @@ -710,8 +708,8 @@ namespace request.SetMetadataDirective(Aws::S3::Model::MetadataDirective::REPLACE); } - const auto & storage_class_name = upload_settings.storage_class_name; - if (!storage_class_name.empty()) + const auto & storage_class_name = request_settings.storage_class_name; + if (!storage_class_name.value.empty()) request.SetStorageClass(Aws::S3::Model::StorageClassMapper::GetStorageClassForName(storage_class_name)); /// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840 @@ -722,7 +720,7 @@ namespace void processCopyRequest(S3::CopyObjectRequest & request) { - size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3CopyObject); diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 8a01f6ca33a..b3cd037e91d 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -237,113 +237,68 @@ RequestSettings RequestSettings::loadFromConfig( const std::string & setting_name_prefix) { auto request_settings = RequestSettings::loadFromSettings(settings, validate_settings); - String prefix = config_prefix + "." + setting_name_prefix; - auto has = [&](const std::string & key) -> bool { return config.has(prefix + key); }; - auto get_uint = [&](const std::string & key) -> size_t { return config.getUInt64(prefix + key); }; - auto get_string = [&](const std::string & key) -> std::string { return config.getString(prefix + key); }; - auto get_bool = [&](const std::string & key) -> bool { return config.getBool(prefix + key); }; - if (has("strict_upload_part_size")) - request_settings.upload_settings.strict_upload_part_size = get_uint("strict_upload_part_size"); - if (has("min_upload_part_size")) - request_settings.upload_settings.min_upload_part_size = get_uint("min_upload_part_size"); - if (has("max_upload_part_size")) - request_settings.upload_settings.max_upload_part_size = get_uint("max_upload_part_size"); - if (has("upload_part_size_multiply_factor")) - request_settings.upload_settings.upload_part_size_multiply_factor = get_uint("upload_part_size_multiply_factor"); - if (has("upload_part_size_multiply_parts_count_threshold")) - request_settings.upload_settings.upload_part_size_multiply_parts_count_threshold = get_uint("upload_part_size_multiply_parts_count_threshold"); - if (has("max_inflight_parts_for_one_file")) - request_settings.upload_settings.max_inflight_parts_for_one_file = get_uint("max_inflight_parts_for_one_file"); - if (has("max_part_number")) - request_settings.upload_settings.max_part_number = get_uint("max_part_number"); - if (has("max_single_part_upload_size")) - request_settings.upload_settings.max_single_part_upload_size = get_uint("max_single_part_upload_size"); - if (has("max_single_operation_copy_size")) - request_settings.upload_settings.max_single_operation_copy_size = get_uint("max_single_operation_copy_size"); - if (has("s3_storage_class")) - request_settings.upload_settings.storage_class_name = get_string("s3_storage_class"); + auto values = request_settings.allMutable(); + for (auto & field : values) + { + const auto path = prefix + field.getName(); + if (config.has(path)) + { + auto which = field.getValue().getType(); + if (isInt64OrUInt64FieldType(which)) + field.setValue(config.getUInt64(path)); + else if (which == Field::Types::String) + field.setValue(config.getString(path)); + else if (which == Field::Types::Bool) + field.setValue(config.getBool(path)); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); + } + } + + if (!request_settings.storage_class_name.value.empty()) + request_settings.storage_class_name = Poco::toUpperInPlace(request_settings.storage_class_name.value); - request_settings.upload_settings.storage_class_name = Poco::toUpperInPlace(request_settings.upload_settings.storage_class_name); if (validate_settings) - request_settings.upload_settings.validate(); + request_settings.validateUploadSettings(); - if (has("max_single_read_retries")) - request_settings.max_single_read_retries = get_uint("max_single_read_retries"); - if (has("check_objects_after_upload")) - request_settings.check_objects_after_upload = get_bool("check_objects_after_upload"); - if (has("list_object_keys_size")) - request_settings.list_object_keys_size = get_uint("list_object_keys_size"); - if (has("allow_native_copy")) - request_settings.allow_native_copy = get_bool("allow_native_copy"); - if (has("throw_on_zero_files_match")) - request_settings.throw_on_zero_files_match = get_bool("throw_on_zero_files_match"); - if (has("request_timeout_ms")) - request_settings.request_timeout_ms = get_uint("request_timeout_ms"); + request_settings.initializeThrottler(settings); - /// NOTE: it would be better to reuse old throttlers - /// to avoid losing token bucket state on every config reload, - /// which could lead to exceeding limit for short time. - /// But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = has("max_get_rps") ? get_uint("max_get_rps") : settings.s3_max_get_rps) - { - size_t default_max_get_burst = settings.s3_max_get_burst - ? settings.s3_max_get_burst - : (Throttler::default_burst_seconds * max_get_rps); - size_t max_get_burst = has("max_get_burst") ? get_uint("max_get_burst") : default_max_get_burst; - request_settings.get_request_throttler = std::make_shared(max_get_rps, max_get_burst); - } - if (UInt64 max_put_rps = has("max_put_rps") ? get_uint("max_put_rps") : settings.s3_max_put_rps) - { - size_t default_max_put_burst = settings.s3_max_put_burst - ? settings.s3_max_put_burst - : (Throttler::default_burst_seconds * max_put_rps); - size_t max_put_burst = has("max_put_burst") ? get_uint("max_put_burst") : default_max_put_burst; - request_settings.put_request_throttler = std::make_shared(max_put_rps, max_put_burst); - } return request_settings; } RequestSettings RequestSettings::loadFromNamedCollection(const NamedCollection & collection, bool validate_settings) { - RequestSettings settings{}; + RequestSettings request_settings{}; - if (collection.has("strict_upload_part_size")) - settings.upload_settings.strict_upload_part_size = collection.get("strict_upload_part_size"); - if (collection.has("min_upload_part_size")) - settings.upload_settings.min_upload_part_size = collection.get("min_upload_part_size"); - if (collection.has("max_upload_part_size")) - settings.upload_settings.min_upload_part_size = collection.get("max_upload_part_size"); - if (collection.has("upload_part_size_multiply_factor")) - settings.upload_settings.upload_part_size_multiply_factor = collection.get("upload_part_size_multiply_factor"); - if (collection.has("upload_part_size_multiply_parts_count_threshold")) - settings.upload_settings.upload_part_size_multiply_parts_count_threshold = collection.get("upload_part_size_multiply_parts_count_threshold"); - if (collection.has("max_inflight_parts_for_one_file")) - settings.upload_settings.max_inflight_parts_for_one_file = collection.get("max_inflight_parts_for_one_file"); - if (collection.has("max_part_number")) - settings.upload_settings.max_single_part_upload_size = collection.get("max_part_number"); - if (collection.has("max_single_part_upload_size")) - settings.upload_settings.max_single_part_upload_size = collection.get("max_single_part_upload_size"); - if (collection.has("max_single_operation_copy_size")) - settings.upload_settings.max_single_part_upload_size = collection.get("max_single_operation_copy_size"); - if (collection.has("s3_storage_class")) - settings.upload_settings.storage_class_name = collection.get("s3_storage_class"); + auto values = request_settings.allMutable(); + for (auto & field : values) + { + const auto path = field.getName(); + if (collection.has(path)) + { + auto which = field.getValue().getType(); + if (isInt64OrUInt64FieldType(which)) + field.setValue(collection.get(path)); + else if (which == Field::Types::String) + field.setValue(collection.get(path)); + else if (which == Field::Types::Bool) + field.setValue(collection.get(path)); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); + } + } + + if (!request_settings.storage_class_name.value.empty()) + request_settings.storage_class_name = Poco::toUpperInPlace(request_settings.storage_class_name.value); - settings.upload_settings.storage_class_name = Poco::toUpperInPlace(settings.upload_settings.storage_class_name); if (validate_settings) - settings.upload_settings.validate(); + request_settings.validateUploadSettings(); - if (collection.has("max_single_read_retries")) - settings.max_single_read_retries = collection.get("max_single_read_retries"); - if (collection.has("list_object_keys_size")) - settings.list_object_keys_size = collection.get("list_object_keys_size"); - if (collection.has("allow_native_copy")) - settings.allow_native_copy = collection.get("allow_native_copy"); - if (collection.has("throw_on_zero_files_match")) - settings.throw_on_zero_files_match = collection.get("throw_on_zero_files_match"); + // request_settings.initializeThrottler(settings); - return settings; + return request_settings; } RequestSettings RequestSettings::loadFromSettings(const DB::Settings & settings, bool validate_settings) @@ -355,58 +310,57 @@ RequestSettings RequestSettings::loadFromSettings(const DB::Settings & settings, void RequestSettings::updateFromSettings(const DB::Settings & settings, bool if_changed, bool validate_settings) { - if (!if_changed || settings.s3_strict_upload_part_size.changed) - upload_settings.strict_upload_part_size = settings.s3_strict_upload_part_size; - if (!if_changed || settings.s3_min_upload_part_size.changed) - upload_settings.min_upload_part_size = settings.s3_min_upload_part_size; - if (!if_changed || settings.s3_max_upload_part_size.changed) - upload_settings.max_upload_part_size = settings.s3_max_upload_part_size; - if (!if_changed || settings.s3_upload_part_size_multiply_factor.changed) - upload_settings.upload_part_size_multiply_factor = settings.s3_upload_part_size_multiply_factor; - if (!if_changed || settings.s3_upload_part_size_multiply_parts_count_threshold.changed) - upload_settings.upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; - if (!if_changed || settings.s3_max_inflight_parts_for_one_file.changed) - upload_settings.max_inflight_parts_for_one_file = settings.s3_max_inflight_parts_for_one_file; - if (!if_changed || settings.s3_max_part_number.changed) - upload_settings.max_part_number = settings.s3_max_part_number; - if (!if_changed || settings.s3_max_single_part_upload_size.changed) - upload_settings.max_single_part_upload_size = settings.s3_max_single_part_upload_size; - if (!if_changed || settings.s3_max_single_operation_copy_size.changed) - upload_settings.max_part_number = settings.s3_max_single_operation_copy_size; + for (auto & field : allMutable()) + { + const auto setting_name = "s3_" + field.getName(); + if (settings.has(setting_name) && (!if_changed || settings.isChanged(setting_name))) + { + set(field.getName(), settings.get(setting_name)); + } + } + + if (!storage_class_name.value.empty()) + storage_class_name = Poco::toUpperInPlace(storage_class_name.value); if (validate_settings) - upload_settings.validate(); + validateUploadSettings(); +} - if (!if_changed || settings.s3_max_single_read_retries.changed) - max_single_read_retries = settings.s3_max_single_read_retries; - if (!if_changed || settings.s3_check_objects_after_upload.changed) - check_objects_after_upload = settings.s3_check_objects_after_upload; - if (!if_changed || settings.s3_max_unexpected_write_error_retries.changed) - max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; - if (!if_changed || settings.s3_list_object_keys_size.changed) - list_object_keys_size = settings.s3_list_object_keys_size; - if (!if_changed || settings.s3_throw_on_zero_files_match.changed) - throw_on_zero_files_match = settings.s3_throw_on_zero_files_match; - if (!if_changed || settings.s3_request_timeout_ms.changed) - request_timeout_ms = settings.s3_request_timeout_ms; - - if ((!if_changed || settings.s3_max_get_rps.changed || settings.s3_max_get_burst.changed) && settings.s3_max_get_rps) +void RequestSettings::updateIfChanged(const RequestSettings & settings) +{ + for (auto & setting : settings.all()) { - size_t max_get_burst = settings.s3_max_get_burst - ? settings.s3_max_get_burst - : Throttler::default_burst_seconds * settings.s3_max_get_rps; - get_request_throttler = std::make_shared(settings.s3_max_get_rps, max_get_burst); - } - if ((!if_changed || settings.s3_max_put_rps.changed || settings.s3_max_put_burst.changed) && settings.s3_max_put_rps) - { - size_t max_put_burst = settings.s3_max_put_burst - ? settings.s3_max_put_burst - : Throttler::default_burst_seconds * settings.s3_max_put_rps; - put_request_throttler = std::make_shared(settings.s3_max_put_rps, max_put_burst); + if (setting.isValueChanged()) + set(setting.getName(), setting.getValue()); } } -void RequestSettings::PartUploadSettings::validate() +void RequestSettings::initializeThrottler(const DB::Settings & settings) +{ + /// NOTE: it would be better to reuse old throttlers + /// to avoid losing token bucket state on every config reload, + /// which could lead to exceeding limit for short time. + /// But it is good enough unless very high `burst` values are used. + if (UInt64 max_get_rps = isChanged("max_get_rps") ? get("max_get_rps").get() : settings.s3_max_get_rps) + { + size_t default_max_get_burst = settings.s3_max_get_burst + ? settings.s3_max_get_burst + : (Throttler::default_burst_seconds * max_get_rps); + + size_t max_get_burst = isChanged("max_get_burts") ? get("max_get_burst").get() : default_max_get_burst; + get_request_throttler = std::make_shared(max_get_rps, max_get_burst); + } + if (UInt64 max_put_rps = isChanged("max_put_rps") ? get("max_put_rps").get() : settings.s3_max_put_rps) + { + size_t default_max_put_burst = settings.s3_max_put_burst + ? settings.s3_max_put_burst + : (Throttler::default_burst_seconds * max_put_rps); + size_t max_put_burst = isChanged("max_put_burts") ? get("max_put_burst").get() : default_max_put_burst; + put_request_throttler = std::make_shared(max_put_rps, max_put_burst); + } +} + +void RequestSettings::validateUploadSettings() { static constexpr size_t min_upload_part_size_limit = 5 * 1024 * 1024; if (strict_upload_part_size && strict_upload_part_size < min_upload_part_size_limit) @@ -469,7 +423,7 @@ void RequestSettings::PartUploadSettings::validate() ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); size_t maybe_overflow; - if (common::mulOverflow(max_upload_part_size, upload_part_size_multiply_factor, maybe_overflow)) + if (common::mulOverflow(max_upload_part_size.value, upload_part_size_multiply_factor.value, maybe_overflow)) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, "Setting upload_part_size_multiply_factor is too big ({}). " @@ -477,11 +431,11 @@ void RequestSettings::PartUploadSettings::validate() ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); std::unordered_set storage_class_names {"STANDARD", "INTELLIGENT_TIERING"}; - if (!storage_class_name.empty() && !storage_class_names.contains(storage_class_name)) + if (!storage_class_name.value.empty() && !storage_class_names.contains(storage_class_name)) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, "Setting storage_class has invalid value {} which only supports STANDARD and INTELLIGENT_TIERING", - storage_class_name); + storage_class_name.value); /// TODO: it's possible to set too small limits. /// We can check that max possible object size is not too small. @@ -489,4 +443,6 @@ void RequestSettings::PartUploadSettings::validate() } +IMPLEMENT_SETTINGS_TRAITS(S3::RequestSettingsTraits, REQUEST_SETTINGS_LIST) + } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 01a7ca56779..b47e3e79409 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -123,34 +124,38 @@ private: bool operator==(const AuthSettings & other) const = default; }; -struct RequestSettings +#define REQUEST_SETTINGS(M, ALIAS) \ + M(UInt64, max_single_read_retries, 4, "", 0) \ + M(UInt64, request_timeout_ms, DEFAULT_REQUEST_TIMEOUT_MS, "", 0) \ + M(UInt64, list_object_keys_size, 1000, "", 0) \ + M(Bool, allow_native_copy, true, "", 0) \ + M(Bool, check_objects_after_upload, false, "", 0) \ + M(Bool, throw_on_zero_files_match, false, "", 0) \ + M(UInt64, max_single_operation_copy_size, DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE, "", 0) \ + M(String, storage_class_name, "", "", 0) \ + +#define PART_UPLOAD_SETTINGS(M, ALIAS) \ + M(UInt64, strict_upload_part_size, 0, "", 0) \ + M(UInt64, min_upload_part_size, DEFAULT_MIN_UPLOAD_PART_SIZE, "", 0) \ + M(UInt64, max_upload_part_size, DEFAULT_MAX_UPLOAD_PART_SIZE, "", 0) \ + M(UInt64, upload_part_size_multiply_factor, DEFAULT_UPLOAD_PART_SIZE_MULTIPLY_FACTOR, "", 0) \ + M(UInt64, upload_part_size_multiply_parts_count_threshold, DEFAULT_UPLOAD_PART_SIZE_MULTIPLY_PARTS_COUNT_THRESHOLD, "", 0) \ + M(UInt64, max_inflight_parts_for_one_file, DEFAULT_MAX_INFLIGHT_PARTS_FOR_ONE_FILE, "", 0) \ + M(UInt64, max_part_number, DEFAULT_MAX_PART_NUMBER, "", 0) \ + M(UInt64, max_single_part_upload_size, DEFAULT_MAX_SINGLE_PART_UPLOAD_SIZE, "", 0) \ + M(UInt64, max_unexpected_write_error_retries, 4, "", 0) \ + + +#define REQUEST_SETTINGS_LIST(M, ALIAS) \ + REQUEST_SETTINGS(M, ALIAS) \ + PART_UPLOAD_SETTINGS(M, ALIAS) + +DECLARE_SETTINGS_TRAITS(RequestSettingsTraits, REQUEST_SETTINGS_LIST) + +struct RequestSettings : public BaseSettings { - size_t max_single_read_retries = 4; - size_t request_timeout_ms = 30000; - size_t max_unexpected_write_error_retries = 4; - size_t list_object_keys_size = 1000; + void validateUploadSettings(); - bool allow_native_copy = true; - bool check_objects_after_upload = false; - bool throw_on_zero_files_match = false; - - struct PartUploadSettings - { - size_t strict_upload_part_size = 0; - size_t min_upload_part_size = 16 * 1024 * 1024; - size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; - size_t upload_part_size_multiply_factor = 2; - size_t upload_part_size_multiply_parts_count_threshold = 500; - size_t max_inflight_parts_for_one_file = 20; - size_t max_part_number = 10000; - size_t max_single_part_upload_size = 32 * 1024 * 1024; - size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; - String storage_class_name; - - void validate(); - }; - - PartUploadSettings upload_settings; ThrottlerPtr get_request_throttler; ThrottlerPtr put_request_throttler; @@ -164,6 +169,10 @@ struct RequestSettings const std::string & setting_name_prefix = ""); void updateFromSettings(const DB::Settings & settings, bool if_changed, bool validate_settings = true); + void updateIfChanged(const RequestSettings & settings); + +private: + void initializeThrottler(const DB::Settings & settings); }; } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index b83ca174820..982d1485efd 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -72,7 +72,7 @@ struct WriteBufferFromS3::PartData } }; -BufferAllocationPolicyPtr createBufferAllocationPolicy(const S3::RequestSettings::PartUploadSettings & settings) +BufferAllocationPolicyPtr createBufferAllocationPolicy(const S3::RequestSettings & settings) { BufferAllocationPolicy::Settings allocation_settings; allocation_settings.strict_size = settings.strict_upload_part_size; @@ -100,15 +100,14 @@ WriteBufferFromS3::WriteBufferFromS3( , bucket(bucket_) , key(key_) , request_settings(request_settings_) - , upload_settings(request_settings.upload_settings) , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) - , buffer_allocation_policy(createBufferAllocationPolicy(upload_settings)) + , buffer_allocation_policy(createBufferAllocationPolicy(request_settings)) , task_tracker( std::make_unique( std::move(schedule_), - upload_settings.max_inflight_parts_for_one_file, + request_settings.max_inflight_parts_for_one_file, limitedLog)) , blob_log(std::move(blob_log_)) { @@ -165,7 +164,7 @@ void WriteBufferFromS3::preFinalize() if (multipart_upload_id.empty() && detached_part_data.size() <= 1) { - if (detached_part_data.empty() || detached_part_data.front().data_size <= upload_settings.max_single_part_upload_size) + if (detached_part_data.empty() || detached_part_data.front().data_size <= request_settings.max_single_part_upload_size) do_single_part_upload = true; } @@ -499,18 +498,18 @@ void WriteBufferFromS3::writePart(WriteBufferFromS3::PartData && data) "Unable to write a part without multipart_upload_id, details: WriteBufferFromS3 created for bucket {}, key {}", bucket, key); - if (part_number > upload_settings.max_part_number) + if (part_number > request_settings.max_part_number) { throw Exception( ErrorCodes::INVALID_CONFIG_PARAMETER, "Part number exceeded {} while writing {} bytes to S3. Check min_upload_part_size = {}, max_upload_part_size = {}, " "upload_part_size_multiply_factor = {}, upload_part_size_multiply_parts_count_threshold = {}, max_single_part_upload_size = {}", - upload_settings.max_part_number, count(), upload_settings.min_upload_part_size, upload_settings.max_upload_part_size, - upload_settings.upload_part_size_multiply_factor, upload_settings.upload_part_size_multiply_parts_count_threshold, - upload_settings.max_single_part_upload_size); + request_settings.max_part_number, count(), request_settings.min_upload_part_size, request_settings.max_upload_part_size, + request_settings.upload_part_size_multiply_factor, request_settings.upload_part_size_multiply_parts_count_threshold, + request_settings.max_single_part_upload_size); } - if (data.data_size > upload_settings.max_upload_part_size) + if (data.data_size > request_settings.max_upload_part_size) { throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -518,7 +517,7 @@ void WriteBufferFromS3::writePart(WriteBufferFromS3::PartData && data) getShortLogDetails(), part_number, data.data_size, - upload_settings.max_upload_part_size + request_settings.max_upload_part_size ); } @@ -605,7 +604,7 @@ void WriteBufferFromS3::completeMultipartUpload() req.SetMultipartUpload(multipart_upload); - size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); @@ -663,8 +662,8 @@ S3::PutObjectRequest WriteBufferFromS3::getPutRequest(PartData & data) req.SetBody(data.createAwsBuffer()); if (object_metadata.has_value()) req.SetMetadata(object_metadata.value()); - if (!upload_settings.storage_class_name.empty()) - req.SetStorageClass(Aws::S3::Model::StorageClassMapper::GetStorageClassForName(upload_settings.storage_class_name)); + if (!request_settings.storage_class_name.value.empty()) + req.SetStorageClass(Aws::S3::Model::StorageClassMapper::GetStorageClassForName(request_settings.storage_class_name)); /// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840 req.SetContentType("binary/octet-stream"); @@ -688,7 +687,7 @@ void WriteBufferFromS3::makeSinglepartUpload(WriteBufferFromS3::PartData && data auto & request = std::get<0>(*worker_data); size_t content_length = request.GetContentLength(); - size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3PutObject); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index d4e25ea1733..973ca4c7526 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -79,7 +79,6 @@ private: const String bucket; const String key; const S3::RequestSettings request_settings; - const S3::RequestSettings::PartUploadSettings & upload_settings; const WriteSettings write_settings; const std::shared_ptr client_ptr; const std::optional> object_metadata; From 8eb79c7cb3bbfe5d74e030507e4b9a2e695b1971 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 27 May 2024 16:47:50 +0200 Subject: [PATCH 149/856] adjust test test_mask_sensitive_info/test.py::test_backup_to_s3 --- tests/integration/test_mask_sensitive_info/configs/users.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_mask_sensitive_info/configs/users.xml b/tests/integration/test_mask_sensitive_info/configs/users.xml index f129a5bb3e3..f767216e907 100644 --- a/tests/integration/test_mask_sensitive_info/configs/users.xml +++ b/tests/integration/test_mask_sensitive_info/configs/users.xml @@ -2,6 +2,7 @@ 5 + 5 From c4e32a8961a8dd6c3220bc0d6675bfbb2c5a40ee Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 27 May 2024 16:00:45 +0000 Subject: [PATCH 150/856] Fix --- src/Planner/PlannerJoinTree.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 68ebf4135c7..96b73a26095 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -891,11 +891,13 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (reading) break; + last_node = node; if (!node->children.empty()) + { node = node->children.at(0); + } else { - last_node = node; node = nullptr; } } From 00b07bba146848cfecf8a4f59c78161d24aa3566 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 27 May 2024 16:23:01 +0000 Subject: [PATCH 151/856] 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 56c7301d468d5bda2fc3a0fd512eff70ce6b1b3b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 May 2024 17:14:19 +0200 Subject: [PATCH 152/856] Use BaseSettings for auth settings --- src/Backups/BackupIO_S3.cpp | 12 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 10 +- .../ObjectStorages/ObjectStorageFactory.cpp | 6 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 8 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 47 ++- src/Disks/ObjectStorages/S3/diskSettings.h | 1 + src/IO/S3Common.cpp | 269 +++++++----------- src/IO/S3Common.h | 133 +++++---- src/IO/S3Defines.h | 3 + src/IO/S3Settings.cpp | 4 +- .../ObjectStorage/Azure/Configuration.cpp | 2 +- .../ObjectStorage/Azure/Configuration.h | 2 +- .../ObjectStorage/HDFS/Configuration.cpp | 2 +- .../ObjectStorage/HDFS/Configuration.h | 2 +- .../ObjectStorage/S3/Configuration.cpp | 17 +- src/Storages/ObjectStorage/S3/Configuration.h | 2 +- .../ObjectStorage/StorageObjectStorage.cpp | 2 +- .../ObjectStorage/StorageObjectStorage.h | 2 +- 18 files changed, 235 insertions(+), 289 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 8c16aa0c291..b8ade4be027 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -88,14 +88,10 @@ namespace std::move(headers), S3::CredentialsConfiguration { - settings.auth_settings.use_environment_credentials.value_or( - context->getConfigRef().getBool("s3.use_environment_credentials", true)), - settings.auth_settings.use_insecure_imds_request.value_or( - context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - settings.auth_settings.expiration_window_seconds.value_or( - context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), - settings.auth_settings.no_sign_request.value_or( - context->getConfigRef().getBool("s3.no_sign_request", false)), + settings.auth_settings.use_environment_credentials, + settings.auth_settings.use_insecure_imds_request, + settings.auth_settings.expiration_window_seconds, + settings.auth_settings.no_sign_request }); } diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index acf8faa9edd..b8c8d10d497 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -65,7 +65,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo return; } - auto auth_settings = S3::AuthSettings::loadFromConfig(config, config_prefix, Context::getGlobalContextInstance()->getSettingsRef()); + auto auth_settings = S3::AuthSettings(config, config_prefix, Context::getGlobalContextInstance()->getSettingsRef()); String endpoint = macros->expand(config.getString(config_prefix + ".endpoint")); auto new_uri = S3::URI{endpoint}; @@ -119,10 +119,10 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo std::move(headers), S3::CredentialsConfiguration { - auth_settings.use_environment_credentials.value_or(true), - auth_settings.use_insecure_imds_request.value_or(false), - auth_settings.expiration_window_seconds.value_or(S3::DEFAULT_EXPIRATION_WINDOW_SECONDS), - auth_settings.no_sign_request.value_or(false), + auth_settings.use_environment_credentials, + auth_settings.use_insecure_imds_request, + auth_settings.expiration_window_seconds, + auth_settings.no_sign_request, }, credentials.GetSessionToken()); diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 81de22811fe..14c0d656cbf 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -190,8 +190,8 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) { auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); - auto settings = getSettings(config, config_prefix, context, /* for_disk_s3 */true, /* validate_settings */true); auto endpoint = getEndpoint(config, config_prefix, context); + auto settings = getSettings(config, config_prefix, context, endpoint, /* for_disk_s3 */true, /* validate_settings */true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); @@ -227,8 +227,8 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); - auto settings = getSettings(config, config_prefix, context, /* for_disk_s3 */true, /* validate_settings */true); auto endpoint = getEndpoint(config, config_prefix, context); + auto settings = getSettings(config, config_prefix, context, endpoint, /* for_disk_s3 */true, /* validate_settings */true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); @@ -262,8 +262,8 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); - auto settings = getSettings(config, config_prefix, context, /* for_disk_s3 */true, /* validate_settings */true); auto endpoint = getEndpoint(config, config_prefix, context); + auto settings = getSettings(config, config_prefix, context, endpoint, /* for_disk_s3 */true, /* validate_settings */true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 00a1216e52a..84af340e5d0 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -574,13 +574,13 @@ void S3ObjectStorage::applyNewSettings( ContextPtr context, const ApplyNewSettingsOptions & options) { - auto settings_from_config = getSettings(config, config_prefix, context, for_disk_s3, context->getSettingsRef().s3_validate_request_settings); + auto settings_from_config = getSettings(config, config_prefix, context, uri.endpoint, for_disk_s3, context->getSettingsRef().s3_validate_request_settings); auto modified_settings = std::make_unique(*s3_settings.get()); - modified_settings->auth_settings.updateFrom(settings_from_config->auth_settings); + modified_settings->auth_settings.updateIfChanged(settings_from_config->auth_settings); modified_settings->request_settings.updateIfChanged(settings_from_config->request_settings); if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString(), context->getUserName())) - modified_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); + modified_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); auto current_settings = s3_settings.get(); if (options.allow_client_change @@ -598,7 +598,7 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( const std::string & config_prefix, ContextPtr context) { - auto new_s3_settings = getSettings(config, config_prefix, context, for_disk_s3, true); + auto new_s3_settings = getSettings(config, config_prefix, context, uri.endpoint, for_disk_s3, true); auto new_client = getClient(uri, *new_s3_settings, context, for_disk_s3); auto new_uri{uri}; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index f66af556ce1..591b1e6623d 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -34,14 +34,18 @@ std::unique_ptr getSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, + const std::string & endpoint, bool for_disk_s3, bool validate_settings) { const auto & settings = context->getSettingsRef(); const std::string setting_name_prefix = for_disk_s3 ? "s3_" : ""; - auto auth_settings = S3::AuthSettings::loadFromConfig(config, config_prefix, settings); - auto request_settings = S3::RequestSettings::loadFromConfig(config, config_prefix, settings, validate_settings, setting_name_prefix); + auto auth_settings = S3::AuthSettings(config, config_prefix, settings); + auto request_settings = S3::RequestSettings(config, config_prefix, settings, validate_settings, setting_name_prefix); + + request_settings.proxy_resolver = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + ProxyConfiguration::protocolFromString(S3::URI(endpoint).uri.getScheme()), config_prefix, config); return std::make_unique( request_settings, @@ -75,7 +79,7 @@ std::unique_ptr getClient( const auto & request_settings = settings.request_settings; const bool is_s3_express_bucket = S3::isS3ExpressEndpoint(url.endpoint); - if (is_s3_express_bucket && auth_settings.region.empty()) + if (is_s3_express_bucket && auth_settings.region.value.empty()) { throw Exception( ErrorCodes::NO_ELEMENTS_IN_CONFIG, @@ -93,43 +97,36 @@ std::unique_ptr getClient( request_settings.put_request_throttler, url.uri.getScheme()); - client_configuration.connectTimeoutMs = auth_settings.connect_timeout_ms.value_or(S3::DEFAULT_CONNECT_TIMEOUT_MS); - client_configuration.requestTimeoutMs = auth_settings.request_timeout_ms.value_or(S3::DEFAULT_REQUEST_TIMEOUT_MS); - client_configuration.maxConnections = static_cast(auth_settings.max_connections.value_or(S3::DEFAULT_MAX_CONNECTIONS)); - client_configuration.http_keep_alive_timeout = auth_settings.http_keep_alive_timeout.value_or(S3::DEFAULT_KEEP_ALIVE_TIMEOUT); - client_configuration.http_keep_alive_max_requests = auth_settings.http_keep_alive_max_requests.value_or(S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS); + client_configuration.connectTimeoutMs = auth_settings.connect_timeout_ms; + client_configuration.requestTimeoutMs = auth_settings.request_timeout_ms; + client_configuration.maxConnections = static_cast(auth_settings.max_connections); + client_configuration.http_keep_alive_timeout = auth_settings.http_keep_alive_timeout; + client_configuration.http_keep_alive_max_requests = auth_settings.http_keep_alive_max_requests; client_configuration.endpointOverride = url.endpoint; - client_configuration.s3_use_adaptive_timeouts = auth_settings.use_adaptive_timeouts.value_or(S3::DEFAULT_USE_ADAPTIVE_TIMEOUTS); + client_configuration.s3_use_adaptive_timeouts = auth_settings.use_adaptive_timeouts; - if (for_disk_s3) + if (request_settings.proxy_resolver) { - /// TODO: move to S3Common auth settings parsing /* * Override proxy configuration for backwards compatibility with old configuration format. * */ - // if (auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( - // ProxyConfiguration::protocolFromString(url.uri.getScheme()), config_prefix, config)) - // { - // client_configuration.per_request_configuration - // = [proxy_config]() { return proxy_config->resolve(); }; - // client_configuration.error_report - // = [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); }; - // } + client_configuration.per_request_configuration = [=]() { return request_settings.proxy_resolver->resolve(); }; + client_configuration.error_report = [=](const auto & request_config) { request_settings.proxy_resolver->errorReport(request_config); }; } S3::ClientSettings client_settings{ .use_virtual_addressing = url.is_virtual_hosted_style, - .disable_checksum = auth_settings.disable_checksum.value_or(S3::DEFAULT_DISABLE_CHECKSUM), - .gcs_issue_compose_request = auth_settings.gcs_issue_compose_request.value_or(false), + .disable_checksum = auth_settings.disable_checksum, + .gcs_issue_compose_request = auth_settings.gcs_issue_compose_request, }; auto credentials_configuration = S3::CredentialsConfiguration { - auth_settings.use_environment_credentials.value_or(S3::DEFAULT_USE_ENVIRONMENT_CREDENTIALS), - auth_settings.use_insecure_imds_request.value_or(false), - auth_settings.expiration_window_seconds.value_or(S3::DEFAULT_EXPIRATION_WINDOW_SECONDS), - auth_settings.no_sign_request.value_or(S3::DEFAULT_NO_SIGN_REQUEST), + auth_settings.use_environment_credentials, + auth_settings.use_insecure_imds_request, + auth_settings.expiration_window_seconds, + auth_settings.no_sign_request, }; return S3::ClientFactory::instance().create( diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 5e7a18152d1..41aa85991a7 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -18,6 +18,7 @@ std::unique_ptr getSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, + const std::string & endpoint, bool for_disk_s3, bool validate_settings); diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index b3cd037e91d..ef42b4b2642 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -61,6 +61,7 @@ namespace DB namespace ErrorCodes { extern const int INVALID_CONFIG_PARAMETER; + extern const int BAD_ARGUMENTS; } namespace S3 @@ -101,146 +102,14 @@ ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, c return sse_kms_config; } -AuthSettings AuthSettings::loadFromConfig( +AuthSettings::AuthSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - const DB::Settings & settings, + const DB::Settings &, /// TODO: use settings const std::string & setting_name_prefix) { - auto auth_settings = AuthSettings::loadFromSettings(settings); - const std::string prefix = config_prefix + "." + setting_name_prefix; - auto has = [&](const std::string & key) -> bool { return config.has(prefix + key); }; - auto get_uint = [&](const std::string & key) -> size_t { return config.getUInt64(prefix + key); }; - auto get_bool = [&](const std::string & key) -> bool { return config.getBool(prefix + key); }; - auto get_string = [&](const std::string & key) -> std::string { return config.getString(prefix + key); }; - - if (has("access_key_id")) - auth_settings.access_key_id = get_string("access_key_id"); - if (has("secret_access_key")) - auth_settings.secret_access_key = get_string("secret_access_key"); - if (has("session_token")) - auth_settings.secret_access_key = get_string("session_token"); - - if (has("region")) - auth_settings.region = get_string("region"); - if (has("server_side_encryption_customer_key_base64")) - auth_settings.region = get_string("server_side_encryption_customer_key_base64"); - - if (has("connect_timeout_ms")) - auth_settings.connect_timeout_ms = get_uint("connect_timeout_ms"); - if (has("request_timeout_ms")) - auth_settings.request_timeout_ms = get_uint("request_timeout_ms"); - if (has("max_connections")) - auth_settings.max_connections = get_uint("max_connections"); - - if (has("http_keep_alive_timeout")) - auth_settings.http_keep_alive_timeout = get_uint("http_keep_alive_timeout"); - if (has("http_keep_alive_max_requests")) - auth_settings.http_keep_alive_max_requests = get_uint("http_keep_alive_max_requests"); - - if (has("use_environment_credentials")) - auth_settings.use_environment_credentials = get_bool("use_environment_credentials"); - if (has("use_adaptive_timeouts")) - auth_settings.use_adaptive_timeouts = get_bool("use_adaptive_timeouts"); - if (has("no_sing_request")) - auth_settings.no_sign_request = get_bool("no_sing_request"); - if (has("expiration_window_seconds")) - auth_settings.expiration_window_seconds = get_uint("expiration_window_seconds"); - if (has("gcs_issue_compose_request")) - auth_settings.gcs_issue_compose_request = get_bool("gcs_issue_compose_request"); - if (has("use_insecure_imds_request")) - auth_settings.use_insecure_imds_request = get_bool("use_insecure_imds_request"); - - auth_settings.headers = getHTTPHeaders(config_prefix, config); - auth_settings.server_side_encryption_kms_config = getSSEKMSConfig(config_prefix, config); - - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_prefix, keys); - for (const auto & key : keys) - { - if (startsWith(key, "user")) - auth_settings.users.insert(config.getString(config_prefix + "." + key)); - } - - return auth_settings; -} - -AuthSettings AuthSettings::loadFromSettings(const DB::Settings & settings) -{ - AuthSettings auth_settings{}; - auth_settings.updateFromSettings(settings, /* if_changed */false); - return auth_settings; -} - -void AuthSettings::updateFromSettings(const DB::Settings & settings, bool if_changed) -{ - if (!if_changed || settings.s3_connect_timeout_ms.changed) - connect_timeout_ms = settings.s3_connect_timeout_ms; - if (!if_changed || settings.s3_request_timeout_ms.changed) - request_timeout_ms = settings.s3_request_timeout_ms; - if (!if_changed || settings.s3_max_connections.changed) - max_connections = settings.s3_max_connections; - if (!if_changed || settings.s3_use_adaptive_timeouts.changed) - use_adaptive_timeouts = settings.s3_use_adaptive_timeouts; - if (!if_changed || settings.s3_disable_checksum.changed) - disable_checksum = settings.s3_disable_checksum; -} - -bool AuthSettings::hasUpdates(const AuthSettings & other) const -{ - AuthSettings copy = *this; - copy.updateFrom(other); - return *this != copy; -} - -void AuthSettings::updateFrom(const AuthSettings & from) -{ - /// Update with check for emptyness only parameters which - /// can be passed not only from config, but via ast. - - if (!from.access_key_id.empty()) - access_key_id = from.access_key_id; - if (!from.secret_access_key.empty()) - secret_access_key = from.secret_access_key; - if (!from.session_token.empty()) - session_token = from.session_token; - - if (!from.headers.empty()) - headers = from.headers; - if (!from.region.empty()) - region = from.region; - - server_side_encryption_customer_key_base64 = from.server_side_encryption_customer_key_base64; - server_side_encryption_kms_config = from.server_side_encryption_kms_config; - - if (from.use_environment_credentials.has_value()) - use_environment_credentials = from.use_environment_credentials; - - if (from.use_insecure_imds_request.has_value()) - use_insecure_imds_request = from.use_insecure_imds_request; - - if (from.expiration_window_seconds.has_value()) - expiration_window_seconds = from.expiration_window_seconds; - - if (from.no_sign_request.has_value()) - no_sign_request = from.no_sign_request; - - users.insert(from.users.begin(), from.users.end()); -} - -RequestSettings RequestSettings::loadFromConfig( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - const DB::Settings & settings, - bool validate_settings, - const std::string & setting_name_prefix) -{ - auto request_settings = RequestSettings::loadFromSettings(settings, validate_settings); - String prefix = config_prefix + "." + setting_name_prefix; - - auto values = request_settings.allMutable(); - for (auto & field : values) + for (auto & field : allMutable()) { const auto path = prefix + field.getName(); if (config.has(path)) @@ -257,22 +126,92 @@ RequestSettings RequestSettings::loadFromConfig( } } - if (!request_settings.storage_class_name.value.empty()) - request_settings.storage_class_name = Poco::toUpperInPlace(request_settings.storage_class_name.value); + headers = getHTTPHeaders(config_prefix, config); + server_side_encryption_kms_config = getSSEKMSConfig(config_prefix, config); - if (validate_settings) - request_settings.validateUploadSettings(); - - request_settings.initializeThrottler(settings); - - return request_settings; + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_prefix, keys); + for (const auto & key : keys) + { + if (startsWith(key, "user")) + users.insert(config.getString(config_prefix + "." + key)); + } } -RequestSettings RequestSettings::loadFromNamedCollection(const NamedCollection & collection, bool validate_settings) +AuthSettings::AuthSettings(const DB::Settings & settings) { - RequestSettings request_settings{}; + updateFromSettings(settings, /* if_changed */false); +} - auto values = request_settings.allMutable(); +void AuthSettings::updateFromSettings(const DB::Settings & settings, bool if_changed) +{ + for (auto & field : allMutable()) + { + const auto setting_name = "s3_" + field.getName(); + if (settings.has(setting_name) && (!if_changed || settings.isChanged(setting_name))) + { + set(field.getName(), settings.get(setting_name)); + } + } +} + +bool AuthSettings::hasUpdates(const AuthSettings & other) const +{ + AuthSettings copy = *this; + copy.updateIfChanged(other); + return *this != copy; +} + +void AuthSettings::updateIfChanged(const AuthSettings & settings) +{ + /// Update with check for emptyness only parameters which + /// can be passed not only from config, but via ast. + + for (auto & setting : settings.all()) + { + if (setting.isValueChanged()) + set(setting.getName(), setting.getValue()); + } + + if (!settings.headers.empty()) + headers = settings.headers; + server_side_encryption_kms_config = settings.server_side_encryption_kms_config; + users.insert(settings.users.begin(), settings.users.end()); +} + +RequestSettings::RequestSettings( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings, + bool validate_settings, + const std::string & setting_name_prefix) +{ + String prefix = config_prefix + "." + setting_name_prefix; + for (auto & field : allMutable()) + { + const auto path = prefix + field.getName(); + if (config.has(path)) + { + auto which = field.getValue().getType(); + if (isInt64OrUInt64FieldType(which)) + field.setValue(config.getUInt64(path)); + else if (which == Field::Types::String) + field.setValue(config.getString(path)); + else if (which == Field::Types::Bool) + field.setValue(config.getBool(path)); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); + } + } + finishInit(settings, validate_settings); +} + +RequestSettings::RequestSettings( + const NamedCollection & collection, + const DB::Settings & settings, + bool validate_settings) +{ + auto values = allMutable(); for (auto & field : values) { const auto path = field.getName(); @@ -289,26 +228,17 @@ RequestSettings RequestSettings::loadFromNamedCollection(const NamedCollection & throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); } } - - if (!request_settings.storage_class_name.value.empty()) - request_settings.storage_class_name = Poco::toUpperInPlace(request_settings.storage_class_name.value); - - if (validate_settings) - request_settings.validateUploadSettings(); - - // request_settings.initializeThrottler(settings); - - return request_settings; + finishInit(settings, validate_settings); } -RequestSettings RequestSettings::loadFromSettings(const DB::Settings & settings, bool validate_settings) +RequestSettings::RequestSettings(const DB::Settings & settings, bool validate_settings) { - RequestSettings request_settings{}; - request_settings.updateFromSettings(settings, /* if_changed */false, validate_settings); - return request_settings; + updateFromSettings(settings, /* if_changed */false, validate_settings); + finishInit(settings, validate_settings); } -void RequestSettings::updateFromSettings(const DB::Settings & settings, bool if_changed, bool validate_settings) +void RequestSettings::updateFromSettings( + const DB::Settings & settings, bool if_changed, bool /* validate_settings */) /// TODO: process validate_settings { for (auto & field : allMutable()) { @@ -318,12 +248,6 @@ void RequestSettings::updateFromSettings(const DB::Settings & settings, bool if_ set(field.getName(), settings.get(setting_name)); } } - - if (!storage_class_name.value.empty()) - storage_class_name = Poco::toUpperInPlace(storage_class_name.value); - - if (validate_settings) - validateUploadSettings(); } void RequestSettings::updateIfChanged(const RequestSettings & settings) @@ -335,8 +259,14 @@ void RequestSettings::updateIfChanged(const RequestSettings & settings) } } -void RequestSettings::initializeThrottler(const DB::Settings & settings) +void RequestSettings::finishInit(const DB::Settings & settings, bool validate_settings) { + if (!storage_class_name.value.empty() && storage_class_name.changed) + storage_class_name = Poco::toUpperInPlace(storage_class_name.value); + + if (validate_settings) + validateUploadSettings(); + /// NOTE: it would be better to reuse old throttlers /// to avoid losing token bucket state on every config reload, /// which could lead to exceeding limit for short time. @@ -443,6 +373,9 @@ void RequestSettings::validateUploadSettings() } +/// TODO: sometimes disk settings have fallback to "s3" section settings from config, support this. + +IMPLEMENT_SETTINGS_TRAITS(S3::AuthSettingsTraits, CLIENT_SETTINGS_LIST) IMPLEMENT_SETTINGS_TRAITS(S3::RequestSettingsTraits, REQUEST_SETTINGS_LIST) } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index b47e3e79409..b27b9ec1136 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -32,6 +32,7 @@ namespace ErrorCodes class RemoteHostFilter; class NamedCollection; +struct ProxyConfigurationResolver; class S3Exception : public Exception { @@ -72,64 +73,34 @@ namespace Poco::Util namespace DB::S3 { -HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); +#define AUTH_SETTINGS(M, ALIAS) \ + M(String, access_key_id, "", "", 0) \ + M(String, secret_access_key, "", "", 0) \ + M(String, session_token, "", "", 0) \ + M(String, region, "", "", 0) \ + M(String, server_side_encryption_customer_key_base64, "", "", 0) \ -ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); - -struct AuthSettings -{ - std::string access_key_id; - std::string secret_access_key; - std::string session_token; - std::string region; - std::string server_side_encryption_customer_key_base64; - - HTTPHeaderEntries headers; - std::unordered_set users; - ServerSideEncryptionKMSConfig server_side_encryption_kms_config; - - std::optional connect_timeout_ms; - std::optional request_timeout_ms; - std::optional max_connections; - std::optional http_keep_alive_timeout; - std::optional http_keep_alive_max_requests; - std::optional expiration_window_seconds; - - std::optional use_environment_credentials; - std::optional no_sign_request; - std::optional use_adaptive_timeouts; - std::optional use_insecure_imds_request; - std::optional is_virtual_hosted_style; - std::optional disable_checksum; - std::optional gcs_issue_compose_request; - - bool hasUpdates(const AuthSettings & other) const; - void updateFrom(const AuthSettings & from); - - bool canBeUsedByUser(const String & user) const { return users.empty() || users.contains(user); } - - static AuthSettings loadFromConfig( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - const DB::Settings & settings, - const std::string & setting_name_prefix = ""); - - static AuthSettings loadFromSettings(const DB::Settings & settings); - - static AuthSettings loadFromNamedCollection(const NamedCollection & collection); - - void updateFromSettings(const DB::Settings & settings, bool if_changed); - -private: - bool operator==(const AuthSettings & other) const = default; -}; +#define CLIENT_SETTINGS(M, ALIAS) \ + M(UInt64, connect_timeout_ms, DEFAULT_CONNECT_TIMEOUT_MS, "", 0) \ + M(UInt64, request_timeout_ms, DEFAULT_REQUEST_TIMEOUT_MS, "", 0) \ + M(UInt64, max_connections, DEFAULT_MAX_CONNECTIONS, "", 0) \ + M(UInt64, http_keep_alive_timeout, DEFAULT_KEEP_ALIVE_TIMEOUT, "", 0) \ + M(UInt64, http_keep_alive_max_requests, DEFAULT_KEEP_ALIVE_MAX_REQUESTS, "", 0) \ + M(UInt64, expiration_window_seconds, DEFAULT_EXPIRATION_WINDOW_SECONDS, "", 0) \ + M(Bool, use_environment_credentials, DEFAULT_USE_ENVIRONMENT_CREDENTIALS, "", 0) \ + M(Bool, no_sign_request, DEFAULT_NO_SIGN_REQUEST, "", 0) \ + M(Bool, use_insecure_imds_request, false, "", 0) \ + M(Bool, use_adaptive_timeouts, DEFAULT_USE_ADAPTIVE_TIMEOUTS, "", 0) \ + M(Bool, is_virtual_hosted_style, false, "", 0) \ + M(Bool, disable_checksum, DEFAULT_DISABLE_CHECKSUM, "", 0) \ + M(Bool, gcs_issue_compose_request, false, "", 0) \ #define REQUEST_SETTINGS(M, ALIAS) \ M(UInt64, max_single_read_retries, 4, "", 0) \ M(UInt64, request_timeout_ms, DEFAULT_REQUEST_TIMEOUT_MS, "", 0) \ - M(UInt64, list_object_keys_size, 1000, "", 0) \ - M(Bool, allow_native_copy, true, "", 0) \ - M(Bool, check_objects_after_upload, false, "", 0) \ + M(UInt64, list_object_keys_size, DEFAULT_LIST_OBJECT_KEYS_SIZE, "", 0) \ + M(Bool, allow_native_copy, DEFAULT_ALLOW_NATIVE_COPY, "", 0) \ + M(Bool, check_objects_after_upload, DEFAULT_CHECK_OBJECTS_AFTER_UPLOAD, "", 0) \ M(Bool, throw_on_zero_files_match, false, "", 0) \ M(UInt64, max_single_operation_copy_size, DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE, "", 0) \ M(String, storage_class_name, "", "", 0) \ @@ -145,23 +116,56 @@ private: M(UInt64, max_single_part_upload_size, DEFAULT_MAX_SINGLE_PART_UPLOAD_SIZE, "", 0) \ M(UInt64, max_unexpected_write_error_retries, 4, "", 0) \ +#define CLIENT_SETTINGS_LIST(M, ALIAS) \ + CLIENT_SETTINGS(M, ALIAS) \ + AUTH_SETTINGS(M, ALIAS) #define REQUEST_SETTINGS_LIST(M, ALIAS) \ REQUEST_SETTINGS(M, ALIAS) \ PART_UPLOAD_SETTINGS(M, ALIAS) +DECLARE_SETTINGS_TRAITS(AuthSettingsTraits, CLIENT_SETTINGS_LIST) DECLARE_SETTINGS_TRAITS(RequestSettingsTraits, REQUEST_SETTINGS_LIST) +struct AuthSettings : public BaseSettings +{ + AuthSettings() = default; + + AuthSettings( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings, + const std::string & setting_name_prefix = ""); + + AuthSettings(const DB::Settings & settings); + + AuthSettings(const NamedCollection & collection); + + void updateFromSettings(const DB::Settings & settings, bool if_changed); + bool hasUpdates(const AuthSettings & other) const; + void updateIfChanged(const AuthSettings & settings); + bool canBeUsedByUser(const String & user) const { return users.empty() || users.contains(user); } + + HTTPHeaderEntries headers; + std::unordered_set users; + ServerSideEncryptionKMSConfig server_side_encryption_kms_config; +}; + struct RequestSettings : public BaseSettings { - void validateUploadSettings(); + RequestSettings() = default; - ThrottlerPtr get_request_throttler; - ThrottlerPtr put_request_throttler; + /// Create request settings from DB::Settings. + explicit RequestSettings(const DB::Settings & settings, bool validate_settings = true); - static RequestSettings loadFromSettings(const DB::Settings & settings, bool validate_settings = true); - static RequestSettings loadFromNamedCollection(const NamedCollection & collection, bool validate_settings = true); - static RequestSettings loadFromConfig( + /// Create request settings from NamedCollection. + RequestSettings( + const NamedCollection & collection, + const DB::Settings & settings, + bool validate_settings = true); + + /// Create request settings from Config. + RequestSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DB::Settings & settings, @@ -170,9 +174,18 @@ struct RequestSettings : public BaseSettings void updateFromSettings(const DB::Settings & settings, bool if_changed, bool validate_settings = true); void updateIfChanged(const RequestSettings & settings); + void validateUploadSettings(); + + ThrottlerPtr get_request_throttler; + ThrottlerPtr put_request_throttler; + std::shared_ptr proxy_resolver; private: - void initializeThrottler(const DB::Settings & settings); + void finishInit(const DB::Settings & settings, bool validate_settings); }; +HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); + +ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); + } diff --git a/src/IO/S3Defines.h b/src/IO/S3Defines.h index ad01920adda..eedd0df81a6 100644 --- a/src/IO/S3Defines.h +++ b/src/IO/S3Defines.h @@ -28,5 +28,8 @@ inline static constexpr uint64_t DEFAULT_MAX_PART_NUMBER = 10000; /// Other settings. inline static constexpr uint64_t DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE = 32 * 1024 * 1024; inline static constexpr uint64_t DEFAULT_MAX_INFLIGHT_PARTS_FOR_ONE_FILE = 20; +inline static constexpr uint64_t DEFAULT_LIST_OBJECT_KEYS_SIZE = 1000; +inline static constexpr uint64_t DEFAULT_ALLOW_NATIVE_COPY = true; +inline static constexpr uint64_t DEFAULT_CHECK_OBJECTS_AFTER_UPLOAD = false; } diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp index 6a7b2ea5627..4197014c454 100644 --- a/src/IO/S3Settings.cpp +++ b/src/IO/S3Settings.cpp @@ -32,8 +32,8 @@ void S3SettingsByEndpoint::loadFromConfig( if (config.has(endpoint_path)) { auto endpoint = config.getString(endpoint_path); - auto auth_settings = S3::AuthSettings::loadFromConfig(config, key_path, settings); - auto request_settings = S3::RequestSettings::loadFromConfig(config, key_path, settings); + auto auth_settings = S3::AuthSettings(config, key_path, settings); + auto request_settings = S3::RequestSettings(config, key_path, settings); s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); } } diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index ada3e2e9323..163f08be420 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -249,7 +249,7 @@ AzureClientPtr StorageAzureConfiguration::createClient(bool is_read_only, bool a return result; } -void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & collection) + void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) { validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h index 35b19079ca9..bbaa82c51ba 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -51,7 +51,7 @@ public: ContextPtr context) override; protected: - void fromNamedCollection(const NamedCollection & collection) override; + void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index a8a9ab5b557..155f51adf61 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -119,7 +119,7 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool wit setURL(url_str); } -void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection & collection) +void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) { std::string url_str; diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 01a8b9c5e3b..04884542908 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -46,7 +46,7 @@ public: ContextPtr context) override; private: - void fromNamedCollection(const NamedCollection &) override; + void fromNamedCollection(const NamedCollection &, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; void setURL(const std::string & url_); diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 537af4421f2..327efba2169 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -108,10 +108,12 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, const auto & settings = context->getSettingsRef(); const std::string config_prefix = "s3."; - auto s3_settings = getSettings(config, config_prefix, context, /* for_disk_s3 */false, settings.s3_validate_request_settings); + auto s3_settings = getSettings( + config, config_prefix, context, url.endpoint, /* for_disk_s3 */false, + settings.s3_validate_request_settings); request_settings.updateFromSettings(settings, /* if_changed */true); - auth_settings.updateFrom(s3_settings->auth_settings); + auth_settings.updateIfChanged(s3_settings->auth_settings); s3_settings->auth_settings = auth_settings; s3_settings->request_settings = request_settings; @@ -124,7 +126,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, } if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) - s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); + s3_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); auto client = getClient(url, *s3_settings, context, /* for_disk_s3 */false); auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); @@ -139,8 +141,9 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, key_generator, "StorageS3", false); } -void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection) +void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection, ContextPtr context) { + const auto settings = context->getSettingsRef(); validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); auto filename = collection.getOrDefault("filename", ""); @@ -159,9 +162,9 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); structure = collection.getOrDefault("structure", "auto"); - request_settings = S3::RequestSettings::loadFromNamedCollection(collection, /* validate_settings */true); + request_settings = S3::RequestSettings(collection, settings, /* validate_settings */true); - static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); + static_configuration = !auth_settings.access_key_id.value.empty() || auth_settings.no_sign_request.changed; keys = {url.key}; } @@ -357,7 +360,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ if (no_sign_request) auth_settings.no_sign_request = no_sign_request; - static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); + static_configuration = !auth_settings.access_key_id.value.empty() || auth_settings.no_sign_request.changed; auth_settings.no_sign_request = no_sign_request; keys = {url.key}; diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 5a952497851..39a646c7df2 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -51,7 +51,7 @@ public: ContextPtr context) override; private: - void fromNamedCollection(const NamedCollection & collection) override; + void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; S3::URI url; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 2c8e60b49d0..90a97a9ea62 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -424,7 +424,7 @@ void StorageObjectStorage::Configuration::initialize( bool with_table_structure) { if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - configuration.fromNamedCollection(*named_collection); + configuration.fromNamedCollection(*named_collection, local_context); else configuration.fromAST(engine_args, local_context, with_table_structure); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index f45d8c1f01a..cf8ec113653 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -193,7 +193,7 @@ public: String structure = "auto"; protected: - virtual void fromNamedCollection(const NamedCollection & collection) = 0; + virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; void assertInitialized() const; From d0d5b6d0cbbc9841f6bce59ef2feee4cc00b1b1f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 27 May 2024 16:30:46 +0000 Subject: [PATCH 153/856] 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 54735e6292ebbce528a4a0681d294ac56c71cbb5 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 27 May 2024 17:52:09 +0000 Subject: [PATCH 154/856] fix --- src/Common/ThreadStatus.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 04fb568540b..49594116b91 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -187,8 +187,8 @@ public: /// For this purpose we dynamically change `untracked_memory_limit` after every tracking event using a simple formula: /// untracked_memory_limit = clamp(untracked_memory_ratio * cur_memory_bytes, min_untracked_memory, max_untracked_memory) /// Note that this values are updated when thread is attached to a group - Int64 min_untracked_memory = 4 * 1024 * 1024; - Int64 max_untracked_memory = 4 * 1024; + Int64 min_untracked_memory = 4 * 1024; + Int64 max_untracked_memory = 4 * 1024 * 1024; /// Statistics of read and write rows/bytes Progress progress_in; From 78e161ff15b5399aa18141b5cf896353a2fc9e00 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 27 May 2024 19:02:17 +0000 Subject: [PATCH 155/856] 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 c973addee64c4dba156ad6ea741afdf97e8a46cd Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 27 May 2024 19:13:56 +0000 Subject: [PATCH 156/856] disable precise memory tracking for some tests --- tests/integration/test_settings_constraints_distributed/test.py | 2 +- .../0_stateless/03030_system_flush_distributed_settings.sql | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index fbebbac276e..295347192bd 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -136,7 +136,7 @@ def test_select_clamps_settings(): ) assert ( - distributed.query(query, settings={"max_memory_usage": 1}) + distributed.query(query, settings={"max_memory_usage": 1, "min_untracked_memory": 4194304}) == "node1\tmax_memory_usage\t11111111\n" "node1\treadonly\t0\n" "node2\tmax_memory_usage\t0\n" diff --git a/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql b/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql index da2a387e07c..e8a3da174a6 100644 --- a/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql +++ b/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql @@ -13,6 +13,8 @@ create table dist_out as data engine=Distributed(test_shard_localhost, currentDa set prefer_localhost_replica=0; +set min_untracked_memory='4Mi' -- Disable precise memory tracking + insert into dist_in select number/100, number from system.numbers limit 1e6 settings max_memory_usage='20Mi'; system flush distributed dist_in; -- { serverError MEMORY_LIMIT_EXCEEDED } system flush distributed dist_in settings max_memory_usage=0; From 18dce4169f1b3a3692f4975fb688a3b137b547c4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 27 May 2024 19:22:24 +0000 Subject: [PATCH 157/856] Automatic style fix --- .../integration/test_settings_constraints_distributed/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index 295347192bd..a1f44af1069 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -136,7 +136,9 @@ def test_select_clamps_settings(): ) assert ( - distributed.query(query, settings={"max_memory_usage": 1, "min_untracked_memory": 4194304}) + distributed.query( + query, settings={"max_memory_usage": 1, "min_untracked_memory": 4194304} + ) == "node1\tmax_memory_usage\t11111111\n" "node1\treadonly\t0\n" "node2\tmax_memory_usage\t0\n" From f225649332d022ec5f8572d994038c01aee6f1ab Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 27 May 2024 21:09:11 +0000 Subject: [PATCH 158/856] calculate skip indexes on vertical merge --- src/Storages/MergeTree/MergeTask.cpp | 217 ++++++++++++++++----------- src/Storages/MergeTree/MergeTask.h | 10 +- 2 files changed, 141 insertions(+), 86 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e43b6c615b3..bfe2f4673db 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -5,8 +5,15 @@ #include #include +#include "Common/DateLUT.h" #include #include +#include "Core/NamesAndTypes.h" +#include "Storages/ColumnsDescription.h" +#include "Storages/IndicesDescription.h" +#include "Storages/MergeTree/MergeTreeIndices.h" +#include "Storages/ProjectionsDescription.h" +#include "Storages/StorageInMemoryMetadata.h" #include #include #include @@ -48,59 +55,37 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } - -/// PK columns are sorted and merged, ordinary columns are gathered using info from merge step -static void extractMergingAndGatheringColumns( - const NamesAndTypesList & storage_columns, - const ExpressionActionsPtr & sorting_key_expr, - const IndicesDescription & indexes, - const MergeTreeData::MergingParams & merging_params, - NamesAndTypesList & gathering_columns, Names & gathering_column_names, - NamesAndTypesList & merging_columns, Names & merging_column_names) +static Statistics getStatisticsForColumns( + const NamesAndTypesList & columns_to_read, + const StorageMetadataPtr & metadata_snapshot) { - Names sort_key_columns_vec = sorting_key_expr->getRequiredColumns(); - std::set key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend()); - for (const auto & index : indexes) + Statistics statistics; + const auto & all_columns = metadata_snapshot->getColumns(); + + for (const auto & column : columns_to_read) { - Names index_columns_vec = index.expression->getRequiredColumns(); - std::copy(index_columns_vec.cbegin(), index_columns_vec.cend(), - std::inserter(key_columns, key_columns.end())); - } - - /// Force sign column for Collapsing mode - if (merging_params.mode == MergeTreeData::MergingParams::Collapsing) - key_columns.emplace(merging_params.sign_column); - - /// Force version column for Replacing mode - if (merging_params.mode == MergeTreeData::MergingParams::Replacing) - { - key_columns.emplace(merging_params.is_deleted_column); - key_columns.emplace(merging_params.version_column); - } - - /// Force sign column for VersionedCollapsing mode. Version is already in primary key. - if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) - key_columns.emplace(merging_params.sign_column); - - /// Force to merge at least one column in case of empty key - if (key_columns.empty()) - key_columns.emplace(storage_columns.front().name); - - /// TODO: also force "summing" and "aggregating" columns to make Horizontal merge only for such columns - - for (const auto & column : storage_columns) - { - if (key_columns.contains(column.name)) + const auto & desc = all_columns.get(column.name); + if (desc.stat) { - merging_columns.emplace_back(column); - merging_column_names.emplace_back(column.name); - } - else - { - gathering_columns.emplace_back(column); - gathering_column_names.emplace_back(column.name); + auto statistic = MergeTreeStatisticsFactory::instance().get(*desc.stat); + statistics.push_back(std::move(statistic)); } } + return statistics; +} + +static void addSkipIndexesExpressions( + QueryPipelineBuilder & builder, + const IndicesDescription & indexes, + const StorageMetadataPtr & metadata_snapshot, + const ContextPtr & context) +{ + builder.addTransform(std::make_shared( + builder.getHeader(), + indexes.getSingleExpressionForIndices(metadata_snapshot->getColumns(), + context))); + + builder.addTransform(std::make_shared(builder.getHeader())); } static void addMissedColumnsToSerializationInfos( @@ -129,6 +114,76 @@ static void addMissedColumnsToSerializationInfos( } } +/// PK columns are sorted and merged, ordinary columns are gathered using info from merge step +void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColumns() +{ + const auto & sorting_key_expr = global_ctx->metadata_snapshot->getSortingKey().expression; + Names sort_key_columns_vec = sorting_key_expr->getRequiredColumns(); + + std::set key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend()); + + /// Force sign column for Collapsing mode + if (ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing) + key_columns.emplace(ctx->merging_params.sign_column); + + /// Force version column for Replacing mode + if (ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing) + { + key_columns.emplace(ctx->merging_params.is_deleted_column); + key_columns.emplace(ctx->merging_params.version_column); + } + + /// Force sign column for VersionedCollapsing mode. Version is already in primary key. + if (ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) + key_columns.emplace(ctx->merging_params.sign_column); + + /// Force to merge at least one column in case of empty key + if (key_columns.empty()) + key_columns.emplace(global_ctx->storage_columns.front().name); + + const auto & skip_indexes = global_ctx->metadata_snapshot->getSecondaryIndices(); + + for (const auto & index : skip_indexes) + { + auto index_columns = index.expression->getRequiredColumns(); + + if (index_columns.size() == 1) + { + const auto & column_name = index_columns.front(); + global_ctx->skip_indexes_by_column[column_name].push_back(index); + } + else + { + std::ranges::copy(index_columns, std::inserter(key_columns, key_columns.end())); + global_ctx->merging_skip_indexes.push_back(index); + } + } + + /// TODO: also force "summing" and "aggregating" columns to make Horizontal merge only for such columns + + for (const auto & column : global_ctx->storage_columns) + { + if (key_columns.contains(column.name)) + { + global_ctx->merging_columns.emplace_back(column); + global_ctx->merging_column_names.emplace_back(column.name); + + auto it = global_ctx->skip_indexes_by_column.find(column.name); + if (it != global_ctx->skip_indexes_by_column.end()) + { + for (auto && index : it->second) + global_ctx->merging_skip_indexes.push_back(std::move(index)); + + global_ctx->skip_indexes_by_column.erase(it); + } + } + else + { + global_ctx->gathering_columns.emplace_back(column); + global_ctx->gathering_column_names.emplace_back(column.name); + } + } +} bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() { @@ -204,19 +259,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() extendObjectColumns(global_ctx->storage_columns, object_columns, false); global_ctx->storage_snapshot = std::make_shared(*global_ctx->data, global_ctx->metadata_snapshot, std::move(object_columns)); - extractMergingAndGatheringColumns( - global_ctx->storage_columns, - global_ctx->metadata_snapshot->getSortingKey().expression, - global_ctx->metadata_snapshot->getSecondaryIndices(), - ctx->merging_params, - global_ctx->gathering_columns, - global_ctx->gathering_column_names, - global_ctx->merging_columns, - global_ctx->merging_column_names); - global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; global_ctx->new_data_part->partition.assign(global_ctx->future_part->getPartition()); global_ctx->new_data_part->is_temp = global_ctx->parent_part == nullptr; + /// In case of replicated merge tree with zero copy replication /// Here Clickhouse claims that this new part can be deleted in temporary state without unlocking the blobs /// The blobs have to be removed along with the part, this temporary part owns them and does not share them yet. @@ -226,10 +272,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->force_ttl = false; if (enabledBlockNumberColumn(global_ctx)) - addGatheringColumn(global_ctx, BlockNumberColumn::name, BlockNumberColumn::type); + addStorageColumn(global_ctx, BlockNumberColumn::name, BlockNumberColumn::type); if (enabledBlockOffsetColumn(global_ctx)) - addGatheringColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); + addStorageColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); SerializationInfo::Settings info_settings = { @@ -299,17 +345,18 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() { global_ctx->merging_columns = global_ctx->storage_columns; global_ctx->merging_column_names = global_ctx->all_column_names; - global_ctx->gathering_columns.clear(); - global_ctx->gathering_column_names.clear(); + global_ctx->merging_skip_indexes = global_ctx->metadata_snapshot->getSecondaryIndices(); break; } case MergeAlgorithm::Vertical: { + extractMergingAndGatheringColumns(); + ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->createRawStream(); ctx->rows_sources_write_buf = std::make_unique(*ctx->rows_sources_uncompressed_write_buf); std::map local_merged_column_to_size; - for (const MergeTreeData::DataPartPtr & part : global_ctx->future_part->parts) + for (const auto & part : global_ctx->future_part->parts) part->accumulateColumnSizes(local_merged_column_to_size); ctx->column_sizes = ColumnSizeEstimator( @@ -376,8 +423,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->new_data_part, global_ctx->metadata_snapshot, global_ctx->merging_columns, - MergeTreeIndexFactory::instance().getMany(global_ctx->metadata_snapshot->getSecondaryIndices()), - MergeTreeStatisticsFactory::instance().getMany(global_ctx->metadata_snapshot->getColumns()), + MergeTreeIndexFactory::instance().getMany(global_ctx->merging_skip_indexes), + getStatisticsForColumns(global_ctx->merging_columns, global_ctx->metadata_snapshot), ctx->compression_codec, global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, @@ -401,7 +448,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() return false; } -void MergeTask::addGatheringColumn(GlobalRuntimeContextPtr global_ctx, const String & name, const DataTypePtr & type) +void MergeTask::addStorageColumn(GlobalRuntimeContextPtr global_ctx, const String & name, const DataTypePtr & type) { if (global_ctx->storage_columns.contains(name)) return; @@ -575,7 +622,6 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const Names column_names{column_name}; ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); - global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); Pipes pipes; @@ -598,7 +644,6 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const } auto pipe = Pipe::unitePipes(std::move(pipes)); - ctx->rows_sources_read_buf->seek(0, 0); const auto data_settings = global_ctx->data->getSettings(); @@ -609,9 +654,20 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const data_settings->merge_max_block_size, data_settings->merge_max_block_size_bytes); - pipe.addTransform(std::move(transform)); + QueryPipelineBuilder builder; + builder.init(std::move(pipe)); + builder.addTransform(std::move(transform)); - ctx->column_parts_pipeline = QueryPipeline(std::move(pipe)); + MergeTreeIndices indexes_to_recalc; + auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name); + + if (indexes_it != global_ctx->skip_indexes_by_column.end()) + { + indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); + addSkipIndexesExpressions(builder, indexes_it->second, global_ctx->metadata_snapshot, global_ctx->data->getContext()); + } + + ctx->column_parts_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); /// Dereference unique_ptr ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback( @@ -621,7 +677,6 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const /// Is calculated inside MergeProgressCallback. ctx->column_parts_pipeline.disableProfileEventUpdate(); - ctx->executor = std::make_unique(ctx->column_parts_pipeline); ctx->column_to = std::make_unique( @@ -629,11 +684,8 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const global_ctx->metadata_snapshot, ctx->executor->getHeader(), ctx->compression_codec, - /// we don't need to recalc indices here - /// because all of them were already recalculated and written - /// as key part of vertical merge - std::vector{}, - std::vector{}, /// TODO: think about it + indexes_to_recalc, + getStatisticsForColumns({*ctx->it_name_and_type}, global_ctx->metadata_snapshot), &global_ctx->written_offset_columns, global_ctx->to->getIndexGranularity()); @@ -1117,13 +1169,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() builder->addTransform(std::move(transform)); } - if (global_ctx->metadata_snapshot->hasSecondaryIndices()) - { - const auto & indices = global_ctx->metadata_snapshot->getSecondaryIndices(); - builder->addTransform(std::make_shared( - builder->getHeader(), indices.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext()))); - builder->addTransform(std::make_shared(builder->getHeader())); - } + if (!global_ctx->merging_skip_indexes.empty()) + addSkipIndexesExpressions(*builder, global_ctx->merging_skip_indexes, global_ctx->metadata_snapshot, global_ctx->data->getContext()); if (!subqueries.empty()) builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), global_ctx->context); @@ -1172,7 +1219,7 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; - bool enough_ordinary_cols = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate; + bool enough_columns = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate; bool enough_total_rows = total_rows_count >= data_settings->vertical_merge_algorithm_min_rows_to_activate; @@ -1180,7 +1227,7 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm bool no_parts_overflow = global_ctx->future_part->parts.size() <= RowSourcePart::MAX_PARTS; - auto merge_alg = (is_supported_storage && enough_total_rows && enough_total_bytes && enough_ordinary_cols && no_parts_overflow) ? + auto merge_alg = (is_supported_storage && enough_total_rows && enough_total_bytes && enough_columns && no_parts_overflow) ? MergeAlgorithm::Vertical : MergeAlgorithm::Horizontal; return merge_alg; diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index c8b0662e3eb..ae7e13dd244 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -2,8 +2,11 @@ #include #include +#include #include +#include "Storages/MergeTree/MergeTreeIndices.h" +#include "Storages/Statistics/Statistics.h" #include #include @@ -170,6 +173,9 @@ private: Names all_column_names{}; MergeTreeData::DataPart::Checksums checksums_gathered_columns{}; + IndicesDescription merging_skip_indexes; + std::unordered_map skip_indexes_by_column; + MergeAlgorithm chosen_merge_algorithm{MergeAlgorithm::Undecided}; size_t gathering_column_names_size{0}; @@ -260,12 +266,14 @@ private: MergeAlgorithm chooseMergeAlgorithm() const; void createMergedStream(); + void extractMergingAndGatheringColumns(); void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) override { ctx = static_pointer_cast(local); global_ctx = static_pointer_cast(global); } + StageRuntimeContextPtr getContextForNextStage() override; ExecuteAndFinalizeHorizontalPartRuntimeContextPtr ctx; @@ -414,7 +422,7 @@ private: return global_ctx->data->getSettings()->enable_block_offset_column && global_ctx->metadata_snapshot->getGroupByTTLs().empty(); } - static void addGatheringColumn(GlobalRuntimeContextPtr global_ctx, const String & name, const DataTypePtr & type); + static void addStorageColumn(GlobalRuntimeContextPtr global_ctx, const String & name, const DataTypePtr & type); }; /// FIXME From b4c2fa7e272b6ecda42ceeb6bc613ce7277382e2 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 28 May 2024 15:17:08 +0800 Subject: [PATCH 159/856] add test case --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 4 ++-- src/Core/SettingsQuirks.cpp | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../Impl/Parquet/ParquetRecordReader.cpp | 4 ++-- .../Impl/Parquet/ParquetRecordReader.h | 2 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 11 +++++----- ...pting_parquet_reader_output_size.reference | 4 ++++ ...64_adapting_parquet_reader_output_size.sql | 21 +++++++++++++++++++ 10 files changed, 40 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference create mode 100644 tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c9efd1e4a97..011541088ac 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1053,7 +1053,7 @@ class IColumn; M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ - M(UInt64, input_format_parquet_max_block_size, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader.", 0) \ + M(UInt64, input_format_parquet_max_block_rows, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader.", 0) \ M(UInt64, input_format_parquet_prefer_block_bytes, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader", 0) \ M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format Protobuf", 0) \ M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format CapnProto", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index be031592c12..ab83da5de8d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,8 +92,8 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, + {"input_format_parquet_max_block_rows", 8192, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader."}, + {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 5541cc19653..4065ee40285 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -117,7 +117,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) "min_insert_block_size_bytes_for_materialized_views", "min_external_table_block_size_rows", "max_joined_block_size_rows", - "input_format_parquet_max_block_size"}; + "input_format_parquet_max_block_rows"}; for (auto const & setting : block_rows_settings) { if (auto block_size = get_current_value(setting).get(); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index e90986f2236..a01be503c4f 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -160,7 +160,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference; format_settings.parquet.output_string_as_string = settings.output_format_parquet_string_as_string; format_settings.parquet.output_fixed_string_as_fixed_byte_array = settings.output_format_parquet_fixed_string_as_fixed_byte_array; - format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; + format_settings.parquet.max_block_rows = settings.input_format_parquet_max_block_rows; format_settings.parquet.prefer_block_bytes = settings.input_format_parquet_prefer_block_bytes; format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; format_settings.parquet.output_compliant_nested_types = settings.output_format_parquet_compliant_nested_types; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 337aafbbe9c..f7b57ddd4aa 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -265,7 +265,7 @@ struct FormatSettings bool preserve_order = false; bool use_custom_encoder = true; bool parallel_encoding = true; - UInt64 max_block_size = DEFAULT_BLOCK_SIZE; + UInt64 max_block_rows = DEFAULT_BLOCK_SIZE; size_t prefer_block_bytes = DEFAULT_BLOCK_SIZE * 256; ParquetVersion output_version; ParquetCompression output_compression_method = ParquetCompression::SNAPPY; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index a7e51f88b3c..ad98db3b8ab 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -307,7 +307,7 @@ ParquetRecordReader::ParquetRecordReader( : file_reader(createFileReader(std::move(arrow_file), std::move(metadata))) , reader_properties(reader_properties_) , header(std::move(header_)) - , max_block_size(format_settings.parquet.max_block_size) + , max_block_rows(format_settings.parquet.max_block_rows) , row_groups_indices(std::move(row_groups_indices_)) , left_rows(getTotalRows(*file_reader->metadata())) { @@ -356,7 +356,7 @@ Chunk ParquetRecordReader::readChunk() } Columns columns(header.columns()); - auto num_rows_read = std::min(max_block_size, cur_row_group_left_rows); + auto num_rows_read = std::min(max_block_rows, cur_row_group_left_rows); for (size_t i = 0; i < header.columns(); i++) { columns[i] = castColumn( diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h index 2f728a586a0..a682d724960 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h @@ -36,7 +36,7 @@ private: std::shared_ptr cur_row_group_reader; ParquetColReaders column_readers; - UInt64 max_block_size; + UInt64 max_block_rows; std::vector parquet_col_indice; std::vector row_groups_indices; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 95da938f4e6..008b7b41b57 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -424,13 +424,14 @@ void ParquetBlockInputFormat::initializeIfNeeded() auto row_group_meta = metadata->RowGroup(row_group_idx); for (int column_index : column_indices) { + auto column = row_group_meta->ColumnChunk(column_index); total_size += row_group_meta->ColumnChunk(column_index)->total_uncompressed_size(); } if (!total_size || !format_settings.parquet.prefer_block_bytes) return 0; - auto average_row_bytes = total_size / row_group_meta->num_rows(); - /// max_block_size >= num_rows >= 128 - auto num_rows = std::min(format_settings.parquet.prefer_block_bytes/average_row_bytes, format_settings.parquet.max_block_size); - return std::max(num_rows, 128UL); + auto average_row_bytes = static_cast(total_size) / row_group_meta->num_rows(); + const size_t preferred_num_rows = static_cast(format_settings.parquet.prefer_block_bytes/average_row_bytes); + const size_t MIN_ROW_NUM = 128; + return std::min(std::max(preferred_num_rows, MIN_ROW_NUM), format_settings.parquet.max_block_rows); }; for (int row_group = 0; row_group < num_row_groups; ++row_group) @@ -453,7 +454,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() row_group_batches.back().total_rows += metadata->RowGroup(row_group)->num_rows(); row_group_batches.back().total_bytes_compressed += metadata->RowGroup(row_group)->total_compressed_size(); auto rows = adative_chunk_size(row_group); - row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_size; + row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_rows; } } diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference new file mode 100644 index 00000000000..332202dd23b --- /dev/null +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference @@ -0,0 +1,4 @@ +65409 +16 +128 +2183 diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql new file mode 100644 index 00000000000..25fe4695e25 --- /dev/null +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet); +INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); +SELECT max(blockSize()) FROM test_parquet; + +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_max_block_rows=16; +INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); +SELECT max(blockSize()) FROM test_parquet; + +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_prefer_block_bytes=30; +INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); +SELECT max(blockSize()) FROM test_parquet; + +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_prefer_block_bytes=30000; +INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); +SELECT max(blockSize()) FROM test_parquet; + +DROP TABLE IF EXISTS test_parquet; \ No newline at end of file From dfc2a04d26e782ea0ac53af0b03520c77192ebb9 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 28 May 2024 16:29:35 +0800 Subject: [PATCH 160/856] add no fasttest tag --- .../0_stateless/03164_adapting_parquet_reader_output_size.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql index 25fe4695e25..9e57f2dd733 100644 --- a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest, no-parallel + DROP TABLE IF EXISTS test_parquet; CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet); INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); From 673f6c981809a6287be939bd50930ca828cece1a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 May 2024 11:19:05 +0000 Subject: [PATCH 161/856] Add env variable for GWPAsan --- src/Common/Allocator.cpp | 8 +++--- src/Common/GWPAsan.cpp | 62 +++++++++++++++++++++++++++++----------- 2 files changed, 50 insertions(+), 20 deletions(-) diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 67ef98cf221..87075a8c709 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -1,9 +1,9 @@ #include -#include -#include -#include -#include #include +#include +#include +#include +#include #include #include diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 6f57af9e982..a46b7d640e4 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -1,36 +1,67 @@ -#include #include #if USE_GWP_ASAN +# include # include # include # include # include +# include +# include # include # include # include +# include +# include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +} + namespace Memory { namespace { - size_t getBackTrace(uintptr_t * trace_buffer, size_t buffer_size) - { - StackTrace stacktrace; - auto trace_size = std::min(buffer_size, stacktrace.getSize()); - const auto & frame_pointers = stacktrace.getFramePointers(); - memcpy(trace_buffer, frame_pointers.data(), std::min(trace_size, buffer_size) * sizeof(uintptr_t)); - return trace_size; - } +size_t getBackTrace(uintptr_t * trace_buffer, size_t buffer_size) +{ + StackTrace stacktrace; + auto trace_size = std::min(buffer_size, stacktrace.getSize()); + const auto & frame_pointers = stacktrace.getFramePointers(); + memcpy(trace_buffer, frame_pointers.data(), trace_size * sizeof(uintptr_t)); + return trace_size; +} + +__attribute__((__format__ (__printf__, 1, 0))) +void printString(const char * format, ...) // NOLINT(cert-dcl50-cpp) +{ + std::array formatted; + va_list args; + va_start(args, format); + + if (vsnprintf(formatted.data(), formatted.size(), format, args) > 0) + std::cerr << formatted.data() << std::endl; + + va_end(args); +} + } gwp_asan::GuardedPoolAllocator GuardedAlloc; + static bool guarded_alloc_initialized = [] { - gwp_asan::options::initOptions(); - gwp_asan::options::Options &opts = gwp_asan::options::getOptions(); - opts.MaxSimultaneousAllocations = 1024; + const char * env_options_raw = std::getenv("GWP_ASAN_OPTIONS"); // NOLINT(concurrency-mt-unsafe) + if (env_options_raw) + gwp_asan::options::initOptions(env_options_raw, printString); + + auto & opts = gwp_asan::options::getOptions(); opts.Backtrace = getBackTrace; GuardedAlloc.init(opts); @@ -53,8 +84,9 @@ bool isGWPAsanError(uintptr_t fault_address) namespace { -struct ScopedEndOfReportDecorator { - explicit ScopedEndOfReportDecorator(Poco::LoggerPtr log_) : log(std::move(log_)) {} +struct ScopedEndOfReportDecorator +{ + explicit ScopedEndOfReportDecorator(Poco::LoggerPtr log_) : log(std::move(log_)) { } ~ScopedEndOfReportDecorator() { LOG_FATAL(log, "*** End GWP-ASan report ***"); } Poco::LoggerPtr log; }; @@ -108,8 +140,6 @@ void printHeader(gwp_asan::Error error, uintptr_t fault_address, const gwp_asan: } } - // Possible number of digits of a 64-bit number: ceil(log10(2^64)) == 20. Add - // a null terminator, and round to the nearest 8-byte boundary. uint64_t thread_id = gwp_asan::getThreadID(); std::string thread_id_string = thread_id == gwp_asan::kInvalidThreadID ? " Date: Tue, 28 May 2024 08:42:17 +0000 Subject: [PATCH 162/856] Fix style --- src/Common/GWPAsan.cpp | 10 ---------- utils/check-style/check-style | 1 + 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index a46b7d640e4..ecff097e365 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -13,16 +13,6 @@ # include # include -# include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -} namespace Memory { diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 23e8b6b2bc4..1786418f9a5 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -311,6 +311,7 @@ std_cerr_cout_excludes=( src/Bridge/IBridge.cpp src/Daemon/BaseDaemon.cpp src/Loggers/Loggers.cpp + src/Common/GWPAsan.cpp ) sources_with_std_cerr_cout=( $( find $ROOT_PATH/{src,base} -name '*.h' -or -name '*.cpp' | \ From 73e9719768944871a44fc8bebfafa50fb0577aa5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 May 2024 19:18:47 +0200 Subject: [PATCH 163/856] Fix build --- src/IO/S3Common.cpp | 10 +++++----- src/IO/S3Common.h | 30 ++++++++++++++++-------------- 2 files changed, 21 insertions(+), 19 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index ef42b4b2642..1a1df2c9e9d 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -2,6 +2,10 @@ #include #include +#include +#include +#include +#include #include #include "config.h" @@ -11,10 +15,6 @@ #include #include #include -#include -#include -#include -#include namespace ProfileEvents @@ -50,7 +50,6 @@ bool S3Exception::isRetryableError() const namespace DB::ErrorCodes { extern const int S3_ERROR; - extern const int INVALID_SETTING_VALUE; } #endif @@ -62,6 +61,7 @@ namespace ErrorCodes { extern const int INVALID_CONFIG_PARAMETER; extern const int BAD_ARGUMENTS; + extern const int INVALID_SETTING_VALUE; } namespace S3 diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index b27b9ec1136..1572b93d3f9 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -3,24 +3,24 @@ #include #include #include - -#include "config.h" - -#if USE_AWS_S3 - +#include #include #include #include #include #include +#include +#include +#include + +#include "config.h" + +#if USE_AWS_S3 #include #include -#include - #include #include -#include namespace DB { @@ -30,10 +30,6 @@ namespace ErrorCodes extern const int S3_ERROR; } -class RemoteHostFilter; -class NamedCollection; -struct ProxyConfigurationResolver; - class S3Exception : public Exception { public: @@ -70,7 +66,12 @@ namespace Poco::Util class AbstractConfiguration; }; -namespace DB::S3 +namespace DB +{ +class NamedCollection; +struct ProxyConfigurationResolver; + +namespace S3 { #define AUTH_SETTINGS(M, ALIAS) \ @@ -139,7 +140,7 @@ struct AuthSettings : public BaseSettings AuthSettings(const DB::Settings & settings); - AuthSettings(const NamedCollection & collection); + AuthSettings(const DB::NamedCollection & collection); void updateFromSettings(const DB::Settings & settings, bool if_changed); bool hasUpdates(const AuthSettings & other) const; @@ -189,3 +190,4 @@ HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Ut ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); } +} From b73902b905aef2f5b05441e32b3e7e198960e249 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 28 May 2024 10:53:00 +0000 Subject: [PATCH 164/856] remove breakpoint --- tests/integration/test_scheduler/test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index d1ae51bc0d7..5779008cc13 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -448,7 +448,5 @@ def test_mutation_workload_change(): ).strip() ) - breakpoint() - assert reads_before < reads_after assert writes_before < writes_after From 1c9f4da6b081832c61842beb2a40c209beb2e5b7 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 28 May 2024 11:16:32 +0000 Subject: [PATCH 165/856] turn off dynamic untracked limit memory for not-attached threads (clients and tests) --- src/Common/ThreadStatus.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 49594116b91..db4854da707 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -187,7 +187,7 @@ public: /// For this purpose we dynamically change `untracked_memory_limit` after every tracking event using a simple formula: /// untracked_memory_limit = clamp(untracked_memory_ratio * cur_memory_bytes, min_untracked_memory, max_untracked_memory) /// Note that this values are updated when thread is attached to a group - Int64 min_untracked_memory = 4 * 1024; + Int64 min_untracked_memory = 4 * 1024 * 1024; // Default value is kept 4MB mostly for tests and client (should be changed to 4KB as default value a setting) Int64 max_untracked_memory = 4 * 1024 * 1024; /// Statistics of read and write rows/bytes From 2f6a86f3a11426b82acdba4d485581f1d7c5e1f7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 28 May 2024 11:45:12 +0000 Subject: [PATCH 166/856] remove unneeded fields --- src/Core/NamesAndTypes.cpp | 12 ++++ src/Core/NamesAndTypes.h | 3 + src/Storages/MergeTree/ColumnSizeEstimator.h | 10 +-- src/Storages/MergeTree/MergeTask.cpp | 70 ++++++++----------- src/Storages/MergeTree/MergeTask.h | 8 +-- .../MergeTreeDataPartWriterOnDisk.cpp | 1 + 6 files changed, 51 insertions(+), 53 deletions(-) diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index d6380a632f1..49ab822c738 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -188,6 +188,18 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const return filter(NameSet(names.begin(), names.end())); } +NamesAndTypesList NamesAndTypesList::eraseNames(const NameSet & names) const +{ + NamesAndTypesList res; + for (const auto & column : *this) + { + if (!names.contains(column.name)) + res.push_back(column); + } + return res; +} + + NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const { /// NOTE: It's better to make a map in `IStorage` than to create it here every time again. diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 915add9b7bc..29f40c45938 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -111,6 +111,9 @@ public: /// Leave only the columns whose names are in the `names`. In `names` there can be superfluous columns. NamesAndTypesList filter(const Names & names) const; + /// Leave only the columns whose names are not in the `names`. + NamesAndTypesList eraseNames(const NameSet & names) const; + /// Unlike `filter`, returns columns in the order in which they go in `names`. NamesAndTypesList addTypes(const Names & names) const; diff --git a/src/Storages/MergeTree/ColumnSizeEstimator.h b/src/Storages/MergeTree/ColumnSizeEstimator.h index 1307a5f493e..59a635a00fb 100644 --- a/src/Storages/MergeTree/ColumnSizeEstimator.h +++ b/src/Storages/MergeTree/ColumnSizeEstimator.h @@ -19,18 +19,18 @@ public: size_t sum_index_columns = 0; size_t sum_ordinary_columns = 0; - ColumnSizeEstimator(ColumnToSize && map_, const Names & key_columns, const Names & ordinary_columns) + ColumnSizeEstimator(ColumnToSize && map_, const NamesAndTypesList & key_columns, const NamesAndTypesList & ordinary_columns) : map(std::move(map_)) { - for (const auto & name : key_columns) + for (const auto & [name, _] : key_columns) if (!map.contains(name)) map[name] = 0; - for (const auto & name : ordinary_columns) + for (const auto & [name, _] : ordinary_columns) if (!map.contains(name)) map[name] = 0; - for (const auto & name : key_columns) + for (const auto & [name, _] : key_columns) sum_index_columns += map.at(name); - for (const auto & name : ordinary_columns) + for (const auto & [name, _] : ordinary_columns) sum_ordinary_columns += map.at(name); sum_total = std::max(static_cast(1), sum_index_columns + sum_ordinary_columns); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index bfe2f4673db..9dc72172a88 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -166,7 +166,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu if (key_columns.contains(column.name)) { global_ctx->merging_columns.emplace_back(column); - global_ctx->merging_column_names.emplace_back(column.name); auto it = global_ctx->skip_indexes_by_column.find(column.name); if (it != global_ctx->skip_indexes_by_column.end()) @@ -180,7 +179,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu else { global_ctx->gathering_columns.emplace_back(column); - global_ctx->gathering_column_names.emplace_back(column.name); } } } @@ -251,8 +249,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (!global_ctx->parent_part) global_ctx->temporary_directory_lock = global_ctx->data->getTemporaryPartDirectoryHolder(local_tmp_part_basename); - global_ctx->all_column_names = global_ctx->metadata_snapshot->getColumns().getNamesOfPhysical(); global_ctx->storage_columns = global_ctx->metadata_snapshot->getColumns().getAllPhysical(); + extractMergingAndGatheringColumns(); auto object_columns = MergeTreeData::getConcreteObjectColumns(global_ctx->future_part->parts, global_ctx->metadata_snapshot->getColumns()); @@ -272,10 +270,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->force_ttl = false; if (enabledBlockNumberColumn(global_ctx)) - addStorageColumn(global_ctx, BlockNumberColumn::name, BlockNumberColumn::type); + addGatheringColumn(global_ctx, BlockNumberColumn::name, BlockNumberColumn::type); if (enabledBlockOffsetColumn(global_ctx)) - addStorageColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); + addGatheringColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); SerializationInfo::Settings info_settings = { @@ -324,6 +322,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->sum_input_rows_upper_bound = global_ctx->merge_list_element_ptr->total_rows_count; ctx->sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed; + global_ctx->chosen_merge_algorithm = chooseMergeAlgorithm(); global_ctx->merge_list_element_ptr->merge_algorithm.store(global_ctx->chosen_merge_algorithm, std::memory_order_relaxed); @@ -344,14 +343,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() case MergeAlgorithm::Horizontal: { global_ctx->merging_columns = global_ctx->storage_columns; - global_ctx->merging_column_names = global_ctx->all_column_names; global_ctx->merging_skip_indexes = global_ctx->metadata_snapshot->getSecondaryIndices(); + global_ctx->gathering_columns.clear(); + global_ctx->skip_indexes_by_column.clear(); break; } case MergeAlgorithm::Vertical: { - extractMergingAndGatheringColumns(); - ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->createRawStream(); ctx->rows_sources_write_buf = std::make_unique(*ctx->rows_sources_uncompressed_write_buf); @@ -361,8 +359,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->column_sizes = ColumnSizeEstimator( std::move(local_merged_column_to_size), - global_ctx->merging_column_names, - global_ctx->gathering_column_names); + global_ctx->merging_columns, + global_ctx->gathering_columns); break; } @@ -370,9 +368,6 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge algorithm must be chosen"); } - assert(global_ctx->gathering_columns.size() == global_ctx->gathering_column_names.size()); - assert(global_ctx->merging_columns.size() == global_ctx->merging_column_names.size()); - /// If merge is vertical we cannot calculate it ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical); @@ -389,28 +384,25 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() /// resources for this). if (!ctx->need_remove_expired_values) { - size_t expired_columns = 0; auto part_serialization_infos = global_ctx->new_data_part->getSerializationInfos(); + NameSet columns_to_remove; for (auto & [column_name, ttl] : global_ctx->new_data_part->ttl_infos.columns_ttl) { if (ttl.finished()) { global_ctx->new_data_part->expired_columns.insert(column_name); LOG_TRACE(ctx->log, "Adding expired column {} for part {}", column_name, global_ctx->new_data_part->name); - std::erase(global_ctx->gathering_column_names, column_name); - std::erase(global_ctx->merging_column_names, column_name); - std::erase(global_ctx->all_column_names, column_name); + columns_to_remove.insert(column_name); part_serialization_infos.erase(column_name); - ++expired_columns; } } - if (expired_columns) + if (!columns_to_remove.empty()) { - global_ctx->gathering_columns = global_ctx->gathering_columns.filter(global_ctx->gathering_column_names); - global_ctx->merging_columns = global_ctx->merging_columns.filter(global_ctx->merging_column_names); - global_ctx->storage_columns = global_ctx->storage_columns.filter(global_ctx->all_column_names); + global_ctx->gathering_columns = global_ctx->gathering_columns.eraseNames(columns_to_remove); + global_ctx->merging_columns = global_ctx->merging_columns.eraseNames(columns_to_remove); + global_ctx->storage_columns = global_ctx->storage_columns.eraseNames(columns_to_remove); global_ctx->new_data_part->setColumns( global_ctx->storage_columns, @@ -448,15 +440,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() return false; } -void MergeTask::addStorageColumn(GlobalRuntimeContextPtr global_ctx, const String & name, const DataTypePtr & type) +void MergeTask::addGatheringColumn(GlobalRuntimeContextPtr global_ctx, const String & name, const DataTypePtr & type) { if (global_ctx->storage_columns.contains(name)) return; global_ctx->storage_columns.emplace_back(name, type); - global_ctx->all_column_names.emplace_back(name); global_ctx->gathering_columns.emplace_back(name, type); - global_ctx->gathering_column_names.emplace_back(name); } @@ -470,7 +460,6 @@ MergeTask::StageRuntimeContextPtr MergeTask::ExecuteAndFinalizeHorizontalPart::g new_ctx->compression_codec = std::move(ctx->compression_codec); new_ctx->tmp_disk = std::move(ctx->tmp_disk); new_ctx->it_name_and_type = std::move(ctx->it_name_and_type); - new_ctx->column_num_for_vertical_merge = std::move(ctx->column_num_for_vertical_merge); new_ctx->read_with_direct_io = std::move(ctx->read_with_direct_io); new_ctx->need_sync = std::move(ctx->need_sync); @@ -557,7 +546,7 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const size_t sum_input_rows_exact = global_ctx->merge_list_element_ptr->rows_read; size_t input_rows_filtered = *global_ctx->input_rows_filtered; - global_ctx->merge_list_element_ptr->columns_written = global_ctx->merging_column_names.size(); + global_ctx->merge_list_element_ptr->columns_written = global_ctx->merging_columns.size(); global_ctx->merge_list_element_ptr->progress.store(ctx->column_sizes->keyColumnsWeight(), std::memory_order_relaxed); ctx->rows_sources_write_buf->next(); @@ -592,14 +581,12 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const /// Move ownership from std::unique_ptr to std::unique_ptr for CompressedReadBufferFromFile. /// First, release ownership from unique_ptr to base type. reread_buf.release(); /// NOLINT(bugprone-unused-return-value,hicpp-ignored-remove-result): we already have the pointer value in `reread_buffer_raw` + /// Then, move ownership to unique_ptr to concrete type. std::unique_ptr reread_buffer_from_file(reread_buffer_raw); + /// CompressedReadBufferFromFile expects std::unique_ptr as argument. ctx->rows_sources_read_buf = std::make_unique(std::move(reread_buffer_from_file)); - - /// For external cycle - global_ctx->gathering_column_names_size = global_ctx->gathering_column_names.size(); - ctx->column_num_for_vertical_merge = 0; ctx->it_name_and_type = global_ctx->gathering_columns.cbegin(); const auto & settings = global_ctx->context->getSettingsRef(); @@ -743,8 +730,7 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const global_ctx->merge_list_element_ptr->bytes_written_uncompressed += bytes; global_ctx->merge_list_element_ptr->progress.store(ctx->progress_before + ctx->column_sizes->columnWeight(column_name), std::memory_order_relaxed); - /// This is the external cycle increment. - ++ctx->column_num_for_vertical_merge; + /// This is the external loop increment. ++ctx->it_name_and_type; } @@ -776,9 +762,9 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c LOG_DEBUG(ctx->log, "Merge sorted {} rows, containing {} columns ({} merged, {} gathered) in {} sec., {} rows/sec., {}/sec.", global_ctx->merge_list_element_ptr->rows_read, - global_ctx->all_column_names.size(), - global_ctx->merging_column_names.size(), - global_ctx->gathering_column_names.size(), + global_ctx->storage_columns.size(), + global_ctx->merging_columns.size(), + global_ctx->gathering_columns.size(), elapsed_seconds, global_ctx->merge_list_element_ptr->rows_read / elapsed_seconds, ReadableSize(global_ctx->merge_list_element_ptr->bytes_read_uncompressed / elapsed_seconds)); @@ -915,7 +901,7 @@ bool MergeTask::VerticalMergeStage::executeVerticalMergeForAllColumns() const return false; /// This is the external cycle condition - if (ctx->column_num_for_vertical_merge >= global_ctx->gathering_column_names_size) + if (ctx->it_name_and_type == global_ctx->gathering_columns.end()) return false; switch (ctx->vertical_merge_one_column_state) @@ -996,6 +982,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); + auto merging_column_names = global_ctx->merging_columns.getNames(); + for (const auto & part : global_ctx->future_part->parts) { Pipe pipe = createMergeTreeSequentialSource( @@ -1003,7 +991,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() *global_ctx->data, global_ctx->storage_snapshot, part, - global_ctx->merging_column_names, + merging_column_names, /*mark_ranges=*/ {}, /*apply_deleted_mask=*/ true, ctx->read_with_direct_io, @@ -1143,12 +1131,12 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() /// If deduplicate_by_columns is empty, add all columns except virtuals. if (global_ctx->deduplicate_by_columns.empty()) { - for (const auto & column_name : global_ctx->merging_column_names) + for (const auto & column : global_ctx->merging_columns) { - if (virtuals.tryGet(column_name, VirtualsKind::Persistent)) + if (virtuals.tryGet(column.name, VirtualsKind::Persistent)) continue; - global_ctx->deduplicate_by_columns.emplace_back(column_name); + global_ctx->deduplicate_by_columns.emplace_back(column.name); } } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index ae7e13dd244..7c509699903 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -167,17 +167,13 @@ private: NamesAndTypesList gathering_columns{}; NamesAndTypesList merging_columns{}; - Names gathering_column_names{}; - Names merging_column_names{}; NamesAndTypesList storage_columns{}; - Names all_column_names{}; MergeTreeData::DataPart::Checksums checksums_gathered_columns{}; IndicesDescription merging_skip_indexes; std::unordered_map skip_indexes_by_column; MergeAlgorithm chosen_merge_algorithm{MergeAlgorithm::Undecided}; - size_t gathering_column_names_size{0}; std::unique_ptr horizontal_stage_progress{nullptr}; std::unique_ptr column_progress{nullptr}; @@ -238,7 +234,6 @@ private: /// Dependencies for next stages std::list::const_iterator it_name_and_type; - size_t column_num_for_vertical_merge{0}; bool need_sync{false}; }; @@ -292,7 +287,6 @@ private: CompressionCodecPtr compression_codec; TemporaryDataOnDiskPtr tmp_disk{nullptr}; std::list::const_iterator it_name_and_type; - size_t column_num_for_vertical_merge{0}; bool read_with_direct_io{false}; bool need_sync{false}; /// End dependencies from previous stages @@ -422,7 +416,7 @@ private: return global_ctx->data->getSettings()->enable_block_offset_column && global_ctx->metadata_snapshot->getGroupByTTLs().empty(); } - static void addStorageColumn(GlobalRuntimeContextPtr global_ctx, const String & name, const DataTypePtr & type); + static void addGatheringColumn(GlobalRuntimeContextPtr global_ctx, const String & name, const DataTypePtr & type); }; /// FIXME diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 0a8920790e0..3754b3beab1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -176,6 +176,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( if (settings.rewrite_primary_key) initPrimaryIndex(); + initSkipIndices(); initStatistics(); } From 9d961d1936f791183812892e028e346643ec6efc Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 May 2024 13:54:34 +0200 Subject: [PATCH 167/856] Better --- src/Core/Settings.h | 17 +-- src/Disks/ObjectStorages/S3/diskSettings.cpp | 6 +- src/IO/S3Common.cpp | 136 +++++++++++++----- src/IO/S3Common.h | 29 ++-- src/IO/S3Defines.h | 10 +- src/IO/S3Settings.cpp | 8 +- .../ObjectStorage/S3/Configuration.cpp | 7 +- 7 files changed, 140 insertions(+), 73 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a7a19702282..ee2dc38b0d7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -79,7 +79,7 @@ class IColumn; M(UInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \ M(UInt64, distributed_connections_pool_size, 1024, "Maximum number of connections with one remote server in the pool.", 0) \ M(UInt64, connections_with_failover_max_tries, 3, "The maximum number of attempts to connect to replicas.", 0) \ - M(UInt64, s3_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to S3 (some implementations does not supports variable size parts).", 0) \ + M(UInt64, s3_strict_upload_part_size, S3::DEFAULT_STRICT_UPLOAD_PART_SIZE, "The exact size of part to upload during multipart upload to S3 (some implementations does not supports variable size parts).", 0) \ M(UInt64, azure_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to Azure blob storage.", 0) \ M(UInt64, azure_max_blocks_in_multipart_upload, 50000, "Maximum number of blocks in multipart upload for Azure.", 0) \ M(UInt64, s3_min_upload_part_size, S3::DEFAULT_MIN_UPLOAD_PART_SIZE, "The minimum size of part to upload during multipart upload to S3.", 0) \ @@ -97,17 +97,17 @@ class IColumn; M(UInt64, s3_max_single_part_upload_size, S3::DEFAULT_MAX_SINGLE_PART_UPLOAD_SIZE, "The maximum size of object to upload using singlepart upload to S3.", 0) \ M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ M(UInt64, azure_max_single_part_copy_size, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage.", 0) \ - M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ + M(UInt64, s3_max_single_read_retries, S3::DEFAULT_MAX_SINGLE_READ_TRIES, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single Azure blob storage read.", 0) \ M(UInt64, azure_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write", 0) \ - M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ - M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ + M(UInt64, s3_max_unexpected_write_error_retries, S3::DEFAULT_MAX_UNEXPECTED_WRITE_ERROR_RETRIES, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ + M(UInt64, s3_max_redirects, S3::DEFAULT_MAX_REDIRECTS, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, S3::DEFAULT_MAX_CONNECTIONS, "The maximum number of connections per server.", 0) \ M(UInt64, s3_max_get_rps, 0, "Limit on S3 GET request per second rate before throttling. Zero means unlimited.", 0) \ M(UInt64, s3_max_get_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_get_rps`", 0) \ M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate before throttling. Zero means unlimited.", 0) \ M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \ - M(UInt64, s3_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ + M(UInt64, s3_list_object_keys_size, S3::DEFAULT_LIST_OBJECT_KEYS_SIZE, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_use_adaptive_timeouts, S3::DEFAULT_USE_ADAPTIVE_TIMEOUTS, "When adaptive timeouts are enabled first two attempts are made with low receive and send timeout", 0) \ M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ @@ -125,14 +125,15 @@ class IColumn; M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ M(Bool, s3_validate_request_settings, true, "Validate S3 request settings", 0) \ - M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ - M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ + M(Bool, s3_disable_checksum, S3::DEFAULT_DISABLE_CHECKSUM, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ + M(UInt64, s3_retry_attempts, S3::DEFAULT_RETRY_ATTEMPTS, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, S3::DEFAULT_REQUEST_TIMEOUT_MS, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ M(UInt64, s3_connect_timeout_ms, S3::DEFAULT_CONNECT_TIMEOUT_MS, "Connection timeout for host from s3 disks.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(String, s3queue_default_zookeeper_path, "/clickhouse/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \ M(Bool, s3queue_enable_logging_to_s3queue_log, false, "Enable writing to system.s3queue_log. The value can be overwritten per table with table settings", 0) \ - M(Bool, s3queue_allow_experimental_sharded_mode, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten", 0) \ + M(Bool, s3queue_allow_experimental_sharded_mode, false \ + , "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 591b1e6623d..14bb5f05071 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -39,10 +39,8 @@ std::unique_ptr getSettings( bool validate_settings) { const auto & settings = context->getSettingsRef(); - const std::string setting_name_prefix = for_disk_s3 ? "s3_" : ""; - - auto auth_settings = S3::AuthSettings(config, config_prefix, settings); - auto request_settings = S3::RequestSettings(config, config_prefix, settings, validate_settings, setting_name_prefix); + auto auth_settings = S3::AuthSettings(config, settings, for_disk_s3, for_disk_s3 ? config_prefix : ""); + auto request_settings = S3::RequestSettings(config, settings, for_disk_s3, validate_settings, for_disk_s3 ? config_prefix : ""); request_settings.proxy_resolver = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( ProxyConfiguration::protocolFromString(S3::URI(endpoint).uri.getScheme()), config_prefix, config); diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 1a1df2c9e9d..ef1e630582d 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -104,28 +104,55 @@ ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, c AuthSettings::AuthSettings( const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - const DB::Settings &, /// TODO: use settings - const std::string & setting_name_prefix) + const DB::Settings & settings, + bool for_disk_s3, + const std::string & disk_config_prefix) { - const std::string prefix = config_prefix + "." + setting_name_prefix; + if (for_disk_s3 && disk_config_prefix.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk config path cannot be empty"); + + auto update_value_if_exists = [&](const std::string & path, SettingFieldRef & field) -> bool + { + if (!config.has(path)) + return false; + + auto which = field.getValue().getType(); + if (isInt64OrUInt64FieldType(which)) + field.setValue(config.getUInt64(path)); + else if (which == Field::Types::String) + field.setValue(config.getString(path)); + else if (which == Field::Types::Bool) + field.setValue(config.getBool(path)); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); + return true; + }; + for (auto & field : allMutable()) { - const auto path = prefix + field.getName(); - if (config.has(path)) + std::string path, fallback_path; + if (for_disk_s3) { - auto which = field.getValue().getType(); - if (isInt64OrUInt64FieldType(which)) - field.setValue(config.getUInt64(path)); - else if (which == Field::Types::String) - field.setValue(config.getString(path)); - else if (which == Field::Types::Bool) - field.setValue(config.getBool(path)); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); + path = fmt::format("{}.s3_{}", disk_config_prefix, field.getName()); + fallback_path = fmt::format("s3.{}", field.getName()); + } + else + path = fmt::format("s3.{}", field.getName()); + + bool updated = update_value_if_exists(path, field); + + if (!updated && !fallback_path.empty()) + updated = update_value_if_exists(fallback_path, field); + + if (!updated) + { + auto setting_name = "s3_" + field.getName(); + if (settings.has(setting_name) && settings.isChanged(setting_name)) + field.setValue(settings.get(setting_name)); } } + const auto config_prefix = for_disk_s3 ? disk_config_prefix : "s3"; headers = getHTTPHeaders(config_prefix, config); server_side_encryption_kms_config = getSSEKMSConfig(config_prefix, config); @@ -150,7 +177,7 @@ void AuthSettings::updateFromSettings(const DB::Settings & settings, bool if_cha const auto setting_name = "s3_" + field.getName(); if (settings.has(setting_name) && (!if_changed || settings.isChanged(setting_name))) { - set(field.getName(), settings.get(setting_name)); + field.setValue(settings.get(setting_name)); } } } @@ -164,9 +191,6 @@ bool AuthSettings::hasUpdates(const AuthSettings & other) const void AuthSettings::updateIfChanged(const AuthSettings & settings) { - /// Update with check for emptyness only parameters which - /// can be passed not only from config, but via ast. - for (auto & setting : settings.all()) { if (setting.isValueChanged()) @@ -175,34 +199,64 @@ void AuthSettings::updateIfChanged(const AuthSettings & settings) if (!settings.headers.empty()) headers = settings.headers; + + if (!settings.users.empty()) + users.insert(settings.users.begin(), settings.users.end()); + server_side_encryption_kms_config = settings.server_side_encryption_kms_config; - users.insert(settings.users.begin(), settings.users.end()); } RequestSettings::RequestSettings( const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, const DB::Settings & settings, + bool for_disk_s3, bool validate_settings, - const std::string & setting_name_prefix) + const std::string & disk_config_path) { - String prefix = config_prefix + "." + setting_name_prefix; + if (for_disk_s3 && disk_config_path.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk config path cannot be empty"); + + auto update_value_if_exists = [&](const std::string & path, SettingFieldRef & field) -> bool + { + if (!config.has(path)) + return false; + + auto which = field.getValue().getType(); + if (isInt64OrUInt64FieldType(which)) + field.setValue(config.getUInt64(path)); + else if (which == Field::Types::String) + field.setValue(config.getString(path)); + else if (which == Field::Types::Bool) + field.setValue(config.getBool(path)); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); + return true; + }; + for (auto & field : allMutable()) { - const auto path = prefix + field.getName(); - if (config.has(path)) + std::string path, fallback_path; + if (for_disk_s3) { - auto which = field.getValue().getType(); - if (isInt64OrUInt64FieldType(which)) - field.setValue(config.getUInt64(path)); - else if (which == Field::Types::String) - field.setValue(config.getString(path)); - else if (which == Field::Types::Bool) - field.setValue(config.getBool(path)); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); + path = fmt::format("{}.s3_{}", disk_config_path, field.getName()); + fallback_path = fmt::format("s3.{}", field.getName()); + } + else + path = fmt::format("s3.{}", field.getName()); + + bool updated = update_value_if_exists(path, field); + + if (!updated && !fallback_path.empty()) + updated = update_value_if_exists(fallback_path, field); + + if (!updated) + { + auto setting_name = "s3_" + field.getName(); + if (settings.has(setting_name) && settings.isChanged(setting_name)) + field.setValue(settings.get(setting_name)); } } + finishInit(settings, validate_settings); } @@ -238,7 +292,7 @@ RequestSettings::RequestSettings(const DB::Settings & settings, bool validate_se } void RequestSettings::updateFromSettings( - const DB::Settings & settings, bool if_changed, bool /* validate_settings */) /// TODO: process validate_settings + const DB::Settings & settings, bool if_changed, bool validate_settings) { for (auto & field : allMutable()) { @@ -248,6 +302,10 @@ void RequestSettings::updateFromSettings( set(field.getName(), settings.get(setting_name)); } } + + normalizeSettings(); + if (validate_settings) + validateUploadSettings(); } void RequestSettings::updateIfChanged(const RequestSettings & settings) @@ -259,11 +317,15 @@ void RequestSettings::updateIfChanged(const RequestSettings & settings) } } -void RequestSettings::finishInit(const DB::Settings & settings, bool validate_settings) +void RequestSettings::normalizeSettings() { if (!storage_class_name.value.empty() && storage_class_name.changed) storage_class_name = Poco::toUpperInPlace(storage_class_name.value); +} +void RequestSettings::finishInit(const DB::Settings & settings, bool validate_settings) +{ + normalizeSettings(); if (validate_settings) validateUploadSettings(); @@ -373,8 +435,6 @@ void RequestSettings::validateUploadSettings() } -/// TODO: sometimes disk settings have fallback to "s3" section settings from config, support this. - IMPLEMENT_SETTINGS_TRAITS(S3::AuthSettingsTraits, CLIENT_SETTINGS_LIST) IMPLEMENT_SETTINGS_TRAITS(S3::RequestSettingsTraits, REQUEST_SETTINGS_LIST) diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 1572b93d3f9..c5b31c4b564 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -73,6 +73,14 @@ struct ProxyConfigurationResolver; namespace S3 { +/// We use s3 settings for DiskS3, StorageS3 (StorageS3Cluster, S3Queue, etc), BackupIO_S3, etc. +/// 1. For DiskS3 we usually have configuration in disk section in configuration file. +/// All s3 related settings start with "s3_" prefix there. +/// If some setting is absent from disk configuration, we look up for it in the "s3." server config section, +/// where s3 settings no longer have "s3_" prefix like in disk configuration section. +/// If the settings is absent there as well, we look up for it in Users config (where query/session settings are also updated). +/// 2. For StorageS3 and similar - we look up to "s3." config section (again - settings there do not have "s3_" prefix). +/// If some setting is absent from there, we lool up for it in Users config. #define AUTH_SETTINGS(M, ALIAS) \ M(String, access_key_id, "", "", 0) \ @@ -134,9 +142,9 @@ struct AuthSettings : public BaseSettings AuthSettings( const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, const DB::Settings & settings, - const std::string & setting_name_prefix = ""); + bool for_disk_s3, + const std::string & disk_config_prefix = ""); AuthSettings(const DB::Settings & settings); @@ -156,6 +164,14 @@ struct RequestSettings : public BaseSettings { RequestSettings() = default; + /// Create request settings from Config. + RequestSettings( + const Poco::Util::AbstractConfiguration & config, + const DB::Settings & settings, + bool for_disk_s3, + bool validate_settings = true, + const std::string & disk_config_path = ""); + /// Create request settings from DB::Settings. explicit RequestSettings(const DB::Settings & settings, bool validate_settings = true); @@ -165,14 +181,6 @@ struct RequestSettings : public BaseSettings const DB::Settings & settings, bool validate_settings = true); - /// Create request settings from Config. - RequestSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - const DB::Settings & settings, - bool validate_settings = true, - const std::string & setting_name_prefix = ""); - void updateFromSettings(const DB::Settings & settings, bool if_changed, bool validate_settings = true); void updateIfChanged(const RequestSettings & settings); void validateUploadSettings(); @@ -183,6 +191,7 @@ struct RequestSettings : public BaseSettings private: void finishInit(const DB::Settings & settings, bool validate_settings); + void normalizeSettings(); }; HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); diff --git a/src/IO/S3Defines.h b/src/IO/S3Defines.h index eedd0df81a6..332ebcfea92 100644 --- a/src/IO/S3Defines.h +++ b/src/IO/S3Defines.h @@ -21,6 +21,7 @@ inline static constexpr bool DEFAULT_USE_ADAPTIVE_TIMEOUTS = true; inline static constexpr uint64_t DEFAULT_MIN_UPLOAD_PART_SIZE = 16 * 1024 * 1024; inline static constexpr uint64_t DEFAULT_MAX_UPLOAD_PART_SIZE = 5ull * 1024 * 1024 * 1024; inline static constexpr uint64_t DEFAULT_MAX_SINGLE_PART_UPLOAD_SIZE = 32 * 1024 * 1024; +inline static constexpr uint64_t DEFAULT_STRICT_UPLOAD_PART_SIZE = 0; inline static constexpr uint64_t DEFAULT_UPLOAD_PART_SIZE_MULTIPLY_FACTOR = 2; inline static constexpr uint64_t DEFAULT_UPLOAD_PART_SIZE_MULTIPLY_PARTS_COUNT_THRESHOLD = 500; inline static constexpr uint64_t DEFAULT_MAX_PART_NUMBER = 10000; @@ -29,7 +30,12 @@ inline static constexpr uint64_t DEFAULT_MAX_PART_NUMBER = 10000; inline static constexpr uint64_t DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE = 32 * 1024 * 1024; inline static constexpr uint64_t DEFAULT_MAX_INFLIGHT_PARTS_FOR_ONE_FILE = 20; inline static constexpr uint64_t DEFAULT_LIST_OBJECT_KEYS_SIZE = 1000; -inline static constexpr uint64_t DEFAULT_ALLOW_NATIVE_COPY = true; -inline static constexpr uint64_t DEFAULT_CHECK_OBJECTS_AFTER_UPLOAD = false; +inline static constexpr uint64_t DEFAULT_MAX_SINGLE_READ_TRIES = 4; +inline static constexpr uint64_t DEFAULT_MAX_UNEXPECTED_WRITE_ERROR_RETRIES = 4; +inline static constexpr uint64_t DEFAULT_MAX_REDIRECTS = 10; +inline static constexpr uint64_t DEFAULT_RETRY_ATTEMPTS = 100; + +inline static constexpr bool DEFAULT_ALLOW_NATIVE_COPY = true; +inline static constexpr bool DEFAULT_CHECK_OBJECTS_AFTER_UPLOAD = false; } diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp index 4197014c454..85f30e7e316 100644 --- a/src/IO/S3Settings.cpp +++ b/src/IO/S3Settings.cpp @@ -1,11 +1,7 @@ #include #include - #include -#include -#include -#include #include @@ -32,8 +28,8 @@ void S3SettingsByEndpoint::loadFromConfig( if (config.has(endpoint_path)) { auto endpoint = config.getString(endpoint_path); - auto auth_settings = S3::AuthSettings(config, key_path, settings); - auto request_settings = S3::RequestSettings(config, key_path, settings); + auto auth_settings = S3::AuthSettings(config, settings, /* for_disk_s3 */false, config_prefix); + auto request_settings = S3::RequestSettings(config, settings, /* for_disk_s3 */false, settings.s3_validate_request_settings, config_prefix); s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); } } diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 327efba2169..d59b3e8ea06 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -112,11 +112,8 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, config, config_prefix, context, url.endpoint, /* for_disk_s3 */false, settings.s3_validate_request_settings); - request_settings.updateFromSettings(settings, /* if_changed */true); - auth_settings.updateIfChanged(s3_settings->auth_settings); - - s3_settings->auth_settings = auth_settings; - s3_settings->request_settings = request_settings; + s3_settings->auth_settings.updateIfChanged(auth_settings); + s3_settings->request_settings.updateIfChanged(request_settings); if (!headers_from_ast.empty()) { From 9e1cd7e8986cb433ded9988268008dab878a9eff Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 May 2024 14:03:51 +0200 Subject: [PATCH 168/856] Unify part of the code --- src/IO/S3Common.cpp | 62 +++++++++++---------------------------------- 1 file changed, 15 insertions(+), 47 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index ef1e630582d..ef5108612ec 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -102,7 +102,9 @@ ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, c return sse_kms_config; } -AuthSettings::AuthSettings( +template +static void updateS3SettingsFromConfig( + Settings & s3_settings, const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, bool for_disk_s3, @@ -111,7 +113,7 @@ AuthSettings::AuthSettings( if (for_disk_s3 && disk_config_prefix.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk config path cannot be empty"); - auto update_value_if_exists = [&](const std::string & path, SettingFieldRef & field) -> bool + auto update_value_if_exists = [&](const std::string & path, Settings::SettingFieldRef & field) -> bool { if (!config.has(path)) return false; @@ -128,7 +130,7 @@ AuthSettings::AuthSettings( return true; }; - for (auto & field : allMutable()) + for (auto & field : s3_settings.allMutable()) { std::string path, fallback_path; if (for_disk_s3) @@ -151,6 +153,15 @@ AuthSettings::AuthSettings( field.setValue(settings.get(setting_name)); } } +} + +AuthSettings::AuthSettings( + const Poco::Util::AbstractConfiguration & config, + const DB::Settings & settings, + bool for_disk_s3, + const std::string & disk_config_prefix) +{ + updateS3SettingsFromConfig(*this, config, settings, for_disk_s3, disk_config_prefix); const auto config_prefix = for_disk_s3 ? disk_config_prefix : "s3"; headers = getHTTPHeaders(config_prefix, config); @@ -213,50 +224,7 @@ RequestSettings::RequestSettings( bool validate_settings, const std::string & disk_config_path) { - if (for_disk_s3 && disk_config_path.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk config path cannot be empty"); - - auto update_value_if_exists = [&](const std::string & path, SettingFieldRef & field) -> bool - { - if (!config.has(path)) - return false; - - auto which = field.getValue().getType(); - if (isInt64OrUInt64FieldType(which)) - field.setValue(config.getUInt64(path)); - else if (which == Field::Types::String) - field.setValue(config.getString(path)); - else if (which == Field::Types::Bool) - field.setValue(config.getBool(path)); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); - return true; - }; - - for (auto & field : allMutable()) - { - std::string path, fallback_path; - if (for_disk_s3) - { - path = fmt::format("{}.s3_{}", disk_config_path, field.getName()); - fallback_path = fmt::format("s3.{}", field.getName()); - } - else - path = fmt::format("s3.{}", field.getName()); - - bool updated = update_value_if_exists(path, field); - - if (!updated && !fallback_path.empty()) - updated = update_value_if_exists(fallback_path, field); - - if (!updated) - { - auto setting_name = "s3_" + field.getName(); - if (settings.has(setting_name) && settings.isChanged(setting_name)) - field.setValue(settings.get(setting_name)); - } - } - + updateS3SettingsFromConfig(*this, config, settings, for_disk_s3, disk_config_path); finishInit(settings, validate_settings); } From 01183902a667d94d71ed9faabeffdc60cdcf95cd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 12:07:20 +0000 Subject: [PATCH 169/856] 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 dfc4d0c60aeb1e796e698d6a11bca05722c593e2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 28 May 2024 14:00:54 +0000 Subject: [PATCH 170/856] Cleanup and diagnostic --- src/Planner/PlannerJoinTree.cpp | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 96b73a26095..d7aa2e8de24 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -884,29 +884,28 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres // (1) find read step QueryPlan::Node * node = query_plan.getRootNode(); ReadFromMergeTree * reading = nullptr; - QueryPlan::Node * last_node = nullptr; while (node) { reading = typeid_cast(node->step.get()); if (reading) break; - last_node = node; + QueryPlan::Node * prev_node = node; if (!node->children.empty()) { + chassert(node->children.size() == 1); node = node->children.at(0); } else { - node = nullptr; + if (prev_node->step) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Step is expected to be ReadFromMergeTree but it's {}", prev_node->step->getName()); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Step is expected to be ReadFromMergeTree, and wtf - last node with empty step"); } } - // chassert(reading); - if (!reading) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading step is expected to be ReadFromMergeTree but it's {}", last_node->step->getName()); - } + chassert(reading); // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) @@ -955,9 +954,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_context, table_expression_query_info.storage_limits); query_plan = std::move(query_plan_parallel_replicas); - - const Block & query_plan_header = query_plan.getCurrentDataStream().header; - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "Parallel replicas query_plan_header:\n{}", query_plan_header.dumpStructure()); } } From 1bedd6192e06e414cd99c9b1939eb153ac679115 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 28 May 2024 14:01:53 +0000 Subject: [PATCH 171/856] add test --- ...3166_skip_indexes_vertical_merge.reference | 32 +++++++++++++++++ .../03166_skip_indexes_vertical_merge.sql | 34 +++++++++++++++++++ 2 files changed, 66 insertions(+) create mode 100644 tests/queries/0_stateless/03166_skip_indexes_vertical_merge.reference create mode 100644 tests/queries/0_stateless/03166_skip_indexes_vertical_merge.sql diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.reference b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.reference new file mode 100644 index 00000000000..02d5765102c --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.reference @@ -0,0 +1,32 @@ +200 +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.t_ind_merge) + Indexes: + PrimaryKey + Condition: true + Parts: 2/2 + Granules: 32/32 + Skip + Name: idx_b + Description: minmax GRANULARITY 1 + Parts: 2/2 + Granules: 4/32 +200 +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.t_ind_merge) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 32/32 + Skip + Name: idx_b + Description: minmax GRANULARITY 1 + Parts: 1/1 + Granules: 4/32 diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.sql new file mode 100644 index 00000000000..b894c054f8a --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS t_ind_merge; + +CREATE TABLE t_ind_merge (a UInt64, b UInt64, c UInt64, d UInt64, INDEX idx_b b TYPE minmax) +ENGINE = MergeTree +ORDER BY a SETTINGS + index_granularity = 64, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + min_bytes_for_wide_part = 0; + +INSERT INTO t_ind_merge SELECT number, number, rand(), rand() from numbers(1000); +INSERT INTO t_ind_merge SELECT number, number, rand(), rand() from numbers(1000); + +SELECT count() FROM t_ind_merge WHERE b < 100 SETTINGS force_data_skipping_indices = 'idx_b'; +EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge WHERE b < 100; + +OPTIMIZE TABLE t_ind_merge FINAL; + +SELECT count() FROM t_ind_merge WHERE b < 100 SETTINGS force_data_skipping_indices = 'idx_b'; +EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge WHERE b < 100; + +DROP TABLE t_ind_merge; +SYSTEM FLUSH LOGS; + +WITH + (SELECT uuid FROM system.tables WHERE database = currentDatabase() AND table = 't_ind_merge') AS uuid, + extractAllGroupsVertical(message, 'containing (\\d+) columns \((\\d+) merged, (\\d+) gathered\)')[1] AS groups +SELECT + groups[1] AS total, + groups[2] AS merged, + groups[3] AS gathered +FROM system.text_log +WHERE query_id = uuid || '::all_1_2_1' AND notEmpty(groups) +ORDER BY event_time_microseconds; From d07c6461e2d480cad7d95aeceed070f78d42bfc5 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 28 May 2024 14:17:33 +0000 Subject: [PATCH 172/856] fix syntax error --- .../0_stateless/03030_system_flush_distributed_settings.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql b/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql index e8a3da174a6..7961444dbc2 100644 --- a/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql +++ b/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql @@ -13,7 +13,7 @@ create table dist_out as data engine=Distributed(test_shard_localhost, currentDa set prefer_localhost_replica=0; -set min_untracked_memory='4Mi' -- Disable precise memory tracking +set min_untracked_memory='4Mi'; -- Disable precise memory tracking insert into dist_in select number/100, number from system.numbers limit 1e6 settings max_memory_usage='20Mi'; system flush distributed dist_in; -- { serverError MEMORY_LIMIT_EXCEEDED } From bb8f887dfb6ff97a06625c3798f8a27ca5269720 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 28 May 2024 14:19:48 +0000 Subject: [PATCH 173/856] fix typo --- docs/en/operations/workload-scheduling.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index 3b950bd36b1..08629492ec6 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -156,6 +156,6 @@ Example: ## See also - [system.scheduler](/docs/en/operations/system-tables/scheduler.md) - [merge_workload](/docs/en/operations/settings/merge-tree-settings.md#merge_workload) merge tree setting - - [merge_workload](/docs/en/operations/server-configuration-parametes/settings.md#merge_workload) global server setting + - [merge_workload](/docs/en/operations/server-configuration-parameters/settings.md#merge_workload) global server setting - [mutation_workload](/docs/en/operations/settings/merge-tree-settings.md#mutation_workload) merge tree setting - - [mutation_workload](/docs/en/operations/server-configuration-parametes/settings.md#mutation_workload) global server setting + - [mutation_workload](/docs/en/operations/server-configuration-parameters/settings.md#mutation_workload) global server setting From dc1c1bcdd04892cb2920f0166ce0297a78d6abf0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 May 2024 17:02:06 +0200 Subject: [PATCH 174/856] Update settings changes history --- src/Core/SettingsChangesHistory.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 16f28d94640..29c48dae422 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,6 +92,8 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, + {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, From 45f6c19c9df5c3f62b1ed4933321053ef6f77c91 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 15:36:19 +0000 Subject: [PATCH 175/856] 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 cec2e40398d668c841fb1645cb8782b45cb7ff8b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 May 2024 18:52:47 +0200 Subject: [PATCH 176/856] Fix build --- src/Coordination/KeeperSnapshotManagerS3.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index b8c8d10d497..db1f84a047f 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -65,7 +65,8 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo return; } - auto auth_settings = S3::AuthSettings(config, config_prefix, Context::getGlobalContextInstance()->getSettingsRef()); + const auto & settings = Context::getGlobalContextInstance()->getSettingsRef(); + auto auth_settings = S3::AuthSettings(config, settings, true, config_prefix); String endpoint = macros->expand(config.getString(config_prefix + ".endpoint")); auto new_uri = S3::URI{endpoint}; From f46a7d64a0163e0cf9140eb0e56c88f2cc6471bb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 17:00:35 +0000 Subject: [PATCH 177/856] 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 dfd19576cde5396850d31eb77d43a091da9f5cfe Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 May 2024 19:42:57 +0200 Subject: [PATCH 178/856] Fix keeper build --- programs/keeper/CMakeLists.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index af360e44ff4..517c7d7ab4b 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -156,8 +156,6 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolReader.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Storages/StorageS3Settings.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/BaseDaemon.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/SentryWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp From 513de6ce19867dc10fedf5c9820363b84655a9f1 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 17:59:44 +0000 Subject: [PATCH 179/856] 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 ef3b802b4e543cb879fe7c45eb1f57423c3e471e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 May 2024 18:42:10 +0200 Subject: [PATCH 180/856] add test --- .../configs/config2.xml | 10 ++++++++ .../test_replicated_database/test.py | 23 ++++++++++++++++++- 2 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_replicated_database/configs/config2.xml diff --git a/tests/integration/test_replicated_database/configs/config2.xml b/tests/integration/test_replicated_database/configs/config2.xml new file mode 100644 index 00000000000..727461697ca --- /dev/null +++ b/tests/integration/test_replicated_database/configs/config2.xml @@ -0,0 +1,10 @@ + + 10 + 1 + + 10 + + 50 + 42 + group + diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index fd1bfc75227..ef86da5af30 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -46,7 +46,7 @@ snapshotting_node = cluster.add_instance( ) snapshot_recovering_node = cluster.add_instance( "snapshot_recovering_node", - main_configs=["configs/config.xml"], + main_configs=["configs/config2.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, ) @@ -1522,3 +1522,24 @@ def test_auto_recovery(started_cluster): assert "42\n" == bad_settings_node.query("SELECT * FROM auto_recovery.t2") assert "137\n" == bad_settings_node.query("SELECT * FROM auto_recovery.t1") + + +def test_all_groups_cluster(started_cluster): + dummy_node.query("DROP DATABASE IF EXISTS db_cluster") + bad_settings_node.query("DROP DATABASE IF EXISTS db_cluster") + dummy_node.query( + "CREATE DATABASE db_cluster ENGINE = Replicated('/clickhouse/databases/all_groups_cluster', 'shard1', 'replica1');" + ) + bad_settings_node.query( + "CREATE DATABASE db_cluster ENGINE = Replicated('/clickhouse/databases/all_groups_cluster', 'shard1', 'replica2');" + ) + + assert "bad_settings_node\ndummy_node\n" == dummy_node.query( + "select host_name from system.clusters where name='db_cluster' order by host_name" + ) + assert "bad_settings_node\n" == bad_settings_node.query( + "select host_name from system.clusters where name='db_cluster' order by host_name" + ) + assert "bad_settings_node\ndummy_node\n" == bad_settings_node.query( + "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" + ) From 3bd3717d3437ab00aa240e4908b9b82542c42de1 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 28 May 2024 17:47:39 +0800 Subject: [PATCH 181/856] revert setting rename --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Core/SettingsQuirks.cpp | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp | 4 ++-- src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h | 2 +- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 5 ++--- .../03164_adapting_parquet_reader_output_size.sql | 2 +- 9 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 011541088ac..c9efd1e4a97 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1053,7 +1053,7 @@ class IColumn; M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ - M(UInt64, input_format_parquet_max_block_rows, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader.", 0) \ + M(UInt64, input_format_parquet_max_block_size, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader.", 0) \ M(UInt64, input_format_parquet_prefer_block_bytes, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader", 0) \ M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format Protobuf", 0) \ M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format CapnProto", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ab83da5de8d..e6fb628809a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,7 +92,7 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"input_format_parquet_max_block_rows", 8192, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader."}, + {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader."}, {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 4065ee40285..5541cc19653 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -117,7 +117,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) "min_insert_block_size_bytes_for_materialized_views", "min_external_table_block_size_rows", "max_joined_block_size_rows", - "input_format_parquet_max_block_rows"}; + "input_format_parquet_max_block_size"}; for (auto const & setting : block_rows_settings) { if (auto block_size = get_current_value(setting).get(); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a01be503c4f..e90986f2236 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -160,7 +160,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference; format_settings.parquet.output_string_as_string = settings.output_format_parquet_string_as_string; format_settings.parquet.output_fixed_string_as_fixed_byte_array = settings.output_format_parquet_fixed_string_as_fixed_byte_array; - format_settings.parquet.max_block_rows = settings.input_format_parquet_max_block_rows; + format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; format_settings.parquet.prefer_block_bytes = settings.input_format_parquet_prefer_block_bytes; format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; format_settings.parquet.output_compliant_nested_types = settings.output_format_parquet_compliant_nested_types; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index f7b57ddd4aa..337aafbbe9c 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -265,7 +265,7 @@ struct FormatSettings bool preserve_order = false; bool use_custom_encoder = true; bool parallel_encoding = true; - UInt64 max_block_rows = DEFAULT_BLOCK_SIZE; + UInt64 max_block_size = DEFAULT_BLOCK_SIZE; size_t prefer_block_bytes = DEFAULT_BLOCK_SIZE * 256; ParquetVersion output_version; ParquetCompression output_compression_method = ParquetCompression::SNAPPY; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index ad98db3b8ab..a7e51f88b3c 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -307,7 +307,7 @@ ParquetRecordReader::ParquetRecordReader( : file_reader(createFileReader(std::move(arrow_file), std::move(metadata))) , reader_properties(reader_properties_) , header(std::move(header_)) - , max_block_rows(format_settings.parquet.max_block_rows) + , max_block_size(format_settings.parquet.max_block_size) , row_groups_indices(std::move(row_groups_indices_)) , left_rows(getTotalRows(*file_reader->metadata())) { @@ -356,7 +356,7 @@ Chunk ParquetRecordReader::readChunk() } Columns columns(header.columns()); - auto num_rows_read = std::min(max_block_rows, cur_row_group_left_rows); + auto num_rows_read = std::min(max_block_size, cur_row_group_left_rows); for (size_t i = 0; i < header.columns(); i++) { columns[i] = castColumn( diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h index a682d724960..2f728a586a0 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h @@ -36,7 +36,7 @@ private: std::shared_ptr cur_row_group_reader; ParquetColReaders column_readers; - UInt64 max_block_rows; + UInt64 max_block_size; std::vector parquet_col_indice; std::vector row_groups_indices; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 008b7b41b57..7db332dbbbe 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -424,14 +424,13 @@ void ParquetBlockInputFormat::initializeIfNeeded() auto row_group_meta = metadata->RowGroup(row_group_idx); for (int column_index : column_indices) { - auto column = row_group_meta->ColumnChunk(column_index); total_size += row_group_meta->ColumnChunk(column_index)->total_uncompressed_size(); } if (!total_size || !format_settings.parquet.prefer_block_bytes) return 0; auto average_row_bytes = static_cast(total_size) / row_group_meta->num_rows(); const size_t preferred_num_rows = static_cast(format_settings.parquet.prefer_block_bytes/average_row_bytes); const size_t MIN_ROW_NUM = 128; - return std::min(std::max(preferred_num_rows, MIN_ROW_NUM), format_settings.parquet.max_block_rows); + return std::min(std::max(preferred_num_rows, MIN_ROW_NUM), format_settings.parquet.max_block_size); }; for (int row_group = 0; row_group < num_row_groups; ++row_group) @@ -454,7 +453,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() row_group_batches.back().total_rows += metadata->RowGroup(row_group)->num_rows(); row_group_batches.back().total_bytes_compressed += metadata->RowGroup(row_group)->total_compressed_size(); auto rows = adative_chunk_size(row_group); - row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_rows; + row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_size; } } diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql index 9e57f2dd733..96d02035fcd 100644 --- a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql @@ -6,7 +6,7 @@ INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() SELECT max(blockSize()) FROM test_parquet; DROP TABLE IF EXISTS test_parquet; -CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_max_block_rows=16; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_max_block_size=16; INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); SELECT max(blockSize()) FROM test_parquet; From c9b929216895503bc41c4b8a07ecbeff95cbb048 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 29 May 2024 15:22:42 +0800 Subject: [PATCH 182/856] fix build error --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 7db332dbbbe..0c1b8e5c3d5 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -430,7 +430,8 @@ void ParquetBlockInputFormat::initializeIfNeeded() auto average_row_bytes = static_cast(total_size) / row_group_meta->num_rows(); const size_t preferred_num_rows = static_cast(format_settings.parquet.prefer_block_bytes/average_row_bytes); const size_t MIN_ROW_NUM = 128; - return std::min(std::max(preferred_num_rows, MIN_ROW_NUM), format_settings.parquet.max_block_size); + // size_t != UInt64 in darwin + return std::min(std::max(preferred_num_rows, MIN_ROW_NUM), static_cast(format_settings.parquet.max_block_size)); }; for (int row_group = 0; row_group < num_row_groups; ++row_group) From f85030f48150a294b0b5751307a81d366eea6991 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 29 May 2024 12:29:17 +0200 Subject: [PATCH 183/856] Better --- src/Coordination/KeeperSnapshotManagerS3.cpp | 2 +- src/Coordination/Standalone/Context.cpp | 6 +- src/Coordination/Standalone/Context.h | 4 +- .../ObjectStorages/ObjectStorageFactory.cpp | 6 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 5 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 5 +- src/Disks/ObjectStorages/S3/diskSettings.h | 1 - src/IO/S3Common.cpp | 120 +++++++++++------- src/IO/S3Common.h | 20 ++- src/IO/S3Settings.cpp | 4 +- .../ObjectStorage/S3/Configuration.cpp | 4 +- 11 files changed, 103 insertions(+), 74 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index db1f84a047f..a24ad81fd64 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -66,7 +66,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo } const auto & settings = Context::getGlobalContextInstance()->getSettingsRef(); - auto auth_settings = S3::AuthSettings(config, settings, true, config_prefix); + auto auth_settings = S3::AuthSettings(config, settings, config_prefix); String endpoint = macros->expand(config.getString(config_prefix + ".endpoint")); auto new_uri = S3::URI{endpoint}; diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 2802d51ae26..2128a78387f 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -146,7 +146,7 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes - std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage + std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage mutable std::mutex keeper_dispatcher_mutex; mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); @@ -455,14 +455,14 @@ std::shared_ptr Context::getZooKeeper() const throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot connect to ZooKeeper from Keeper"); } -const StorageS3Settings & Context::getStorageS3Settings() const +const S3SettingsByEndpoint & Context::getStorageS3Settings() const { std::lock_guard lock(shared->mutex); if (!shared->storage_s3_settings) { const auto & config = shared->config ? *shared->config : Poco::Util::Application::instance().config(); - shared->storage_s3_settings.emplace().loadFromConfig("s3", config, getSettingsRef()); + shared->storage_s3_settings.emplace().loadFromConfig(config, "s3", getSettingsRef()); } return *shared->storage_s3_settings; diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index 7e4d1794f7d..bba52f8d493 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -37,7 +37,7 @@ class FilesystemCacheLog; class FilesystemReadPrefetchesLog; class BlobStorageLog; class IOUringReader; -class StorageS3Settings; +class S3SettingsByEndpoint; /// A small class which owns ContextShared. /// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete. @@ -163,7 +163,7 @@ public: zkutil::ZooKeeperPtr getZooKeeper() const; - const StorageS3Settings & getStorageS3Settings() const; + const S3SettingsByEndpoint & getStorageS3Settings() const; const String & getUserName() const { static std::string user; return user; } diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 14c0d656cbf..a2fff253465 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -191,7 +191,7 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto endpoint = getEndpoint(config, config_prefix, context); - auto settings = getSettings(config, config_prefix, context, endpoint, /* for_disk_s3 */true, /* validate_settings */true); + auto settings = getSettings(config, config_prefix, context, endpoint, /* validate_settings */true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); @@ -228,7 +228,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto endpoint = getEndpoint(config, config_prefix, context); - auto settings = getSettings(config, config_prefix, context, endpoint, /* for_disk_s3 */true, /* validate_settings */true); + auto settings = getSettings(config, config_prefix, context, endpoint, /* validate_settings */true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); @@ -263,7 +263,7 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto endpoint = getEndpoint(config, config_prefix, context); - auto settings = getSettings(config, config_prefix, context, endpoint, /* for_disk_s3 */true, /* validate_settings */true); + auto settings = getSettings(config, config_prefix, context, endpoint, /* validate_settings */true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 84af340e5d0..60bd9b1673c 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -574,7 +574,7 @@ void S3ObjectStorage::applyNewSettings( ContextPtr context, const ApplyNewSettingsOptions & options) { - auto settings_from_config = getSettings(config, config_prefix, context, uri.endpoint, for_disk_s3, context->getSettingsRef().s3_validate_request_settings); + auto settings_from_config = getSettings(config, config_prefix, context, uri.uri_str, context->getSettingsRef().s3_validate_request_settings); auto modified_settings = std::make_unique(*s3_settings.get()); modified_settings->auth_settings.updateIfChanged(settings_from_config->auth_settings); modified_settings->request_settings.updateIfChanged(settings_from_config->request_settings); @@ -598,7 +598,8 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( const std::string & config_prefix, ContextPtr context) { - auto new_s3_settings = getSettings(config, config_prefix, context, uri.endpoint, for_disk_s3, true); + const auto & settings = context->getSettingsRef(); + auto new_s3_settings = getSettings(config, config_prefix, context, uri.uri_str, settings.s3_validate_request_settings); auto new_client = getClient(uri, *new_s3_settings, context, for_disk_s3); auto new_uri{uri}; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 14bb5f05071..bdaacbf62cd 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -35,12 +35,11 @@ std::unique_ptr getSettings( const String & config_prefix, ContextPtr context, const std::string & endpoint, - bool for_disk_s3, bool validate_settings) { const auto & settings = context->getSettingsRef(); - auto auth_settings = S3::AuthSettings(config, settings, for_disk_s3, for_disk_s3 ? config_prefix : ""); - auto request_settings = S3::RequestSettings(config, settings, for_disk_s3, validate_settings, for_disk_s3 ? config_prefix : ""); + auto auth_settings = S3::AuthSettings(config, settings, config_prefix); + auto request_settings = S3::RequestSettings(config, settings, config_prefix, validate_settings); request_settings.proxy_resolver = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( ProxyConfiguration::protocolFromString(S3::URI(endpoint).uri.getScheme()), config_prefix, config); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 41aa85991a7..aa427bee41a 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -19,7 +19,6 @@ std::unique_ptr getSettings( const String & config_prefix, ContextPtr context, const std::string & endpoint, - bool for_disk_s3, bool validate_settings); std::unique_ptr getClient( diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index ef5108612ec..a545d12aade 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -103,48 +103,45 @@ ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, c } template -static void updateS3SettingsFromConfig( - Settings & s3_settings, +static bool setValueFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & path, + typename Settings::SettingFieldRef & field) +{ + if (!config.has(path)) + return false; + + auto which = field.getValue().getType(); + if (isInt64OrUInt64FieldType(which)) + field.setValue(config.getUInt64(path)); + else if (which == Field::Types::String) + field.setValue(config.getString(path)); + else if (which == Field::Types::Bool) + field.setValue(config.getBool(path)); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); + + return true; +} + +AuthSettings::AuthSettings( const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, - bool for_disk_s3, - const std::string & disk_config_prefix) + const std::string & config_prefix, + const std::string & fallback_config_prefix) { - if (for_disk_s3 && disk_config_prefix.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk config path cannot be empty"); + if (config_prefix.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config path cannot be empty"); - auto update_value_if_exists = [&](const std::string & path, Settings::SettingFieldRef & field) -> bool + for (auto & field : allMutable()) { - if (!config.has(path)) - return false; + auto path = fmt::format("{}.{}", config_prefix, field.getName()); + auto fallback_path = fallback_config_prefix.empty() ? "" : fmt::format("{}.{}", fallback_config_prefix, field.getName()); - auto which = field.getValue().getType(); - if (isInt64OrUInt64FieldType(which)) - field.setValue(config.getUInt64(path)); - else if (which == Field::Types::String) - field.setValue(config.getString(path)); - else if (which == Field::Types::Bool) - field.setValue(config.getBool(path)); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type: {}", field.getTypeName()); - return true; - }; - - for (auto & field : s3_settings.allMutable()) - { - std::string path, fallback_path; - if (for_disk_s3) - { - path = fmt::format("{}.s3_{}", disk_config_prefix, field.getName()); - fallback_path = fmt::format("s3.{}", field.getName()); - } - else - path = fmt::format("s3.{}", field.getName()); - - bool updated = update_value_if_exists(path, field); + bool updated = setValueFromConfig(config, path, field); if (!updated && !fallback_path.empty()) - updated = update_value_if_exists(fallback_path, field); + updated = setValueFromConfig(config, fallback_path, field); if (!updated) { @@ -153,17 +150,7 @@ static void updateS3SettingsFromConfig( field.setValue(settings.get(setting_name)); } } -} -AuthSettings::AuthSettings( - const Poco::Util::AbstractConfiguration & config, - const DB::Settings & settings, - bool for_disk_s3, - const std::string & disk_config_prefix) -{ - updateS3SettingsFromConfig(*this, config, settings, for_disk_s3, disk_config_prefix); - - const auto config_prefix = for_disk_s3 ? disk_config_prefix : "s3"; headers = getHTTPHeaders(config_prefix, config); server_side_encryption_kms_config = getSSEKMSConfig(config_prefix, config); @@ -220,11 +207,48 @@ void AuthSettings::updateIfChanged(const AuthSettings & settings) RequestSettings::RequestSettings( const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, - bool for_disk_s3, - bool validate_settings, - const std::string & disk_config_path) + const std::string & config_prefix, + bool validate_settings) + : RequestSettings( + config, + settings, + config_prefix, + validate_settings, + config_prefix == "s3" ? "" : "s3_", /* setting_name_prefix */ + config_prefix == "s3" ? "" : "s3", /* fallback_config_prefix */ + "") /* fallback_setting_name_prefix */ { - updateS3SettingsFromConfig(*this, config, settings, for_disk_s3, disk_config_path); +} + +RequestSettings::RequestSettings( + const Poco::Util::AbstractConfiguration & config, + const DB::Settings & settings, + const std::string & config_prefix, + bool validate_settings, + const std::string & setting_name_prefix, + const std::string & fallback_config_prefix, + const std::string & fallback_setting_name_prefix) +{ + if (config_prefix.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config path cannot be empty"); + + for (auto & field : allMutable()) + { + auto path = fmt::format("{}.{}{}", config_prefix, setting_name_prefix, field.getName()); + auto fallback_path = fallback_config_prefix.empty() ? "" : fmt::format("{}.{}{}", fallback_config_prefix, fallback_setting_name_prefix, field.getName()); + + bool updated = setValueFromConfig(config, path, field); + + if (!updated && !fallback_path.empty()) + updated = setValueFromConfig(config, fallback_path, field); + + if (!updated) + { + auto setting_name = "s3_" + field.getName(); + if (settings.has(setting_name) && settings.isChanged(setting_name)) + field.setValue(settings.get(setting_name)); + } + } finishInit(settings, validate_settings); } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index c5b31c4b564..6391075a5ab 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -75,7 +75,8 @@ namespace S3 { /// We use s3 settings for DiskS3, StorageS3 (StorageS3Cluster, S3Queue, etc), BackupIO_S3, etc. /// 1. For DiskS3 we usually have configuration in disk section in configuration file. -/// All s3 related settings start with "s3_" prefix there. +/// REQUEST_SETTINGS, PART_UPLOAD_SETTINGS start with "s3_" prefix there, while AUTH_SETTINGS and CLIENT_SETTINGS do not +/// (does not make sense, but it happened this way). /// If some setting is absent from disk configuration, we look up for it in the "s3." server config section, /// where s3 settings no longer have "s3_" prefix like in disk configuration section. /// If the settings is absent there as well, we look up for it in Users config (where query/session settings are also updated). @@ -143,8 +144,8 @@ struct AuthSettings : public BaseSettings AuthSettings( const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, - bool for_disk_s3, - const std::string & disk_config_prefix = ""); + const std::string & config_prefix, + const std::string & fallback_config_prefix = ""); AuthSettings(const DB::Settings & settings); @@ -168,9 +169,8 @@ struct RequestSettings : public BaseSettings RequestSettings( const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, - bool for_disk_s3, - bool validate_settings = true, - const std::string & disk_config_path = ""); + const std::string & config_prefix, + bool validate_settings = true); /// Create request settings from DB::Settings. explicit RequestSettings(const DB::Settings & settings, bool validate_settings = true); @@ -190,6 +190,14 @@ struct RequestSettings : public BaseSettings std::shared_ptr proxy_resolver; private: + RequestSettings( + const Poco::Util::AbstractConfiguration & config, + const DB::Settings & settings, + const std::string & config_prefix, + bool validate_settings, + const std::string & setting_name_prefix, + const std::string & fallback_config_prefix, + const std::string & fallback_setting_name_prefix); void finishInit(const DB::Settings & settings, bool validate_settings); void normalizeSettings(); }; diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp index 85f30e7e316..0323da53117 100644 --- a/src/IO/S3Settings.cpp +++ b/src/IO/S3Settings.cpp @@ -28,8 +28,8 @@ void S3SettingsByEndpoint::loadFromConfig( if (config.has(endpoint_path)) { auto endpoint = config.getString(endpoint_path); - auto auth_settings = S3::AuthSettings(config, settings, /* for_disk_s3 */false, config_prefix); - auto request_settings = S3::RequestSettings(config, settings, /* for_disk_s3 */false, settings.s3_validate_request_settings, config_prefix); + auto auth_settings = S3::AuthSettings(config, settings, config_prefix); + auto request_settings = S3::RequestSettings(config, settings, config_prefix, settings.s3_validate_request_settings); s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); } } diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index d59b3e8ea06..1252c010769 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -106,11 +106,9 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, const auto & config = context->getConfigRef(); const auto & settings = context->getSettingsRef(); - const std::string config_prefix = "s3."; auto s3_settings = getSettings( - config, config_prefix, context, url.endpoint, /* for_disk_s3 */false, - settings.s3_validate_request_settings); + config, "s3"/* config_prefix */, context, url.endpoint, settings.s3_validate_request_settings); s3_settings->auth_settings.updateIfChanged(auth_settings); s3_settings->request_settings.updateIfChanged(request_settings); From e50584ee0898ec84ebbffe4003f6d6b5f15f60f4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 29 May 2024 10:42:57 +0000 Subject: [PATCH 184/856] Fix crash: empty plan from read --- src/Planner/PlannerJoinTree.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index d7aa2e8de24..a907bad0c60 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -879,7 +879,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres return true; }; - if (parallel_replicas_enabled_for_storage(storage, settings) && query_context->canUseParallelReplicasOnInitiator()) + /// query_plan can be empty if there is nothing to read + if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings) && query_context->canUseParallelReplicasOnInitiator()) { // (1) find read step QueryPlan::Node * node = query_plan.getRootNode(); @@ -898,10 +899,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } else { - if (prev_node->step) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Step is expected to be ReadFromMergeTree but it's {}", prev_node->step->getName()); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Step is expected to be ReadFromMergeTree, and wtf - last node with empty step"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Step is expected to be ReadFromMergeTree but it's {}", + prev_node->step->getName()); } } From 92e004394de811a9845a200b05126a088de305be Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 29 May 2024 12:53:28 +0200 Subject: [PATCH 185/856] Named collections in keeper --- .../NamedCollections/NamedCollectionUtils.cpp | 390 +++++++++++++----- .../named_collections_with_zookeeper.xml | 11 + .../test_named_collections/test.py | 63 ++- 3 files changed, 361 insertions(+), 103 deletions(-) create mode 100644 tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 21fa9b64c22..33962f96c7e 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -14,6 +14,9 @@ #include #include #include +#include +#include +#include #include #include @@ -29,6 +32,7 @@ namespace ErrorCodes extern const int NAMED_COLLECTION_ALREADY_EXISTS; extern const int NAMED_COLLECTION_DOESNT_EXIST; extern const int BAD_ARGUMENTS; + extern const int INVALID_CONFIG_PARAMETER; } namespace NamedCollectionUtils @@ -104,44 +108,45 @@ private: } }; +class INamedCollectionsStorage +{ +public: + virtual ~INamedCollectionsStorage() = default; -class LoadFromSQL : private WithContext + virtual bool exists(const std::string & path) const = 0; + + virtual std::vector list() const = 0; + + virtual std::string read(const std::string & path) const = 0; + + virtual void write(const std::string & path, const std::string & data, bool replace) = 0; + + virtual void remove(const std::string & path) = 0; + + virtual bool removeIfExists(const std::string & path) = 0; +}; + +using NamedCollectionsStoragePtr = std::unique_ptr; + + +class NamedCollectionsMetadata : private WithContext { private: - const std::string metadata_path; + NamedCollectionsStoragePtr storage; public: - explicit LoadFromSQL(ContextPtr context_) + NamedCollectionsMetadata(NamedCollectionsStoragePtr storage_, ContextPtr context_) : WithContext(context_) - , metadata_path(fs::weakly_canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY) - { - if (fs::exists(metadata_path)) - cleanup(); - } + , storage(std::move(storage_)) {} std::vector listCollections() const { - if (!fs::exists(metadata_path)) - return {}; - - std::vector collection_names; - fs::directory_iterator it{metadata_path}; - for (; it != fs::directory_iterator{}; ++it) - { - const auto & current_path = it->path(); - if (current_path.extension() == ".sql") - { - collection_names.push_back(it->path().stem()); - } - else - { - LOG_WARNING( - getLogger("NamedCollectionsLoadFromSQL"), - "Unexpected file {} in named collections directory", - current_path.filename().string()); - } - } - return collection_names; + auto paths = storage->list(); + std::vector collections; + collections.reserve(paths.size()); + for (const auto & path : paths) + collections.push_back(fs::path(path).stem()); + return collections; } NamedCollectionsMap getAll() const @@ -163,26 +168,19 @@ public: MutableNamedCollectionPtr get(const std::string & collection_name) const { - const auto query = readCreateQueryFromMetadata( - getMetadataPath(collection_name), - getContext()->getSettingsRef()); + const auto query = readCreateQuery(collection_name); return createNamedCollectionFromAST(query); } MutableNamedCollectionPtr create(const ASTCreateNamedCollectionQuery & query) { - writeCreateQueryToMetadata( - query, - getMetadataPath(query.collection_name), - getContext()->getSettingsRef()); - + writeCreateQuery(query); return createNamedCollectionFromAST(query); } void update(const ASTAlterNamedCollectionQuery & query) { - const auto path = getMetadataPath(query.collection_name); - auto create_query = readCreateQueryFromMetadata(path, getContext()->getSettings()); + auto create_query = readCreateQuery(query.collection_name); std::unordered_map result_changes_map; for (const auto & [name, value] : query.changes) @@ -236,31 +234,27 @@ public: "Named collection cannot be empty (collection name: {})", query.collection_name); - writeCreateQueryToMetadata( - create_query, - getMetadataPath(query.collection_name), - getContext()->getSettingsRef(), - true); + chassert(create_query.collection_name == query.collection_name); + writeCreateQuery(create_query, true); } void remove(const std::string & collection_name) { - auto collection_path = getMetadataPath(collection_name); - if (!fs::exists(collection_path)) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, - "Cannot remove collection `{}`, because it doesn't exist", - collection_name); - } - (void)fs::remove(collection_path); + storage->remove(getFileName(collection_name)); + } + + bool removeIfExists(const std::string & collection_name) + { + return storage->removeIfExists(getFileName(collection_name)); } private: - static constexpr auto NAMED_COLLECTIONS_METADATA_DIRECTORY = "named_collections"; + std::string getFileName(const std::string & collection_name) const + { + return escapeForFileName(collection_name) + ".sql"; + } - static MutableNamedCollectionPtr createNamedCollectionFromAST( - const ASTCreateNamedCollectionQuery & query) + static MutableNamedCollectionPtr createNamedCollectionFromAST(const ASTCreateNamedCollectionQuery & query) { const auto & collection_name = query.collection_name; const auto config = NamedCollectionConfiguration::createConfiguration(collection_name, query.changes, query.overridability); @@ -273,46 +267,83 @@ private: *config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true); } - std::string getMetadataPath(const std::string & collection_name) const + ASTCreateNamedCollectionQuery readCreateQuery(const std::string & collection_name) const { - return fs::path(metadata_path) / (escapeForFileName(collection_name) + ".sql"); - } - - /// Delete .tmp files. They could be left undeleted in case of - /// some exception or abrupt server restart. - void cleanup() - { - fs::directory_iterator it{metadata_path}; - std::vector files_to_remove; - for (; it != fs::directory_iterator{}; ++it) - { - const auto & current_path = it->path(); - if (current_path.extension() == ".tmp") - files_to_remove.push_back(current_path); - } - for (const auto & file : files_to_remove) - (void)fs::remove(file); - } - - static ASTCreateNamedCollectionQuery readCreateQueryFromMetadata( - const std::string & path, - const Settings & settings) - { - ReadBufferFromFile in(path); - std::string query; - readStringUntilEOF(query, in); + const auto path = getFileName(collection_name); + auto query = storage->read(path); ParserCreateNamedCollectionQuery parser; - auto ast = parseQuery(parser, query, "in file " + path, 0, settings.max_parser_depth, settings.max_parser_backtracks); + auto ast = parseQuery(parser, query, "in file " + path, 0, getContext()->getSettingsRef().max_parser_depth, getContext()->getSettingsRef().max_parser_backtracks); const auto & create_query = ast->as(); return create_query; } - void writeCreateQueryToMetadata( - const ASTCreateNamedCollectionQuery & query, - const std::string & path, - const Settings & settings, - bool replace = false) const + void writeCreateQuery(const ASTCreateNamedCollectionQuery & query, bool replace = false) + { + auto normalized_query = query.clone(); + auto & changes = typeid_cast(normalized_query.get())->changes; + ::sort( + changes.begin(), changes.end(), + [](const SettingChange & lhs, const SettingChange & rhs) { return lhs.name < rhs.name; }); + + storage->write(getFileName(query.collection_name), serializeAST(*normalized_query), replace); + } +}; + +class NamedCollectionsLocalStorage : public INamedCollectionsStorage, private WithContext +{ +private: + std::string root_path; + +public: + NamedCollectionsLocalStorage(ContextPtr context_, const std::string & path_) + : WithContext(context_) + , root_path(path_) + { + if (fs::exists(root_path)) + cleanup(); + } + + ~NamedCollectionsLocalStorage() override = default; + + std::vector list() const override + { + if (!fs::exists(root_path)) + return {}; + + std::vector elements; + for (fs::directory_iterator it{root_path}; it != fs::directory_iterator{}; ++it) + { + const auto & current_path = it->path(); + if (current_path.extension() == ".sql") + { + elements.push_back(it->path()); + } + else + { + LOG_WARNING( + getLogger("NamedCollectionsLocalStorage"), + "Unexpected file {} in named collections directory", + current_path.filename().string()); + } + } + return elements; + } + + bool exists(const std::string & path) const override + { + return fs::exists(getPath(path)); + } + + std::string read(const std::string & path) const override + { + ReadBufferFromFile in(getPath(path)); + std::string data; + readStringUntilEOF(data, in); + return data; + } + + void write(const std::string & path, const std::string & data, bool replace) override { if (!replace && fs::exists(path)) { @@ -322,22 +353,155 @@ private: path); } - fs::create_directories(metadata_path); + fs::create_directories(root_path); - auto tmp_path = path + ".tmp"; - String formatted_query = serializeAST(query); - WriteBufferFromFile out(tmp_path, formatted_query.size(), O_WRONLY | O_CREAT | O_EXCL); - writeString(formatted_query, out); + auto tmp_path = getPath(path + ".tmp"); + WriteBufferFromFile out(tmp_path, data.size(), O_WRONLY | O_CREAT | O_EXCL); + writeString(data, out); out.next(); - if (settings.fsync_metadata) + if (getContext()->getSettingsRef().fsync_metadata) out.sync(); out.close(); - fs::rename(tmp_path, path); + fs::rename(tmp_path, getPath(path)); + } + + void remove(const std::string & path) override + { + if (!removeIfExists(getPath(path))) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "Cannot remove `{}`, because it doesn't exist", path); + } + } + + bool removeIfExists(const std::string & path) override + { + return fs::remove(getPath(path)); + } + +private: + std::string getPath(const std::string & path) const + { + return fs::path(root_path) / path; + } + + /// Delete .tmp files. They could be left undeleted in case of + /// some exception or abrupt server restart. + void cleanup() + { + std::vector files_to_remove; + for (fs::directory_iterator it{root_path}; it != fs::directory_iterator{}; ++it) + { + const auto & current_path = it->path(); + if (current_path.extension() == ".tmp") + files_to_remove.push_back(current_path); + } + for (const auto & file : files_to_remove) + fs::remove(file); } }; + +class NamedCollectionsZooKeeperStorage : public INamedCollectionsStorage, private WithContext +{ +private: + std::string root_path; + mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; + +public: + NamedCollectionsZooKeeperStorage(ContextPtr context_, const std::string & path_) + : WithContext(context_) + , root_path(path_) + { + if (root_path.empty()) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Collections path cannot be empty"); + + if (root_path != "/" && root_path.back() == '/') + root_path.resize(root_path.size() - 1); + if (root_path.front() != '/') + root_path = "/" + root_path; + + auto client = getClient(); + if (root_path != "/" && !client->exists(root_path)) + { + client->createAncestors(root_path); + client->createIfNotExists(root_path, ""); + } + } + + ~NamedCollectionsZooKeeperStorage() override = default; + + std::vector list() const override + { + return getClient()->getChildren(root_path); + } + + bool exists(const std::string & path) const override + { + return getClient()->exists(getPath(path)); + } + + std::string read(const std::string & path) const override + { + return getClient()->get(getPath(path)); + } + + void write(const std::string & path, const std::string & data, bool replace) override + { + if (replace) + { + getClient()->createOrUpdate(getPath(path), data, zkutil::CreateMode::Persistent); + } + else + { + auto code = getClient()->tryCreate(getPath(path), data, zkutil::CreateMode::Persistent); + + if (code == Coordination::Error::ZNODEEXISTS) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Metadata file {} for named collection already exists", + path); + } + } + } + + void remove(const std::string & path) override + { + getClient()->remove(getPath(path)); + } + + bool removeIfExists(const std::string & path) override + { + auto code = getClient()->tryRemove(getPath(path)); + if (code == Coordination::Error::ZOK) + return true; + if (code == Coordination::Error::ZNONODE) + return false; + throw Coordination::Exception::fromPath(code, getPath(path)); + } + +private: + zkutil::ZooKeeperPtr getClient() const + { + if (!zookeeper_client || zookeeper_client->expired()) + { + zookeeper_client = getContext()->getZooKeeper(); + zookeeper_client->sync(root_path); + } + return zookeeper_client; + } + + std::string getPath(const std::string & path) const + { + return fs::path(root_path) / path; + } +}; + + std::unique_lock lockNamedCollectionsTransaction() { static std::mutex transaction_lock; @@ -371,9 +535,35 @@ void reloadFromConfig(const Poco::Util::AbstractConfiguration & config) is_loaded_from_config = true; } +auto getNamedCollectionsStorage(ContextPtr context) +{ + static const std::string storage_config_path = "named_collections_storage"; + + const auto & config = context->getConfigRef(); + const auto storage = config.getString(storage_config_path + ".type", "local"); + + if (storage == "local") + { + const auto path = config.getString(storage_config_path + ".path", fs::path(context->getPath()) / "named_collections"); + return NamedCollectionsMetadata( + std::make_unique(context, path), context); + } + if (storage == "zookeeper") + { + return NamedCollectionsMetadata( + std::make_unique( + context, config.getString(storage_config_path + ".path")), + context); + } + + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "Unknown storage for named collections: {}", storage); +} + void loadFromSQLUnlocked(ContextPtr context, std::unique_lock &) { - auto named_collections = LoadFromSQL(context).getAll(); + auto named_collections = getNamedCollectionsStorage(context).getAll(); LOG_TRACE( getLogger("NamedCollectionsUtils"), "Loaded {} collections from SQL", named_collections.size()); @@ -421,7 +611,7 @@ void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context } return; } - LoadFromSQL(context).remove(query.collection_name); + getNamedCollectionsStorage(context).remove(query.collection_name); instance.remove(query.collection_name); } @@ -441,7 +631,7 @@ void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr conte } return; } - instance.add(query.collection_name, LoadFromSQL(context).create(query)); + instance.add(query.collection_name, getNamedCollectionsStorage(context).create(query)); } void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context) @@ -460,7 +650,7 @@ void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr contex } return; } - LoadFromSQL(context).update(query); + getNamedCollectionsStorage(context).update(query); auto collection = instance.getMutable(query.collection_name); auto collection_lock = collection->lock(); diff --git a/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml new file mode 100644 index 00000000000..4111bdec4eb --- /dev/null +++ b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml @@ -0,0 +1,11 @@ + + + zookeeper + /named_collections_path/ + + + + value1 + + + diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index cbb8c94c701..12bf11bad0b 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -9,6 +9,7 @@ NAMED_COLLECTIONS_CONFIG = os.path.join( SCRIPT_DIR, "./configs/config.d/named_collections.xml" ) +ZK_PATH = "/named_collections_path" @pytest.fixture(scope="module") def cluster(): @@ -24,6 +25,17 @@ def cluster(): ], stay_alive=True, ) + cluster.add_instance( + "node_with_keeper", + main_configs=[ + "configs/config.d/named_collections_with_zookeeper.xml", + ], + user_configs=[ + "configs/users.d/users.xml", + ], + stay_alive=True, + with_zookeeper=True, + ) cluster.add_instance( "node_only_named_collection_control", main_configs=[ @@ -53,7 +65,6 @@ def cluster(): finally: cluster.shutdown() - def replace_in_server_config(node, old, new): node.replace_in_config( "/etc/clickhouse-server/config.d/named_collections.xml", @@ -447,8 +458,16 @@ def test_config_reload(cluster): ) -def test_sql_commands(cluster): - node = cluster.instances["node"] +@pytest.mark.parametrize("with_keeper", [False, True]) +def test_sql_commands(cluster, with_keeper): + zk = None + node = None + if with_keeper: + node = cluster.instances["node_with_keeper"] + zk = cluster.get_kazoo_client("zoo1") + else: + node = cluster.instances["node"] + assert "1" == node.query("select count() from system.named_collections").strip() node.query("CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'") @@ -479,6 +498,14 @@ def test_sql_commands(cluster): "select collection['key2'] from system.named_collections where name = 'collection2'" ).strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 1, key2 = 'value2'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) check_created() node.restart_clickhouse() @@ -508,6 +535,15 @@ def test_sql_commands(cluster): ).strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 4, key2 = 'value2', key3 = 'value3'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + check_altered() node.restart_clickhouse() check_altered() @@ -522,6 +558,15 @@ def test_sql_commands(cluster): ).strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 4, key3 = 'value3'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + check_deleted() node.restart_clickhouse() check_deleted() @@ -552,6 +597,15 @@ def test_sql_commands(cluster): ).strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key3 = 3, key4 = 'value4'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + check_altered_and_deleted() node.restart_clickhouse() check_altered_and_deleted() @@ -564,6 +618,9 @@ def test_sql_commands(cluster): "collection1" == node.query("select name from system.named_collections").strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 0 == len(children) check_dropped() node.restart_clickhouse() From 4112efc2ced3f768e321f4e70d122cfed7444342 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 29 May 2024 11:05:47 +0000 Subject: [PATCH 186/856] Automatic style fix --- tests/integration/test_named_collections/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 12bf11bad0b..3460ed2e564 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -11,6 +11,7 @@ NAMED_COLLECTIONS_CONFIG = os.path.join( ZK_PATH = "/named_collections_path" + @pytest.fixture(scope="module") def cluster(): try: @@ -65,6 +66,7 @@ def cluster(): finally: cluster.shutdown() + def replace_in_server_config(node, old, new): node.replace_in_config( "/etc/clickhouse-server/config.d/named_collections.xml", From 1175c3d27ed157d68246db5b46a4dac638e2e474 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 May 2024 14:26:16 +0000 Subject: [PATCH 187/856] slightly better --- src/Storages/MergeTree/MergeTask.cpp | 54 +++++++++++++--------------- src/Storages/MergeTree/MergeTask.h | 4 +-- 2 files changed, 25 insertions(+), 33 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 84567281291..d59501f033f 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -5,15 +5,9 @@ #include #include -#include "Common/DateLUT.h" #include #include #include "Core/NamesAndTypes.h" -#include "Storages/ColumnsDescription.h" -#include "Storages/IndicesDescription.h" -#include "Storages/MergeTree/MergeTreeIndices.h" -#include "Storages/ProjectionsDescription.h" -#include "Storages/StorageInMemoryMetadata.h" #include #include #include @@ -74,20 +68,6 @@ static Statistics getStatisticsForColumns( return statistics; } -static void addSkipIndexesExpressions( - QueryPipelineBuilder & builder, - const IndicesDescription & indexes, - const StorageMetadataPtr & metadata_snapshot, - const ContextPtr & context) -{ - builder.addTransform(std::make_shared( - builder.getHeader(), - indexes.getSingleExpressionForIndices(metadata_snapshot->getColumns(), - context))); - - builder.addTransform(std::make_shared(builder.getHeader())); -} - static void addMissedColumnsToSerializationInfos( size_t num_rows_in_parts, const Names & part_columns, @@ -147,6 +127,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu { auto index_columns = index.expression->getRequiredColumns(); + /// Calculate indexes that depend only on one column on vertical + /// stage and other indexes on horizonatal stage of merge. if (index_columns.size() == 1) { const auto & column_name = index_columns.front(); @@ -167,6 +149,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu { global_ctx->merging_columns.emplace_back(column); + /// If column is in horizontal stage we need to calculate its indexes on horizontal stage as well auto it = global_ctx->skip_indexes_by_column.find(column.name); if (it != global_ctx->skip_indexes_by_column.end()) { @@ -630,12 +613,12 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const pipes.emplace_back(std::move(pipe)); } - bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; - auto pipe = Pipe::unitePipes(std::move(pipes)); ctx->rows_sources_read_buf->seek(0, 0); const auto data_settings = global_ctx->data->getSettings(); + bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; + auto transform = std::make_unique( pipe.getHeader(), pipe.numOutputPorts(), @@ -644,9 +627,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const data_settings->merge_max_block_size_bytes, is_result_sparse); - QueryPipelineBuilder builder; - builder.init(std::move(pipe)); - builder.addTransform(std::move(transform)); + pipe.addTransform(std::move(transform)); MergeTreeIndices indexes_to_recalc; auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name); @@ -654,10 +635,16 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const if (indexes_it != global_ctx->skip_indexes_by_column.end()) { indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); - addSkipIndexesExpressions(builder, indexes_it->second, global_ctx->metadata_snapshot, global_ctx->data->getContext()); + + pipe.addTransform(std::make_shared( + pipe.getHeader(), + indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), + global_ctx->data->getContext()))); + + pipe.addTransform(std::make_shared(pipe.getHeader())); } - ctx->column_parts_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); + ctx->column_parts_pipeline = QueryPipeline(std::move(pipe)); /// Dereference unique_ptr ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback( @@ -1161,7 +1148,14 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() } if (!global_ctx->merging_skip_indexes.empty()) - addSkipIndexesExpressions(*builder, global_ctx->merging_skip_indexes, global_ctx->metadata_snapshot, global_ctx->data->getContext()); + { + builder->addTransform(std::make_shared( + builder->getHeader(), + global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), + global_ctx->data->getContext()))); + + builder->addTransform(std::make_shared(builder->getHeader())); + } if (!subqueries.empty()) builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), global_ctx->context); @@ -1210,7 +1204,7 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; - bool enough_columns = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate; + bool enough_ordinary_cols = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate; bool enough_total_rows = total_rows_count >= data_settings->vertical_merge_algorithm_min_rows_to_activate; @@ -1218,7 +1212,7 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm bool no_parts_overflow = global_ctx->future_part->parts.size() <= RowSourcePart::MAX_PARTS; - auto merge_alg = (is_supported_storage && enough_total_rows && enough_total_bytes && enough_columns && no_parts_overflow) ? + auto merge_alg = (is_supported_storage && enough_total_rows && enough_total_bytes && enough_ordinary_cols && no_parts_overflow) ? MergeAlgorithm::Vertical : MergeAlgorithm::Horizontal; return merge_alg; diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 7c509699903..a2f18f8807f 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -2,11 +2,8 @@ #include #include -#include #include -#include "Storages/MergeTree/MergeTreeIndices.h" -#include "Storages/Statistics/Statistics.h" #include #include @@ -27,6 +24,7 @@ #include #include #include +#include namespace DB From f0e9d6b459cfee4331861d4f0e3c92c1e9d67c72 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 14:30:48 +0000 Subject: [PATCH 188/856] 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 28e71af95cb4008ce791dceaf381d84e32d716e5 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 29 May 2024 14:55:32 +0000 Subject: [PATCH 189/856] disable precise memory tracking to avoid memory_exceeded exception in test --- tests/integration/test_failed_async_inserts/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_failed_async_inserts/test.py b/tests/integration/test_failed_async_inserts/test.py index ecb506c36bc..3a6159107ac 100644 --- a/tests/integration/test_failed_async_inserts/test.py +++ b/tests/integration/test_failed_async_inserts/test.py @@ -46,7 +46,7 @@ def test_failed_async_inserts(started_cluster): ) select_query = ( - "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery'" + "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = 4194304" ) assert node.query(select_query) == "4\n" From f51a145437df6f173d67e5fc7f1259c1e0154a98 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 15:00:22 +0000 Subject: [PATCH 190/856] 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 c083896c590d547e4ed3649259d4ef4b00fd91d0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 29 May 2024 15:06:31 +0000 Subject: [PATCH 191/856] Automatic style fix --- tests/integration/test_failed_async_inserts/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_failed_async_inserts/test.py b/tests/integration/test_failed_async_inserts/test.py index 3a6159107ac..2bb56b250ea 100644 --- a/tests/integration/test_failed_async_inserts/test.py +++ b/tests/integration/test_failed_async_inserts/test.py @@ -45,9 +45,7 @@ def test_failed_async_inserts(started_cluster): ignore_error=True, ) - select_query = ( - "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = 4194304" - ) + select_query = "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = 4194304" assert node.query(select_query) == "4\n" From e2b7ca7d1137aed5c449eb9750b97e9be567681b Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 29 May 2024 11:54:31 -0400 Subject: [PATCH 192/856] Fix restore from backup for definers --- src/Interpreters/InterpreterCreateQuery.cpp | 8 +++---- src/Interpreters/InterpreterCreateQuery.h | 2 +- .../test_backup_restore_new/test.py | 24 +++++++++++++++++++ 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b30fc8bc092..bf6c4f68947 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1086,7 +1086,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.sql_security = std::make_shared(); if (create.sql_security) - processSQLSecurityOption(getContext(), create.sql_security->as(), create.attach, create.is_materialized_view); + processSQLSecurityOption(getContext(), create.sql_security->as(), create.attach, create.is_materialized_view, is_restore_from_backup); DDLGuardPtr ddl_guard; @@ -1880,7 +1880,7 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr } } -void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view) +void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view, bool is_restore_from_backup_) { /// If no SQL security is specified, apply default from default_*_view_sql_security setting. if (!sql_security.type) @@ -1921,7 +1921,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ } /// Checks the permissions for the specified definer user. - if (sql_security.definer && !sql_security.is_definer_current_user && !is_attach) + if (sql_security.definer && !sql_security.is_definer_current_user && !is_attach && !is_restore_from_backup_) { const auto definer_name = sql_security.definer->toString(); @@ -1931,7 +1931,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ context_->checkAccess(AccessType::SET_DEFINER, definer_name); } - if (sql_security.type == SQLSecurityType::NONE && !is_attach) + if (sql_security.type == SQLSecurityType::NONE && !is_attach && !is_restore_from_backup_) context_->checkAccess(AccessType::ALLOW_SQL_SECURITY_NONE); } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index be4a10eaf1d..c4bfa7b36d8 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -82,7 +82,7 @@ public: void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; /// Check access right, validate definer statement and replace `CURRENT USER` with actual name. - static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach = false, bool is_materialized_view = false); + static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach = false, bool is_materialized_view = false, bool is_restore_from_backup_ = false); private: struct TableProperties diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index ef9e536976b..a6621dce231 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -168,6 +168,30 @@ def test_restore_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" +def test_restore_materialized_view_with_definer(): + instance.query("CREATE DATABASE test") + instance.query("CREATE TABLE test.test_table (s String) ENGINE = MergeTree ORDER BY s") + instance.query("CREATE USER u1") + instance.query("GRANT SELECT ON *.* TO u1") + instance.query("GRANT INSERT ON *.* TO u1") + + instance.query( + """ + CREATE MATERIALIZED VIEW test.test_mv_1 (s String) + ENGINE = MergeTree ORDER BY s + DEFINER = u1 SQL SECURITY DEFINER + AS SELECT * FROM test.test_table + """ + ) + + backup_name = new_backup_name() + instance.query(f"BACKUP DATABASE test TO {backup_name}") + instance.query("DROP DATABASE test") + instance.query("DROP USER u1") + + instance.query(f"RESTORE DATABASE test FROM {backup_name}") + + @pytest.mark.parametrize( "engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"] ) From b908421259dab2849c5da8c01d8909ba8966e6c1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 29 May 2024 16:05:50 +0000 Subject: [PATCH 193/856] Automatic style fix --- tests/integration/test_backup_restore_new/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index a6621dce231..f5a63cd69a3 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -170,7 +170,9 @@ def test_restore_table(engine): def test_restore_materialized_view_with_definer(): instance.query("CREATE DATABASE test") - instance.query("CREATE TABLE test.test_table (s String) ENGINE = MergeTree ORDER BY s") + instance.query( + "CREATE TABLE test.test_table (s String) ENGINE = MergeTree ORDER BY s" + ) instance.query("CREATE USER u1") instance.query("GRANT SELECT ON *.* TO u1") instance.query("GRANT INSERT ON *.* TO u1") From d351c05243cc42dd05b3a4edf90dfe2044786e9a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 16:13:21 +0000 Subject: [PATCH 194/856] 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 f69918dba0f0dcd3bd90f58691635423d5c3c0b5 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 29 May 2024 16:03:29 -0400 Subject: [PATCH 195/856] skip_check_permissions --- src/Interpreters/InterpreterCreateQuery.cpp | 8 ++++---- src/Interpreters/InterpreterCreateQuery.h | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index bf6c4f68947..a3eaced0d02 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1086,7 +1086,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.sql_security = std::make_shared(); if (create.sql_security) - processSQLSecurityOption(getContext(), create.sql_security->as(), create.attach, create.is_materialized_view, is_restore_from_backup); + processSQLSecurityOption(getContext(), create.sql_security->as(), create.is_materialized_view, /* skip_check_permissions= */ is_restore_from_backup || create.attach); DDLGuardPtr ddl_guard; @@ -1880,7 +1880,7 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr } } -void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view, bool is_restore_from_backup_) +void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_materialized_view, bool skip_check_permissions) { /// If no SQL security is specified, apply default from default_*_view_sql_security setting. if (!sql_security.type) @@ -1921,7 +1921,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ } /// Checks the permissions for the specified definer user. - if (sql_security.definer && !sql_security.is_definer_current_user && !is_attach && !is_restore_from_backup_) + if (sql_security.definer && !sql_security.is_definer_current_user && !skip_check_permissions) { const auto definer_name = sql_security.definer->toString(); @@ -1931,7 +1931,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ context_->checkAccess(AccessType::SET_DEFINER, definer_name); } - if (sql_security.type == SQLSecurityType::NONE && !is_attach && !is_restore_from_backup_) + if (sql_security.type == SQLSecurityType::NONE && !skip_check_permissions) context_->checkAccess(AccessType::ALLOW_SQL_SECURITY_NONE); } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index c4bfa7b36d8..70ef29e6b07 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -82,7 +82,7 @@ public: void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; /// Check access right, validate definer statement and replace `CURRENT USER` with actual name. - static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach = false, bool is_materialized_view = false, bool is_restore_from_backup_ = false); + static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_materialized_view = false, bool skip_check_permissions = false); private: struct TableProperties From d86580ef049fc402d48808e3c125a61f824ed40f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 20:37:49 +0000 Subject: [PATCH 196/856] 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 197/856] 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 198/856] 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 199/856] 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 bfd61043cd346cc1ee8853eb611a6e815a7aafe1 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 29 May 2024 18:10:00 +0800 Subject: [PATCH 200/856] fix ut --- .../03164_adapting_parquet_reader_output_size.reference | 2 +- .../0_stateless/03164_adapting_parquet_reader_output_size.sql | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference index 332202dd23b..900342c13b3 100644 --- a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference @@ -1,4 +1,4 @@ 65409 16 128 -2183 +2235 diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql index 96d02035fcd..330e1928241 100644 --- a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql @@ -1,5 +1,7 @@ -- Tags: no-fasttest, no-parallel +set max_insert_threads=1; + DROP TABLE IF EXISTS test_parquet; CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet); INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); @@ -16,7 +18,7 @@ INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() SELECT max(blockSize()) FROM test_parquet; DROP TABLE IF EXISTS test_parquet; -CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_prefer_block_bytes=30000; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_prefer_block_bytes=30720; INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); SELECT max(blockSize()) FROM test_parquet; From cadf9d466664ceb693dbebb31a3f3df57af84c8b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 10:41:36 +0000 Subject: [PATCH 201/856] 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 202/856] 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 203/856] 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 dd34d316c51789048bdbf744747d45d11f806bc9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 May 2024 16:54:11 +0200 Subject: [PATCH 204/856] Fix tests --- src/IO/tests/gtest_writebuffer_s3.cpp | 2 +- src/Storages/ObjectStorage/S3/Configuration.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 1e1fdc02060..3c1af6538ad 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -547,7 +547,7 @@ public: std::unique_ptr getWriteBuffer(String file_name = "file") { S3::RequestSettings request_settings; - request_settings.updateFromSettings(settings, /* if_changed */true); + request_settings.updateFromSettings(settings, /* if_changed */true, /* validate_settings */false); client->resetCounters(); diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 1252c010769..244d233b302 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -108,7 +108,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, const auto & settings = context->getSettingsRef(); auto s3_settings = getSettings( - config, "s3"/* config_prefix */, context, url.endpoint, settings.s3_validate_request_settings); + config, "s3"/* config_prefix */, context, url.uri_str, settings.s3_validate_request_settings); s3_settings->auth_settings.updateIfChanged(auth_settings); s3_settings->request_settings.updateIfChanged(request_settings); From e864be5a66cfba3d64b424c056a7b3d8898ab775 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 May 2024 17:25:22 +0200 Subject: [PATCH 205/856] Move NamedCollectionsFactory to a separate file --- .../NamedCollections/NamedCollectionUtils.cpp | 1 + .../NamedCollections/NamedCollections.cpp | 157 ---------------- .../NamedCollections/NamedCollections.h | 55 ------ .../NamedCollectionsFactory.cpp | 169 ++++++++++++++++++ .../NamedCollectionsFactory.h | 57 ++++++ src/Common/tests/gtest_named_collections.cpp | 2 +- .../Cached/registerDiskCache.cpp | 2 +- src/Storages/Kafka/StorageKafka.cpp | 1 + src/Storages/NamedCollectionsHelpers.cpp | 1 + .../System/StorageSystemNamedCollections.cpp | 2 +- 10 files changed, 232 insertions(+), 215 deletions(-) create mode 100644 src/Common/NamedCollections/NamedCollectionsFactory.cpp create mode 100644 src/Common/NamedCollections/NamedCollectionsFactory.h diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 33962f96c7e..58c4bcde2b6 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include diff --git a/src/Common/NamedCollections/NamedCollections.cpp b/src/Common/NamedCollections/NamedCollections.cpp index 6ee47fd6523..cc12e78095c 100644 --- a/src/Common/NamedCollections/NamedCollections.cpp +++ b/src/Common/NamedCollections/NamedCollections.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB @@ -22,162 +21,6 @@ namespace ErrorCodes namespace Configuration = NamedCollectionConfiguration; - -NamedCollectionFactory & NamedCollectionFactory::instance() -{ - static NamedCollectionFactory instance; - return instance; -} - -bool NamedCollectionFactory::exists(const std::string & collection_name) const -{ - std::lock_guard lock(mutex); - return existsUnlocked(collection_name, lock); -} - -bool NamedCollectionFactory::existsUnlocked( - const std::string & collection_name, - std::lock_guard & /* lock */) const -{ - return loaded_named_collections.contains(collection_name); -} - -NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_name) const -{ - std::lock_guard lock(mutex); - auto collection = tryGetUnlocked(collection_name, lock); - if (!collection) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, - "There is no named collection `{}`", - collection_name); - } - return collection; -} - -NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection_name) const -{ - std::lock_guard lock(mutex); - return tryGetUnlocked(collection_name, lock); -} - -MutableNamedCollectionPtr NamedCollectionFactory::getMutable( - const std::string & collection_name) const -{ - std::lock_guard lock(mutex); - auto collection = tryGetUnlocked(collection_name, lock); - if (!collection) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, - "There is no named collection `{}`", - collection_name); - } - else if (!collection->isMutable()) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE, - "Cannot get collection `{}` for modification, " - "because collection was defined as immutable", - collection_name); - } - return collection; -} - -MutableNamedCollectionPtr NamedCollectionFactory::tryGetUnlocked( - const std::string & collection_name, - std::lock_guard & /* lock */) const -{ - auto it = loaded_named_collections.find(collection_name); - if (it == loaded_named_collections.end()) - return nullptr; - return it->second; -} - -void NamedCollectionFactory::add( - const std::string & collection_name, - MutableNamedCollectionPtr collection) -{ - std::lock_guard lock(mutex); - addUnlocked(collection_name, collection, lock); -} - -void NamedCollectionFactory::add(NamedCollectionsMap collections) -{ - std::lock_guard lock(mutex); - for (const auto & [collection_name, collection] : collections) - addUnlocked(collection_name, collection, lock); -} - -void NamedCollectionFactory::addUnlocked( - const std::string & collection_name, - MutableNamedCollectionPtr collection, - std::lock_guard & /* lock */) -{ - auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection); - if (!inserted) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, - "A named collection `{}` already exists", - collection_name); - } -} - -void NamedCollectionFactory::remove(const std::string & collection_name) -{ - std::lock_guard lock(mutex); - bool removed = removeIfExistsUnlocked(collection_name, lock); - if (!removed) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, - "There is no named collection `{}`", - collection_name); - } -} - -void NamedCollectionFactory::removeIfExists(const std::string & collection_name) -{ - std::lock_guard lock(mutex); - removeIfExistsUnlocked(collection_name, lock); // NOLINT -} - -bool NamedCollectionFactory::removeIfExistsUnlocked( - const std::string & collection_name, - std::lock_guard & lock) -{ - auto collection = tryGetUnlocked(collection_name, lock); - if (!collection) - return false; - - if (!collection->isMutable()) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE, - "Cannot get collection `{}` for modification, " - "because collection was defined as immutable", - collection_name); - } - loaded_named_collections.erase(collection_name); - return true; -} - -void NamedCollectionFactory::removeById(NamedCollectionUtils::SourceId id) -{ - std::lock_guard lock(mutex); - std::erase_if( - loaded_named_collections, - [&](const auto & value) { return value.second->getSourceId() == id; }); -} - -NamedCollectionsMap NamedCollectionFactory::getAll() const -{ - std::lock_guard lock(mutex); - return loaded_named_collections; -} - class NamedCollection::Impl { private: diff --git a/src/Common/NamedCollections/NamedCollections.h b/src/Common/NamedCollections/NamedCollections.h index de27f4e6083..c253c56594f 100644 --- a/src/Common/NamedCollections/NamedCollections.h +++ b/src/Common/NamedCollections/NamedCollections.h @@ -93,59 +93,4 @@ private: mutable std::mutex mutex; }; -/** - * A factory of immutable named collections. - */ -class NamedCollectionFactory : boost::noncopyable -{ -public: - static NamedCollectionFactory & instance(); - - bool exists(const std::string & collection_name) const; - - NamedCollectionPtr get(const std::string & collection_name) const; - - NamedCollectionPtr tryGet(const std::string & collection_name) const; - - MutableNamedCollectionPtr getMutable(const std::string & collection_name) const; - - void add(const std::string & collection_name, MutableNamedCollectionPtr collection); - - void add(NamedCollectionsMap collections); - - void update(NamedCollectionsMap collections); - - void remove(const std::string & collection_name); - - void removeIfExists(const std::string & collection_name); - - void removeById(NamedCollectionUtils::SourceId id); - - NamedCollectionsMap getAll() const; - -private: - bool existsUnlocked( - const std::string & collection_name, - std::lock_guard & lock) const; - - MutableNamedCollectionPtr tryGetUnlocked( - const std::string & collection_name, - std::lock_guard & lock) const; - - void addUnlocked( - const std::string & collection_name, - MutableNamedCollectionPtr collection, - std::lock_guard & lock); - - bool removeIfExistsUnlocked( - const std::string & collection_name, - std::lock_guard & lock); - - mutable NamedCollectionsMap loaded_named_collections; - - mutable std::mutex mutex; - bool is_initialized = false; -}; - - } diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.cpp b/src/Common/NamedCollections/NamedCollectionsFactory.cpp new file mode 100644 index 00000000000..dd69952429f --- /dev/null +++ b/src/Common/NamedCollections/NamedCollectionsFactory.cpp @@ -0,0 +1,169 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NAMED_COLLECTION_DOESNT_EXIST; + extern const int NAMED_COLLECTION_ALREADY_EXISTS; + extern const int NAMED_COLLECTION_IS_IMMUTABLE; +} + +NamedCollectionFactory & NamedCollectionFactory::instance() +{ + static NamedCollectionFactory instance; + return instance; +} + +bool NamedCollectionFactory::exists(const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + return existsUnlocked(collection_name, lock); +} + +bool NamedCollectionFactory::existsUnlocked( + const std::string & collection_name, + std::lock_guard & /* lock */) const +{ + return loaded_named_collections.contains(collection_name); +} + +NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + auto collection = tryGetUnlocked(collection_name, lock); + if (!collection) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "There is no named collection `{}`", + collection_name); + } + return collection; +} + +NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + return tryGetUnlocked(collection_name, lock); +} + +MutableNamedCollectionPtr NamedCollectionFactory::getMutable( + const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + auto collection = tryGetUnlocked(collection_name, lock); + if (!collection) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "There is no named collection `{}`", + collection_name); + } + else if (!collection->isMutable()) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE, + "Cannot get collection `{}` for modification, " + "because collection was defined as immutable", + collection_name); + } + return collection; +} + +MutableNamedCollectionPtr NamedCollectionFactory::tryGetUnlocked( + const std::string & collection_name, + std::lock_guard & /* lock */) const +{ + auto it = loaded_named_collections.find(collection_name); + if (it == loaded_named_collections.end()) + return nullptr; + return it->second; +} + +void NamedCollectionFactory::add( + const std::string & collection_name, + MutableNamedCollectionPtr collection) +{ + std::lock_guard lock(mutex); + addUnlocked(collection_name, collection, lock); +} + +void NamedCollectionFactory::add(NamedCollectionsMap collections) +{ + std::lock_guard lock(mutex); + for (const auto & [collection_name, collection] : collections) + addUnlocked(collection_name, collection, lock); +} + +void NamedCollectionFactory::addUnlocked( + const std::string & collection_name, + MutableNamedCollectionPtr collection, + std::lock_guard & /* lock */) +{ + auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection); + if (!inserted) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "A named collection `{}` already exists", + collection_name); + } +} + +void NamedCollectionFactory::remove(const std::string & collection_name) +{ + std::lock_guard lock(mutex); + bool removed = removeIfExistsUnlocked(collection_name, lock); + if (!removed) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "There is no named collection `{}`", + collection_name); + } +} + +void NamedCollectionFactory::removeIfExists(const std::string & collection_name) +{ + std::lock_guard lock(mutex); + removeIfExistsUnlocked(collection_name, lock); // NOLINT +} + +bool NamedCollectionFactory::removeIfExistsUnlocked( + const std::string & collection_name, + std::lock_guard & lock) +{ + auto collection = tryGetUnlocked(collection_name, lock); + if (!collection) + return false; + + if (!collection->isMutable()) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE, + "Cannot get collection `{}` for modification, " + "because collection was defined as immutable", + collection_name); + } + loaded_named_collections.erase(collection_name); + return true; +} + +void NamedCollectionFactory::removeById(NamedCollectionUtils::SourceId id) +{ + std::lock_guard lock(mutex); + std::erase_if( + loaded_named_collections, + [&](const auto & value) { return value.second->getSourceId() == id; }); +} + +NamedCollectionsMap NamedCollectionFactory::getAll() const +{ + std::lock_guard lock(mutex); + return loaded_named_collections; +} + +} diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.h b/src/Common/NamedCollections/NamedCollectionsFactory.h new file mode 100644 index 00000000000..17acd99aead --- /dev/null +++ b/src/Common/NamedCollections/NamedCollectionsFactory.h @@ -0,0 +1,57 @@ +#include + +namespace DB +{ + +class NamedCollectionFactory : boost::noncopyable +{ +public: + static NamedCollectionFactory & instance(); + + bool exists(const std::string & collection_name) const; + + NamedCollectionPtr get(const std::string & collection_name) const; + + NamedCollectionPtr tryGet(const std::string & collection_name) const; + + MutableNamedCollectionPtr getMutable(const std::string & collection_name) const; + + void add(const std::string & collection_name, MutableNamedCollectionPtr collection); + + void add(NamedCollectionsMap collections); + + void update(NamedCollectionsMap collections); + + void remove(const std::string & collection_name); + + void removeIfExists(const std::string & collection_name); + + void removeById(NamedCollectionUtils::SourceId id); + + NamedCollectionsMap getAll() const; + +private: + bool existsUnlocked( + const std::string & collection_name, + std::lock_guard & lock) const; + + MutableNamedCollectionPtr tryGetUnlocked( + const std::string & collection_name, + std::lock_guard & lock) const; + + void addUnlocked( + const std::string & collection_name, + MutableNamedCollectionPtr collection, + std::lock_guard & lock); + + bool removeIfExistsUnlocked( + const std::string & collection_name, + std::lock_guard & lock); + + mutable NamedCollectionsMap loaded_named_collections; + + mutable std::mutex mutex; + bool is_initialized = false; +}; + +} diff --git a/src/Common/tests/gtest_named_collections.cpp b/src/Common/tests/gtest_named_collections.cpp index e2482f6ba8b..8a8a364961b 100644 --- a/src/Common/tests/gtest_named_collections.cpp +++ b/src/Common/tests/gtest_named_collections.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp index 6e0453f5f02..917a12eaaaa 100644 --- a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp +++ b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 03a30d47d91..f5c5d093ce1 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index c1e744e8d79..47b69d79ad8 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -1,6 +1,7 @@ #include "NamedCollectionsHelpers.h" #include #include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 156fa5e5a9b..0836560dff0 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB From 49962e7b6e4c444bb9f2599d7a380bfdf273b231 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 30 May 2024 18:12:32 +0200 Subject: [PATCH 206/856] Docs: add info about analyzer incompatibilities --- docs/en/operations/analyzer.md | 92 ++++++++++++++++++++++++++++++++++ 1 file changed, 92 insertions(+) create mode 100644 docs/en/operations/analyzer.md diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md new file mode 100644 index 00000000000..05f7856c5e3 --- /dev/null +++ b/docs/en/operations/analyzer.md @@ -0,0 +1,92 @@ +--- +slug: /en/operations/analyzer +sidebar_label: Analyzer +--- + +# Analyzer + + + +## Known incompatibilities + +In ClickHouse version `24.3`, new query analysis was enabled by default. +Despite fixing a large number of bugs and introducing new optimizations, it also introduces some breaking changes in ClickHouse behaviour. + +### Invalid before optimization queries + +The previous query planning infrastructure applied AST-level optimizations before the query validation step. +Optimizations could rewrite the initial query so it becomes valid and can be executed. + +In the new infrastructure, query validation takes place before the optimization step. +This means that invalid queries that were possible to execute before are now unsupported. + +**Example:** + +The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. +In the old infrastructure, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. + +```sql +SELECT number +FROM numbers(1) +GROUP BY toString(number) +``` + +### Known incompatibilities for JOIN clause + +* Using expression from `SELECT` list in `JOIN` key as an expression from LEFT table. Example. Fix (best effort, should be under compatibility flag). +* Similar issue ^. Alias for column (in select list) now applied to JOIN result (and not to left table). Example from Denny Crane. New behavior is the correct one. Will try to add best-effort compatibility setting. +* Columns names are changed for some queries. This might breaks some scripts. Example. + + +### Projection column names changes + +During projection names computation aliases are not substituted. + +```sql +SELECT + 1 + 1 AS x, + x + 1 +SETTINGS allow_experimental_analyzer = 0 +FORMAT PrettyCompact + + ┌─x─┬─plus(plus(1, 1), 1)─┐ +1. │ 2 │ 3 │ + └───┴─────────────────────┘ + +SELECT + 1 + 1 AS x, + x + 1 +SETTINGS allow_experimental_analyzer = 1 +FORMAT PrettyCompact + +Query id: 2a5e39a3-3b64-49fd-bad3-0e351931ac99 + + ┌─x─┬─plus(x, 1)─┐ +1. │ 2 │ 3 │ + └───┴────────────┘ +``` + +### Incompatible function arguments types + +In the new infrastructure type inference happens during initial query analysis. +This change means that type checks are done before short-circuit evaluation; thus, `if` function arguments must always have a common supertype. + +**Example:** + +The following query fails with `There is no supertype for types Array(UInt8), String because some of them are Array and some of them are not`: + +```sql +SELECT toTypeName(if(0, [2, 3, 4], 'String')) +``` + +### Heterogeneous clusters + +The new analyzer significantly changed the communication protocol between servers in the cluster. Thus, it's impossible to run distributed queries on servers with different `allow_experimental_analyzer` setting values. + +### Unsupported features + +The list of features new analyzer currently doesn't support: + +- Annoy index. +- Hypothesis index. Work in progress [here](https://github.com/ClickHouse/ClickHouse/pull/48381). +- Window view is not supported. There are no plans to support it in the future. From 6ab029b465eb4567840b2e4b354d9be4a24f2325 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 30 May 2024 16:12:52 +0000 Subject: [PATCH 207/856] fix block number columns --- src/Storages/MergeTree/MergeTask.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5b06338cbd2..dc14f737955 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -58,10 +58,10 @@ static Statistics getStatisticsForColumns( for (const auto & column : columns_to_read) { - const auto & desc = all_columns.get(column.name); - if (desc.stat) + const auto * desc = all_columns.tryGet(column.name); + if (desc && desc->stat) { - auto statistic = MergeTreeStatisticsFactory::instance().get(*desc.stat); + auto statistic = MergeTreeStatisticsFactory::instance().get(*desc->stat); statistics.push_back(std::move(statistic)); } } From 450bcd7f70ef04cd6dfdb482350302f706fdfbf8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 30 May 2024 16:23:23 +0000 Subject: [PATCH 208/856] add test --- .../03166_skip_indexes_vertical_merge.sql | 34 --------------- ...6_skip_indexes_vertical_merge_1.reference} | 5 ++- .../03166_skip_indexes_vertical_merge_1.sql | 35 ++++++++++++++++ ...66_skip_indexes_vertical_merge_2.reference | 1 + .../03166_skip_indexes_vertical_merge_2.sql | 41 +++++++++++++++++++ 5 files changed, 80 insertions(+), 36 deletions(-) delete mode 100644 tests/queries/0_stateless/03166_skip_indexes_vertical_merge.sql rename tests/queries/0_stateless/{03166_skip_indexes_vertical_merge.reference => 03166_skip_indexes_vertical_merge_1.reference} (88%) create mode 100644 tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql create mode 100644 tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.reference create mode 100644 tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.sql deleted file mode 100644 index b894c054f8a..00000000000 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.sql +++ /dev/null @@ -1,34 +0,0 @@ -DROP TABLE IF EXISTS t_ind_merge; - -CREATE TABLE t_ind_merge (a UInt64, b UInt64, c UInt64, d UInt64, INDEX idx_b b TYPE minmax) -ENGINE = MergeTree -ORDER BY a SETTINGS - index_granularity = 64, - vertical_merge_algorithm_min_rows_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 1, - min_bytes_for_wide_part = 0; - -INSERT INTO t_ind_merge SELECT number, number, rand(), rand() from numbers(1000); -INSERT INTO t_ind_merge SELECT number, number, rand(), rand() from numbers(1000); - -SELECT count() FROM t_ind_merge WHERE b < 100 SETTINGS force_data_skipping_indices = 'idx_b'; -EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge WHERE b < 100; - -OPTIMIZE TABLE t_ind_merge FINAL; - -SELECT count() FROM t_ind_merge WHERE b < 100 SETTINGS force_data_skipping_indices = 'idx_b'; -EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge WHERE b < 100; - -DROP TABLE t_ind_merge; -SYSTEM FLUSH LOGS; - -WITH - (SELECT uuid FROM system.tables WHERE database = currentDatabase() AND table = 't_ind_merge') AS uuid, - extractAllGroupsVertical(message, 'containing (\\d+) columns \((\\d+) merged, (\\d+) gathered\)')[1] AS groups -SELECT - groups[1] AS total, - groups[2] AS merged, - groups[3] AS gathered -FROM system.text_log -WHERE query_id = uuid || '::all_1_2_1' AND notEmpty(groups) -ORDER BY event_time_microseconds; diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.reference b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.reference similarity index 88% rename from tests/queries/0_stateless/03166_skip_indexes_vertical_merge.reference rename to tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.reference index 02d5765102c..86f79bea4ba 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge.reference +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.reference @@ -3,7 +3,7 @@ Expression ((Project names + Projection)) Aggregating Expression (Before GROUP BY) Filter ((WHERE + Change column names to column identifiers)) - ReadFromMergeTree (default.t_ind_merge) + ReadFromMergeTree (default.t_ind_merge_1) Indexes: PrimaryKey Condition: true @@ -19,7 +19,7 @@ Expression ((Project names + Projection)) Aggregating Expression (Before GROUP BY) Filter ((WHERE + Change column names to column identifiers)) - ReadFromMergeTree (default.t_ind_merge) + ReadFromMergeTree (default.t_ind_merge_1) Indexes: PrimaryKey Condition: true @@ -30,3 +30,4 @@ Expression ((Project names + Projection)) Description: minmax GRANULARITY 1 Parts: 1/1 Granules: 4/32 +4 1 3 diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql new file mode 100644 index 00000000000..a605f9dea81 --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql @@ -0,0 +1,35 @@ +DROP TABLE IF EXISTS t_ind_merge_1; + +CREATE TABLE t_ind_merge_1 (a UInt64, b UInt64, c UInt64, d UInt64, INDEX idx_b b TYPE minmax) +ENGINE = MergeTree +ORDER BY a SETTINGS + index_granularity = 64, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + min_bytes_for_wide_part = 0; + +INSERT INTO t_ind_merge_1 SELECT number, number, rand(), rand() FROM numbers(1000); +INSERT INTO t_ind_merge_1 SELECT number, number, rand(), rand() FROM numbers(1000); + +SELECT count() FROM t_ind_merge_1 WHERE b < 100 SETTINGS force_data_skipping_indices = 'idx_b'; +EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge_1 WHERE b < 100; + +OPTIMIZE TABLE t_ind_merge_1 FINAL; + +SELECT count() FROM t_ind_merge_1 WHERE b < 100 SETTINGS force_data_skipping_indices = 'idx_b'; +EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge_1 WHERE b < 100; + +SYSTEM FLUSH LOGS; + +WITH + (SELECT uuid FROM system.tables WHERE database = currentDatabase() AND table = 't_ind_merge_1') AS uuid, + extractAllGroupsVertical(message, 'containing (\\d+) columns \((\\d+) merged, (\\d+) gathered\)')[1] AS groups +SELECT + groups[1] AS total, + groups[2] AS merged, + groups[3] AS gathered +FROM system.text_log +WHERE query_id = uuid || '::all_1_2_1' AND notEmpty(groups) +ORDER BY event_time_microseconds; + +DROP TABLE t_ind_merge_1; diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.reference b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.reference new file mode 100644 index 00000000000..4c2f01294a4 --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.reference @@ -0,0 +1 @@ +6 3 3 diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql new file mode 100644 index 00000000000..656694242b7 --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql @@ -0,0 +1,41 @@ +DROP TABLE IF EXISTS t_ind_merge_2; + +CREATE TABLE t_ind_merge_2 ( + a UInt64, + b UInt64, + c UInt64, + d UInt64, + e UInt64, + f UInt64, + INDEX idx_a a TYPE minmax, + INDEX idx_b b TYPE minmax, + INDEX idx_cd c * d TYPE minmax, + INDEX idx_d1 d TYPE minmax, + INDEX idx_d2 d TYPE set(3), + INDEX idx_e e TYPE set(3)) +ENGINE = MergeTree +ORDER BY a SETTINGS + index_granularity = 64, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + min_bytes_for_wide_part = 0; + +INSERT INTO t_ind_merge_2 SELECT number, number, rand(), rand(), rand(), rand() FROM numbers(1000); +INSERT INTO t_ind_merge_2 SELECT number, number, rand(), rand(), rand(), rand() FROM numbers(1000); + +OPTIMIZE TABLE t_ind_merge_2 FINAL; +SYSTEM FLUSH LOGS; + +--- merged: a, c, d; gathered: b, e, f +WITH + (SELECT uuid FROM system.tables WHERE database = currentDatabase() AND table = 't_ind_merge_2') AS uuid, + extractAllGroupsVertical(message, 'containing (\\d+) columns \((\\d+) merged, (\\d+) gathered\)')[1] AS groups +SELECT + groups[1] AS total, + groups[2] AS merged, + groups[3] AS gathered +FROM system.text_log +WHERE query_id = uuid || '::all_1_2_1' AND notEmpty(groups) +ORDER BY event_time_microseconds; + +DROP TABLE t_ind_merge_2; From 826bec2575b8ccd3800b0e04be5422c000367dc2 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 16:33:48 +0000 Subject: [PATCH 209/856] 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 47b45fdc1fc8521ad91a69677b1cb398771b2bfb Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 30 May 2024 16:46:13 +0000 Subject: [PATCH 210/856] add hysteresis of untracked memory --- src/Common/CurrentMemoryTracker.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 6166119eccf..b1dcded0b23 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -111,7 +111,8 @@ AllocationTrace CurrentMemoryTracker::free(Int64 size) if (current_thread) { current_thread->untracked_memory -= size; - if (current_thread->untracked_memory < -current_thread->untracked_memory_limit) + // Note that we use `max_untracked_memory` and not `untracked_memory_limit` to create hysteresis to avoid track/untrack cycles + if (current_thread->untracked_memory < -current_thread->max_untracked_memory) { Int64 untracked_memory = current_thread->untracked_memory; current_thread->untracked_memory = 0; From 83c0c9bcdeeb5892e04dab4e12c3d279f7148312 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 May 2024 19:43:05 +0200 Subject: [PATCH 211/856] Move everything to NamedCollecitonFactory --- programs/server/Server.cpp | 6 +- .../NamedCollections/NamedCollectionUtils.cpp | 674 ------------------ .../NamedCollections/NamedCollectionUtils.h | 42 -- .../NamedCollections/NamedCollections.cpp | 3 +- .../NamedCollections/NamedCollections.h | 8 +- .../NamedCollectionsFactory.cpp | 151 +++- .../NamedCollectionsFactory.h | 23 +- .../NamedCollectionsMetadata.cpp | 438 ++++++++++++ .../NamedCollectionsMetadata.h | 49 ++ src/Common/tests/gtest_named_collections.cpp | 7 +- .../InterpreterAlterNamedCollectionQuery.cpp | 4 +- .../InterpreterCreateNamedCollectionQuery.cpp | 4 +- .../InterpreterDropNamedCollectionQuery.cpp | 4 +- src/Storages/NamedCollectionsHelpers.cpp | 2 +- .../System/StorageSystemNamedCollections.cpp | 2 +- 15 files changed, 679 insertions(+), 738 deletions(-) delete mode 100644 src/Common/NamedCollections/NamedCollectionUtils.cpp delete mode 100644 src/Common/NamedCollections/NamedCollectionUtils.h create mode 100644 src/Common/NamedCollections/NamedCollectionsMetadata.cpp create mode 100644 src/Common/NamedCollections/NamedCollectionsMetadata.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 223bc1f77e7..1ffff5e64e2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -48,6 +48,7 @@ #include #include #include +#include #include #include #include @@ -70,7 +71,6 @@ #include #include #include -#include #include #include #include @@ -1337,7 +1337,7 @@ try CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements); #endif - NamedCollectionUtils::loadIfNot(); + NamedCollectionFactory::instance().loadIfNot(); /// Initialize main config reloader. std::string include_from_path = config().getString("include_from", "/etc/metrika.xml"); @@ -1606,7 +1606,7 @@ try #if USE_SSL CertificateReloader::instance().tryLoad(*config); #endif - NamedCollectionUtils::reloadFromConfig(*config); + NamedCollectionFactory::instance().reloadFromConfig(*config); FileCacheFactory::instance().updateSettingsFromConfig(*config); diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp deleted file mode 100644 index 58c4bcde2b6..00000000000 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ /dev/null @@ -1,674 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace fs = std::filesystem; - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NAMED_COLLECTION_ALREADY_EXISTS; - extern const int NAMED_COLLECTION_DOESNT_EXIST; - extern const int BAD_ARGUMENTS; - extern const int INVALID_CONFIG_PARAMETER; -} - -namespace NamedCollectionUtils -{ - -static std::atomic is_loaded_from_config = false; -static std::atomic is_loaded_from_sql = false; - -class LoadFromConfig -{ -private: - const Poco::Util::AbstractConfiguration & config; - -public: - explicit LoadFromConfig(const Poco::Util::AbstractConfiguration & config_) - : config(config_) {} - - std::vector listCollections() const - { - Poco::Util::AbstractConfiguration::Keys collections_names; - config.keys(NAMED_COLLECTIONS_CONFIG_PREFIX, collections_names); - return collections_names; - } - - NamedCollectionsMap getAll() const - { - NamedCollectionsMap result; - for (const auto & collection_name : listCollections()) - { - if (result.contains(collection_name)) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, - "Found duplicate named collection `{}`", - collection_name); - } - result.emplace(collection_name, get(collection_name)); - } - return result; - } - - MutableNamedCollectionPtr get(const std::string & collection_name) const - { - const auto collection_prefix = getCollectionPrefix(collection_name); - std::queue enumerate_input; - std::set> enumerate_result; - - enumerate_input.push(collection_prefix); - NamedCollectionConfiguration::listKeys(config, std::move(enumerate_input), enumerate_result, -1); - - /// Collection does not have any keys. - /// (`enumerate_result` == ). - const bool collection_is_empty = enumerate_result.size() == 1 - && *enumerate_result.begin() == collection_prefix; - std::set> keys; - if (!collection_is_empty) - { - /// Skip collection prefix and add +1 to avoid '.' in the beginning. - for (const auto & path : enumerate_result) - keys.emplace(path.substr(collection_prefix.size() + 1)); - } - - return NamedCollection::create( - config, collection_name, collection_prefix, keys, SourceId::CONFIG, /* is_mutable */false); - } - -private: - static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; - - static std::string getCollectionPrefix(const std::string & collection_name) - { - return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name); - } -}; - -class INamedCollectionsStorage -{ -public: - virtual ~INamedCollectionsStorage() = default; - - virtual bool exists(const std::string & path) const = 0; - - virtual std::vector list() const = 0; - - virtual std::string read(const std::string & path) const = 0; - - virtual void write(const std::string & path, const std::string & data, bool replace) = 0; - - virtual void remove(const std::string & path) = 0; - - virtual bool removeIfExists(const std::string & path) = 0; -}; - -using NamedCollectionsStoragePtr = std::unique_ptr; - - -class NamedCollectionsMetadata : private WithContext -{ -private: - NamedCollectionsStoragePtr storage; - -public: - NamedCollectionsMetadata(NamedCollectionsStoragePtr storage_, ContextPtr context_) - : WithContext(context_) - , storage(std::move(storage_)) {} - - std::vector listCollections() const - { - auto paths = storage->list(); - std::vector collections; - collections.reserve(paths.size()); - for (const auto & path : paths) - collections.push_back(fs::path(path).stem()); - return collections; - } - - NamedCollectionsMap getAll() const - { - NamedCollectionsMap result; - for (const auto & collection_name : listCollections()) - { - if (result.contains(collection_name)) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, - "Found duplicate named collection `{}`", - collection_name); - } - result.emplace(collection_name, get(collection_name)); - } - return result; - } - - MutableNamedCollectionPtr get(const std::string & collection_name) const - { - const auto query = readCreateQuery(collection_name); - return createNamedCollectionFromAST(query); - } - - MutableNamedCollectionPtr create(const ASTCreateNamedCollectionQuery & query) - { - writeCreateQuery(query); - return createNamedCollectionFromAST(query); - } - - void update(const ASTAlterNamedCollectionQuery & query) - { - auto create_query = readCreateQuery(query.collection_name); - - std::unordered_map result_changes_map; - for (const auto & [name, value] : query.changes) - { - auto [it, inserted] = result_changes_map.emplace(name, value); - if (!inserted) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Value with key `{}` is used twice in the SET query (collection name: {})", - name, query.collection_name); - } - } - - for (const auto & [name, value] : create_query.changes) - result_changes_map.emplace(name, value); - - std::unordered_map result_overridability_map; - for (const auto & [name, value] : query.overridability) - result_overridability_map.emplace(name, value); - for (const auto & [name, value] : create_query.overridability) - result_overridability_map.emplace(name, value); - - for (const auto & delete_key : query.delete_keys) - { - auto it = result_changes_map.find(delete_key); - if (it == result_changes_map.end()) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot delete key `{}` because it does not exist in collection", - delete_key); - } - else - { - result_changes_map.erase(it); - auto it_override = result_overridability_map.find(delete_key); - if (it_override != result_overridability_map.end()) - result_overridability_map.erase(it_override); - } - } - - create_query.changes.clear(); - for (const auto & [name, value] : result_changes_map) - create_query.changes.emplace_back(name, value); - create_query.overridability = std::move(result_overridability_map); - - if (create_query.changes.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Named collection cannot be empty (collection name: {})", - query.collection_name); - - chassert(create_query.collection_name == query.collection_name); - writeCreateQuery(create_query, true); - } - - void remove(const std::string & collection_name) - { - storage->remove(getFileName(collection_name)); - } - - bool removeIfExists(const std::string & collection_name) - { - return storage->removeIfExists(getFileName(collection_name)); - } - -private: - std::string getFileName(const std::string & collection_name) const - { - return escapeForFileName(collection_name) + ".sql"; - } - - static MutableNamedCollectionPtr createNamedCollectionFromAST(const ASTCreateNamedCollectionQuery & query) - { - const auto & collection_name = query.collection_name; - const auto config = NamedCollectionConfiguration::createConfiguration(collection_name, query.changes, query.overridability); - - std::set> keys; - for (const auto & [name, _] : query.changes) - keys.insert(name); - - return NamedCollection::create( - *config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true); - } - - ASTCreateNamedCollectionQuery readCreateQuery(const std::string & collection_name) const - { - const auto path = getFileName(collection_name); - auto query = storage->read(path); - - ParserCreateNamedCollectionQuery parser; - auto ast = parseQuery(parser, query, "in file " + path, 0, getContext()->getSettingsRef().max_parser_depth, getContext()->getSettingsRef().max_parser_backtracks); - const auto & create_query = ast->as(); - return create_query; - } - - void writeCreateQuery(const ASTCreateNamedCollectionQuery & query, bool replace = false) - { - auto normalized_query = query.clone(); - auto & changes = typeid_cast(normalized_query.get())->changes; - ::sort( - changes.begin(), changes.end(), - [](const SettingChange & lhs, const SettingChange & rhs) { return lhs.name < rhs.name; }); - - storage->write(getFileName(query.collection_name), serializeAST(*normalized_query), replace); - } -}; - -class NamedCollectionsLocalStorage : public INamedCollectionsStorage, private WithContext -{ -private: - std::string root_path; - -public: - NamedCollectionsLocalStorage(ContextPtr context_, const std::string & path_) - : WithContext(context_) - , root_path(path_) - { - if (fs::exists(root_path)) - cleanup(); - } - - ~NamedCollectionsLocalStorage() override = default; - - std::vector list() const override - { - if (!fs::exists(root_path)) - return {}; - - std::vector elements; - for (fs::directory_iterator it{root_path}; it != fs::directory_iterator{}; ++it) - { - const auto & current_path = it->path(); - if (current_path.extension() == ".sql") - { - elements.push_back(it->path()); - } - else - { - LOG_WARNING( - getLogger("NamedCollectionsLocalStorage"), - "Unexpected file {} in named collections directory", - current_path.filename().string()); - } - } - return elements; - } - - bool exists(const std::string & path) const override - { - return fs::exists(getPath(path)); - } - - std::string read(const std::string & path) const override - { - ReadBufferFromFile in(getPath(path)); - std::string data; - readStringUntilEOF(data, in); - return data; - } - - void write(const std::string & path, const std::string & data, bool replace) override - { - if (!replace && fs::exists(path)) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, - "Metadata file {} for named collection already exists", - path); - } - - fs::create_directories(root_path); - - auto tmp_path = getPath(path + ".tmp"); - WriteBufferFromFile out(tmp_path, data.size(), O_WRONLY | O_CREAT | O_EXCL); - writeString(data, out); - - out.next(); - if (getContext()->getSettingsRef().fsync_metadata) - out.sync(); - out.close(); - - fs::rename(tmp_path, getPath(path)); - } - - void remove(const std::string & path) override - { - if (!removeIfExists(getPath(path))) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, - "Cannot remove `{}`, because it doesn't exist", path); - } - } - - bool removeIfExists(const std::string & path) override - { - return fs::remove(getPath(path)); - } - -private: - std::string getPath(const std::string & path) const - { - return fs::path(root_path) / path; - } - - /// Delete .tmp files. They could be left undeleted in case of - /// some exception or abrupt server restart. - void cleanup() - { - std::vector files_to_remove; - for (fs::directory_iterator it{root_path}; it != fs::directory_iterator{}; ++it) - { - const auto & current_path = it->path(); - if (current_path.extension() == ".tmp") - files_to_remove.push_back(current_path); - } - for (const auto & file : files_to_remove) - fs::remove(file); - } -}; - - -class NamedCollectionsZooKeeperStorage : public INamedCollectionsStorage, private WithContext -{ -private: - std::string root_path; - mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; - -public: - NamedCollectionsZooKeeperStorage(ContextPtr context_, const std::string & path_) - : WithContext(context_) - , root_path(path_) - { - if (root_path.empty()) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Collections path cannot be empty"); - - if (root_path != "/" && root_path.back() == '/') - root_path.resize(root_path.size() - 1); - if (root_path.front() != '/') - root_path = "/" + root_path; - - auto client = getClient(); - if (root_path != "/" && !client->exists(root_path)) - { - client->createAncestors(root_path); - client->createIfNotExists(root_path, ""); - } - } - - ~NamedCollectionsZooKeeperStorage() override = default; - - std::vector list() const override - { - return getClient()->getChildren(root_path); - } - - bool exists(const std::string & path) const override - { - return getClient()->exists(getPath(path)); - } - - std::string read(const std::string & path) const override - { - return getClient()->get(getPath(path)); - } - - void write(const std::string & path, const std::string & data, bool replace) override - { - if (replace) - { - getClient()->createOrUpdate(getPath(path), data, zkutil::CreateMode::Persistent); - } - else - { - auto code = getClient()->tryCreate(getPath(path), data, zkutil::CreateMode::Persistent); - - if (code == Coordination::Error::ZNODEEXISTS) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, - "Metadata file {} for named collection already exists", - path); - } - } - } - - void remove(const std::string & path) override - { - getClient()->remove(getPath(path)); - } - - bool removeIfExists(const std::string & path) override - { - auto code = getClient()->tryRemove(getPath(path)); - if (code == Coordination::Error::ZOK) - return true; - if (code == Coordination::Error::ZNONODE) - return false; - throw Coordination::Exception::fromPath(code, getPath(path)); - } - -private: - zkutil::ZooKeeperPtr getClient() const - { - if (!zookeeper_client || zookeeper_client->expired()) - { - zookeeper_client = getContext()->getZooKeeper(); - zookeeper_client->sync(root_path); - } - return zookeeper_client; - } - - std::string getPath(const std::string & path) const - { - return fs::path(root_path) / path; - } -}; - - -std::unique_lock lockNamedCollectionsTransaction() -{ - static std::mutex transaction_lock; - return std::unique_lock(transaction_lock); -} - -void loadFromConfigUnlocked(const Poco::Util::AbstractConfiguration & config, std::unique_lock &) -{ - auto named_collections = LoadFromConfig(config).getAll(); - LOG_TRACE( - getLogger("NamedCollectionsUtils"), - "Loaded {} collections from config", named_collections.size()); - - NamedCollectionFactory::instance().add(std::move(named_collections)); - is_loaded_from_config = true; -} - -void loadFromConfig(const Poco::Util::AbstractConfiguration & config) -{ - auto lock = lockNamedCollectionsTransaction(); - loadFromConfigUnlocked(config, lock); -} - -void reloadFromConfig(const Poco::Util::AbstractConfiguration & config) -{ - auto lock = lockNamedCollectionsTransaction(); - auto collections = LoadFromConfig(config).getAll(); - auto & instance = NamedCollectionFactory::instance(); - instance.removeById(SourceId::CONFIG); - instance.add(collections); - is_loaded_from_config = true; -} - -auto getNamedCollectionsStorage(ContextPtr context) -{ - static const std::string storage_config_path = "named_collections_storage"; - - const auto & config = context->getConfigRef(); - const auto storage = config.getString(storage_config_path + ".type", "local"); - - if (storage == "local") - { - const auto path = config.getString(storage_config_path + ".path", fs::path(context->getPath()) / "named_collections"); - return NamedCollectionsMetadata( - std::make_unique(context, path), context); - } - if (storage == "zookeeper") - { - return NamedCollectionsMetadata( - std::make_unique( - context, config.getString(storage_config_path + ".path")), - context); - } - - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, - "Unknown storage for named collections: {}", storage); -} - -void loadFromSQLUnlocked(ContextPtr context, std::unique_lock &) -{ - auto named_collections = getNamedCollectionsStorage(context).getAll(); - LOG_TRACE( - getLogger("NamedCollectionsUtils"), - "Loaded {} collections from SQL", named_collections.size()); - - NamedCollectionFactory::instance().add(std::move(named_collections)); - is_loaded_from_sql = true; -} - -void loadFromSQL(ContextPtr context) -{ - auto lock = lockNamedCollectionsTransaction(); - loadFromSQLUnlocked(context, lock); -} - -void loadIfNotUnlocked(std::unique_lock & lock) -{ - auto global_context = Context::getGlobalContextInstance(); - if (!is_loaded_from_config) - loadFromConfigUnlocked(global_context->getConfigRef(), lock); - if (!is_loaded_from_sql) - loadFromSQLUnlocked(global_context, lock); -} - -void loadIfNot() -{ - if (is_loaded_from_sql && is_loaded_from_config) - return; - auto lock = lockNamedCollectionsTransaction(); - loadIfNotUnlocked(lock); -} - -void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context) -{ - auto lock = lockNamedCollectionsTransaction(); - loadIfNotUnlocked(lock); - auto & instance = NamedCollectionFactory::instance(); - if (!instance.exists(query.collection_name)) - { - if (!query.if_exists) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, - "Cannot remove collection `{}`, because it doesn't exist", - query.collection_name); - } - return; - } - getNamedCollectionsStorage(context).remove(query.collection_name); - instance.remove(query.collection_name); -} - -void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context) -{ - auto lock = lockNamedCollectionsTransaction(); - loadIfNotUnlocked(lock); - auto & instance = NamedCollectionFactory::instance(); - if (instance.exists(query.collection_name)) - { - if (!query.if_not_exists) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, - "A named collection `{}` already exists", - query.collection_name); - } - return; - } - instance.add(query.collection_name, getNamedCollectionsStorage(context).create(query)); -} - -void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context) -{ - auto lock = lockNamedCollectionsTransaction(); - loadIfNotUnlocked(lock); - auto & instance = NamedCollectionFactory::instance(); - if (!instance.exists(query.collection_name)) - { - if (!query.if_exists) - { - throw Exception( - ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, - "Cannot remove collection `{}`, because it doesn't exist", - query.collection_name); - } - return; - } - getNamedCollectionsStorage(context).update(query); - - auto collection = instance.getMutable(query.collection_name); - auto collection_lock = collection->lock(); - - for (const auto & [name, value] : query.changes) - { - auto it_override = query.overridability.find(name); - if (it_override != query.overridability.end()) - collection->setOrUpdate(name, convertFieldToString(value), it_override->second); - else - collection->setOrUpdate(name, convertFieldToString(value), {}); - } - - for (const auto & key : query.delete_keys) - collection->remove(key); -} - -} - -} diff --git a/src/Common/NamedCollections/NamedCollectionUtils.h b/src/Common/NamedCollections/NamedCollectionUtils.h deleted file mode 100644 index 293b3ea659d..00000000000 --- a/src/Common/NamedCollections/NamedCollectionUtils.h +++ /dev/null @@ -1,42 +0,0 @@ -#pragma once -#include - -namespace Poco { namespace Util { class AbstractConfiguration; } } - -namespace DB -{ - -class ASTCreateNamedCollectionQuery; -class ASTAlterNamedCollectionQuery; -class ASTDropNamedCollectionQuery; - -namespace NamedCollectionUtils -{ - -enum class SourceId : uint8_t -{ - NONE = 0, - CONFIG = 1, - SQL = 2, -}; - -void loadFromConfig(const Poco::Util::AbstractConfiguration & config); -void reloadFromConfig(const Poco::Util::AbstractConfiguration & config); - -/// Load named collections from `context->getPath() / named_collections /`. -void loadFromSQL(ContextPtr context); - -/// Remove collection as well as its metadata from `context->getPath() / named_collections /`. -void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context); - -/// Create a new collection from AST and put it to `context->getPath() / named_collections /`. -void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context); - -/// Update definition of already existing collection from AST and update result in `context->getPath() / named_collections /`. -void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context); - -void loadIfNot(); - -} - -} diff --git a/src/Common/NamedCollections/NamedCollections.cpp b/src/Common/NamedCollections/NamedCollections.cpp index cc12e78095c..2fe5ced5b36 100644 --- a/src/Common/NamedCollections/NamedCollections.cpp +++ b/src/Common/NamedCollections/NamedCollections.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include @@ -299,7 +298,7 @@ MutableNamedCollectionPtr NamedCollection::duplicate() const auto impl = pimpl->createCopy(collection_name); return std::unique_ptr( new NamedCollection( - std::move(impl), collection_name, NamedCollectionUtils::SourceId::NONE, true)); + std::move(impl), collection_name, SourceId::NONE, true)); } NamedCollection::Keys NamedCollection::getKeys(ssize_t depth, const std::string & prefix) const diff --git a/src/Common/NamedCollections/NamedCollections.h b/src/Common/NamedCollections/NamedCollections.h index c253c56594f..23862c4515a 100644 --- a/src/Common/NamedCollections/NamedCollections.h +++ b/src/Common/NamedCollections/NamedCollections.h @@ -1,7 +1,6 @@ #pragma once #include #include -#include namespace Poco { namespace Util { class AbstractConfiguration; } } @@ -23,7 +22,12 @@ class NamedCollection public: using Key = std::string; using Keys = std::set>; - using SourceId = NamedCollectionUtils::SourceId; + enum class SourceId : uint8_t + { + NONE = 0, + CONFIG = 1, + SQL = 2, + }; static MutableNamedCollectionPtr create( const Poco::Util::AbstractConfiguration & config, diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.cpp b/src/Common/NamedCollections/NamedCollectionsFactory.cpp index dd69952429f..d308b300eea 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.cpp +++ b/src/Common/NamedCollections/NamedCollectionsFactory.cpp @@ -1,5 +1,6 @@ #include -#include +#include +#include namespace DB { @@ -152,7 +153,7 @@ bool NamedCollectionFactory::removeIfExistsUnlocked( return true; } -void NamedCollectionFactory::removeById(NamedCollectionUtils::SourceId id) +void NamedCollectionFactory::removeById(NamedCollection::SourceId id) { std::lock_guard lock(mutex); std::erase_if( @@ -166,4 +167,150 @@ NamedCollectionsMap NamedCollectionFactory::getAll() const return loaded_named_collections; } +namespace +{ + constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; + + std::vector listCollections(const Poco::Util::AbstractConfiguration & config) + { + Poco::Util::AbstractConfiguration::Keys collections_names; + config.keys(NAMED_COLLECTIONS_CONFIG_PREFIX, collections_names); + return collections_names; + } + + MutableNamedCollectionPtr getCollection( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_name) + { + const auto collection_prefix = fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name); + std::queue enumerate_input; + std::set> enumerate_result; + + enumerate_input.push(collection_prefix); + NamedCollectionConfiguration::listKeys(config, std::move(enumerate_input), enumerate_result, -1); + + /// Collection does not have any keys. (`enumerate_result` == ). + const bool collection_is_empty = enumerate_result.size() == 1 + && *enumerate_result.begin() == collection_prefix; + + std::set> keys; + if (!collection_is_empty) + { + /// Skip collection prefix and add +1 to avoid '.' in the beginning. + for (const auto & path : enumerate_result) + keys.emplace(path.substr(collection_prefix.size() + 1)); + } + + return NamedCollection::create( + config, collection_name, collection_prefix, keys, NamedCollection::SourceId::CONFIG, /* is_mutable */false); + } + + NamedCollectionsMap getNamedCollections(const Poco::Util::AbstractConfiguration & config) + { + NamedCollectionsMap result; + for (const auto & collection_name : listCollections(config)) + { + if (result.contains(collection_name)) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Found duplicate named collection `{}`", + collection_name); + } + result.emplace(collection_name, getCollection(config, collection_name)); + } + return result; + } +} + +void NamedCollectionFactory::loadFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + add(getNamedCollections(config)); +} + +void NamedCollectionFactory::reloadFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + auto collections = getNamedCollections(config); + removeById(NamedCollection::SourceId::CONFIG); + add(collections); +} + +void NamedCollectionFactory::loadFromSQL(const ContextPtr & context) +{ + add(NamedCollectionsMetadata::create(context)->getAll()); +} + +void NamedCollectionFactory::loadIfNot() +{ + if (loaded) + return; + auto global_context = Context::getGlobalContextInstance(); + loadFromConfig(global_context->getConfigRef()); + loadFromSQL(global_context); + loaded = true; +} + +void NamedCollectionFactory::createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context) +{ + loadIfNot(); + if (exists(query.collection_name)) + { + if (query.if_not_exists) + return; + + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "A named collection `{}` already exists", + query.collection_name); + } + add(query.collection_name, NamedCollectionsMetadata::create(context)->create(query)); +} + +void NamedCollectionFactory::removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context) +{ + loadIfNot(); + if (!exists(query.collection_name)) + { + if (query.if_exists) + return; + + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "Cannot remove collection `{}`, because it doesn't exist", + query.collection_name); + } + NamedCollectionsMetadata::create(context)->remove(query.collection_name); + remove(query.collection_name); +} + +void NamedCollectionFactory::updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context) +{ + loadIfNot(); + if (!exists(query.collection_name)) + { + if (query.if_exists) + return; + + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "Cannot remove collection `{}`, because it doesn't exist", + query.collection_name); + } + NamedCollectionsMetadata::create(context)->update(query); + + auto collection = getMutable(query.collection_name); + auto collection_lock = collection->lock(); + + for (const auto & [name, value] : query.changes) + { + auto it_override = query.overridability.find(name); + if (it_override != query.overridability.end()) + collection->setOrUpdate(name, convertFieldToString(value), it_override->second); + else + collection->setOrUpdate(name, convertFieldToString(value), {}); + } + + for (const auto & key : query.delete_keys) + collection->remove(key); +} } diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.h b/src/Common/NamedCollections/NamedCollectionsFactory.h index 17acd99aead..f8e5f1d41a4 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.h +++ b/src/Common/NamedCollections/NamedCollectionsFactory.h @@ -1,7 +1,11 @@ #include +#include namespace DB { +class ASTCreateNamedCollectionQuery; +class ASTDropNamedCollectionQuery; +class ASTAlterNamedCollectionQuery; class NamedCollectionFactory : boost::noncopyable { @@ -26,10 +30,23 @@ public: void removeIfExists(const std::string & collection_name); - void removeById(NamedCollectionUtils::SourceId id); + void removeById(NamedCollection::SourceId id); NamedCollectionsMap getAll() const; + void loadIfNot(); + + void reloadFromConfig(const Poco::Util::AbstractConfiguration & config); + + void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context); + + void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context); + + void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context); + + /// This method is public only for unit tests. + void loadFromConfig(const Poco::Util::AbstractConfiguration & config); + private: bool existsUnlocked( const std::string & collection_name, @@ -50,8 +67,12 @@ private: mutable NamedCollectionsMap loaded_named_collections; + LoggerPtr log = getLogger("NamedCollectionFactory"); mutable std::mutex mutex; bool is_initialized = false; + bool loaded = false; + + void loadFromSQL(const ContextPtr & context); }; } diff --git a/src/Common/NamedCollections/NamedCollectionsMetadata.cpp b/src/Common/NamedCollections/NamedCollectionsMetadata.cpp new file mode 100644 index 00000000000..c60be72c741 --- /dev/null +++ b/src/Common/NamedCollections/NamedCollectionsMetadata.cpp @@ -0,0 +1,438 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NAMED_COLLECTION_ALREADY_EXISTS; + extern const int NAMED_COLLECTION_DOESNT_EXIST; + extern const int INVALID_CONFIG_PARAMETER; +} + +namespace +{ + MutableNamedCollectionPtr createNamedCollectionFromAST(const ASTCreateNamedCollectionQuery & query) + { + const auto & collection_name = query.collection_name; + const auto config = NamedCollectionConfiguration::createConfiguration(collection_name, query.changes, query.overridability); + + std::set> keys; + for (const auto & [name, _] : query.changes) + keys.insert(name); + + return NamedCollection::create( + *config, collection_name, "", keys, NamedCollection::SourceId::SQL, /* is_mutable */true); + } + + std::string getFileName(const std::string & collection_name) + { + return escapeForFileName(collection_name) + ".sql"; + } +} + +class NamedCollectionsMetadata::INamedCollectionsStorage +{ +public: + virtual ~INamedCollectionsStorage() = default; + + virtual bool exists(const std::string & path) const = 0; + + virtual std::vector list() const = 0; + + virtual std::string read(const std::string & path) const = 0; + + virtual void write(const std::string & path, const std::string & data, bool replace) = 0; + + virtual void remove(const std::string & path) = 0; + + virtual bool removeIfExists(const std::string & path) = 0; +}; + + +class NamedCollectionsMetadata::LocalStorage : public INamedCollectionsStorage, private WithContext +{ +private: + std::string root_path; + +public: + LocalStorage(ContextPtr context_, const std::string & path_) + : WithContext(context_) + , root_path(path_) + { + if (fs::exists(root_path)) + cleanup(); + } + + ~LocalStorage() override = default; + + std::vector list() const override + { + if (!fs::exists(root_path)) + return {}; + + std::vector elements; + for (fs::directory_iterator it{root_path}; it != fs::directory_iterator{}; ++it) + { + const auto & current_path = it->path(); + if (current_path.extension() == ".sql") + { + elements.push_back(it->path()); + } + else + { + LOG_WARNING( + getLogger("LocalStorage"), + "Unexpected file {} in named collections directory", + current_path.filename().string()); + } + } + return elements; + } + + bool exists(const std::string & path) const override + { + return fs::exists(getPath(path)); + } + + std::string read(const std::string & path) const override + { + ReadBufferFromFile in(getPath(path)); + std::string data; + readStringUntilEOF(data, in); + return data; + } + + void write(const std::string & path, const std::string & data, bool replace) override + { + if (!replace && fs::exists(path)) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Metadata file {} for named collection already exists", + path); + } + + fs::create_directories(root_path); + + auto tmp_path = getPath(path + ".tmp"); + WriteBufferFromFile out(tmp_path, data.size(), O_WRONLY | O_CREAT | O_EXCL); + writeString(data, out); + + out.next(); + if (getContext()->getSettingsRef().fsync_metadata) + out.sync(); + out.close(); + + fs::rename(tmp_path, getPath(path)); + } + + void remove(const std::string & path) override + { + if (!removeIfExists(getPath(path))) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "Cannot remove `{}`, because it doesn't exist", path); + } + } + + bool removeIfExists(const std::string & path) override + { + return fs::remove(getPath(path)); + } + +private: + std::string getPath(const std::string & path) const + { + return fs::path(root_path) / path; + } + + /// Delete .tmp files. They could be left undeleted in case of + /// some exception or abrupt server restart. + void cleanup() + { + std::vector files_to_remove; + for (fs::directory_iterator it{root_path}; it != fs::directory_iterator{}; ++it) + { + const auto & current_path = it->path(); + if (current_path.extension() == ".tmp") + files_to_remove.push_back(current_path); + } + for (const auto & file : files_to_remove) + fs::remove(file); + } +}; + + +class NamedCollectionsMetadata::ZooKeeperStorage : public INamedCollectionsStorage, private WithContext +{ +private: + std::string root_path; + mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; + +public: + ZooKeeperStorage(ContextPtr context_, const std::string & path_) + : WithContext(context_) + , root_path(path_) + { + if (root_path.empty()) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Collections path cannot be empty"); + + if (root_path != "/" && root_path.back() == '/') + root_path.resize(root_path.size() - 1); + if (root_path.front() != '/') + root_path = "/" + root_path; + + auto client = getClient(); + if (root_path != "/" && !client->exists(root_path)) + { + client->createAncestors(root_path); + client->createIfNotExists(root_path, ""); + } + } + + ~ZooKeeperStorage() override = default; + + std::vector list() const override + { + return getClient()->getChildren(root_path); + } + + bool exists(const std::string & path) const override + { + return getClient()->exists(getPath(path)); + } + + std::string read(const std::string & path) const override + { + return getClient()->get(getPath(path)); + } + + void write(const std::string & path, const std::string & data, bool replace) override + { + if (replace) + { + getClient()->createOrUpdate(getPath(path), data, zkutil::CreateMode::Persistent); + } + else + { + auto code = getClient()->tryCreate(getPath(path), data, zkutil::CreateMode::Persistent); + + if (code == Coordination::Error::ZNODEEXISTS) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Metadata file {} for named collection already exists", + path); + } + } + } + + void remove(const std::string & path) override + { + getClient()->remove(getPath(path)); + } + + bool removeIfExists(const std::string & path) override + { + auto code = getClient()->tryRemove(getPath(path)); + if (code == Coordination::Error::ZOK) + return true; + if (code == Coordination::Error::ZNONODE) + return false; + throw Coordination::Exception::fromPath(code, getPath(path)); + } + +private: + zkutil::ZooKeeperPtr getClient() const + { + if (!zookeeper_client || zookeeper_client->expired()) + { + zookeeper_client = getContext()->getZooKeeper(); + zookeeper_client->sync(root_path); + } + return zookeeper_client; + } + + std::string getPath(const std::string & path) const + { + return fs::path(root_path) / path; + } +}; + +std::unique_ptr NamedCollectionsMetadata::create(const ContextPtr & context_) +{ + static const std::string storage_config_path = "named_collections_storage"; + + const auto & config = context_->getConfigRef(); + const auto storage_type = config.getString(storage_config_path + ".type", "local"); + + if (storage_type == "local") + { + const auto path = config.getString( + storage_config_path + ".path", + std::filesystem::path(context_->getPath()) / "named_collections"); + + auto local_storage = std::make_unique(context_, path); + return std::make_unique(std::move(local_storage), context_); + } + if (storage_type == "zookeeper") + { + auto zk_storage = std::make_unique(context_, config.getString(storage_config_path + ".path")); + return std::make_unique(std::move(zk_storage), context_); + } + + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "Unknown storage for named collections: {}", storage_type); +} + +MutableNamedCollectionPtr NamedCollectionsMetadata::get(const std::string & collection_name) const +{ + const auto query = readCreateQuery(collection_name); + return createNamedCollectionFromAST(query); +} + +NamedCollectionsMap NamedCollectionsMetadata::getAll() const +{ + NamedCollectionsMap result; + for (const auto & collection_name : listCollections()) + { + if (result.contains(collection_name)) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Found duplicate named collection `{}`", + collection_name); + } + result.emplace(collection_name, get(collection_name)); + } + return result; +} + +MutableNamedCollectionPtr NamedCollectionsMetadata::create(const ASTCreateNamedCollectionQuery & query) +{ + writeCreateQuery(query); + return createNamedCollectionFromAST(query); +} + +void NamedCollectionsMetadata::remove(const std::string & collection_name) +{ + storage->remove(getFileName(collection_name)); +} + +bool NamedCollectionsMetadata::removeIfExists(const std::string & collection_name) +{ + return storage->removeIfExists(getFileName(collection_name)); +} + +void NamedCollectionsMetadata::update(const ASTAlterNamedCollectionQuery & query) +{ + auto create_query = readCreateQuery(query.collection_name); + + std::unordered_map result_changes_map; + for (const auto & [name, value] : query.changes) + { + auto [it, inserted] = result_changes_map.emplace(name, value); + if (!inserted) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Value with key `{}` is used twice in the SET query (collection name: {})", + name, query.collection_name); + } + } + + for (const auto & [name, value] : create_query.changes) + result_changes_map.emplace(name, value); + + std::unordered_map result_overridability_map; + for (const auto & [name, value] : query.overridability) + result_overridability_map.emplace(name, value); + for (const auto & [name, value] : create_query.overridability) + result_overridability_map.emplace(name, value); + + for (const auto & delete_key : query.delete_keys) + { + auto it = result_changes_map.find(delete_key); + if (it == result_changes_map.end()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot delete key `{}` because it does not exist in collection", + delete_key); + } + else + { + result_changes_map.erase(it); + auto it_override = result_overridability_map.find(delete_key); + if (it_override != result_overridability_map.end()) + result_overridability_map.erase(it_override); + } + } + + create_query.changes.clear(); + for (const auto & [name, value] : result_changes_map) + create_query.changes.emplace_back(name, value); + create_query.overridability = std::move(result_overridability_map); + + if (create_query.changes.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Named collection cannot be empty (collection name: {})", + query.collection_name); + + chassert(create_query.collection_name == query.collection_name); + writeCreateQuery(create_query, true); +} + +std::vector NamedCollectionsMetadata::listCollections() const +{ + auto paths = storage->list(); + std::vector collections; + collections.reserve(paths.size()); + for (const auto & path : paths) + collections.push_back(std::filesystem::path(path).stem()); + return collections; +} + +ASTCreateNamedCollectionQuery NamedCollectionsMetadata::readCreateQuery(const std::string & collection_name) const +{ + const auto path = getFileName(collection_name); + auto query = storage->read(path); + const auto & settings = getContext()->getSettingsRef(); + + ParserCreateNamedCollectionQuery parser; + auto ast = parseQuery(parser, query, "in file " + path, 0, settings.max_parser_depth, settings.max_parser_backtracks); + const auto & create_query = ast->as(); + return create_query; +} + +void NamedCollectionsMetadata::writeCreateQuery(const ASTCreateNamedCollectionQuery & query, bool replace) +{ + auto normalized_query = query.clone(); + auto & changes = typeid_cast(normalized_query.get())->changes; + ::sort( + changes.begin(), changes.end(), + [](const SettingChange & lhs, const SettingChange & rhs) { return lhs.name < rhs.name; }); + + storage->write(getFileName(query.collection_name), serializeAST(*normalized_query), replace); +} + +} diff --git a/src/Common/NamedCollections/NamedCollectionsMetadata.h b/src/Common/NamedCollections/NamedCollectionsMetadata.h new file mode 100644 index 00000000000..4a432b14b96 --- /dev/null +++ b/src/Common/NamedCollections/NamedCollectionsMetadata.h @@ -0,0 +1,49 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +class NamedCollectionsMetadata : private WithContext +{ +public: + static std::unique_ptr create(const ContextPtr & context); + + ~NamedCollectionsMetadata() = default; + + NamedCollectionsMap getAll() const; + + MutableNamedCollectionPtr get(const std::string & collection_name) const; + + MutableNamedCollectionPtr create(const ASTCreateNamedCollectionQuery & query); + + void remove(const std::string & collection_name); + + bool removeIfExists(const std::string & collection_name); + + void update(const ASTAlterNamedCollectionQuery & query); + + class INamedCollectionsStorage; + NamedCollectionsMetadata(std::shared_ptr storage_, ContextPtr context_) + : WithContext(context_) + , storage(std::move(storage_)) {} + /// FIXME: It should be a protected constructor, but I failed make create() method a proper friend. + +private: + class LocalStorage; + class ZooKeeperStorage; + + std::shared_ptr storage; + + std::vector listCollections() const; + + ASTCreateNamedCollectionQuery readCreateQuery(const std::string & collection_name) const; + + void writeCreateQuery(const ASTCreateNamedCollectionQuery & query, bool replace = false); +}; + + +} diff --git a/src/Common/tests/gtest_named_collections.cpp b/src/Common/tests/gtest_named_collections.cpp index 8a8a364961b..0320f825764 100644 --- a/src/Common/tests/gtest_named_collections.cpp +++ b/src/Common/tests/gtest_named_collections.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -29,7 +28,7 @@ TEST(NamedCollections, SimpleConfig) Poco::AutoPtr document = dom_parser.parseString(xml); Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); - NamedCollectionUtils::loadFromConfig(*config); + NamedCollectionFactory::instance().loadFromConfig(*config); ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2")); @@ -119,7 +118,7 @@ TEST(NamedCollections, NestedConfig) Poco::AutoPtr document = dom_parser.parseString(xml); Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); - NamedCollectionUtils::loadFromConfig(*config); + NamedCollectionFactory::instance().loadFromConfig(*config); ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection3")); @@ -171,7 +170,7 @@ TEST(NamedCollections, NestedConfigDuplicateKeys) Poco::AutoPtr document = dom_parser.parseString(xml); Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); - NamedCollectionUtils::loadFromConfig(*config); + NamedCollectionFactory::instance().loadFromConfig(*config); auto collection = NamedCollectionFactory::instance().get("collection"); auto keys = collection->getKeys(); diff --git a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp index a4e86879596..30935b3b89d 100644 --- a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB @@ -23,7 +23,7 @@ BlockIO InterpreterAlterNamedCollectionQuery::execute() return executeDDLQueryOnCluster(query_ptr, current_context, params); } - NamedCollectionUtils::updateFromSQL(query, current_context); + NamedCollectionFactory::instance().updateFromSQL(query, current_context); return {}; } diff --git a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp index 41e87bb73dd..91c19042389 100644 --- a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB @@ -23,7 +23,7 @@ BlockIO InterpreterCreateNamedCollectionQuery::execute() return executeDDLQueryOnCluster(query_ptr, current_context, params); } - NamedCollectionUtils::createFromSQL(query, current_context); + NamedCollectionFactory::instance().createFromSQL(query, current_context); return {}; } diff --git a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp index baadc85f443..62fffc82049 100644 --- a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB @@ -23,7 +23,7 @@ BlockIO InterpreterDropNamedCollectionQuery::execute() return executeDDLQueryOnCluster(query_ptr, current_context, params); } - NamedCollectionUtils::removeFromSQL(query, current_context); + NamedCollectionFactory::instance().removeFromSQL(query, current_context); return {}; } diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index 47b69d79ad8..ba90f21c907 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -95,7 +95,7 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides( if (asts.empty()) return nullptr; - NamedCollectionUtils::loadIfNot(); + NamedCollectionFactory::instance().loadIfNot(); auto collection_name = getCollectionName(asts); if (!collection_name.has_value()) diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 0836560dff0..e98ea155f30 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -33,7 +33,7 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte { const auto & access = context->getAccess(); - NamedCollectionUtils::loadIfNot(); + NamedCollectionFactory::instance().loadIfNot(); auto collections = NamedCollectionFactory::instance().getAll(); for (const auto & [name, collection] : collections) From 7f6d7f34630fde83f3378ce1f9217c32ca183e0e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 30 May 2024 17:55:53 +0000 Subject: [PATCH 212/856] fix indexes with expressions --- src/Storages/MergeTree/MergeTask.cpp | 5 +++-- src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp | 7 +++---- src/Storages/MergeTree/MergedColumnOnlyOutputStream.h | 6 +----- src/Storages/MergeTree/MutateTask.cpp | 2 +- .../0_stateless/03166_skip_indexes_vertical_merge_2.sql | 4 ++-- 5 files changed, 10 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index dc14f737955..2660602d652 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -679,14 +679,15 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const /// Is calculated inside MergeProgressCallback. ctx->column_parts_pipeline.disableProfileEventUpdate(); ctx->executor = std::make_unique(ctx->column_parts_pipeline); + NamesAndTypesList columns_list = {*ctx->it_name_and_type}; ctx->column_to = std::make_unique( global_ctx->new_data_part, global_ctx->metadata_snapshot, - ctx->executor->getHeader(), + columns_list, ctx->compression_codec, indexes_to_recalc, - getStatisticsForColumns({*ctx->it_name_and_type}, global_ctx->metadata_snapshot), + getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), &global_ctx->written_offset_columns, global_ctx->to->getIndexGranularity()); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 674a9bd498f..307c5231761 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -13,15 +13,14 @@ namespace ErrorCodes MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeMutableDataPartPtr & data_part, const StorageMetadataPtr & metadata_snapshot_, - const Block & header_, + const NamesAndTypesList & columns_list_, CompressionCodecPtr default_codec, const MergeTreeIndices & indices_to_recalc, const Statistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) - : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, header_.getNamesAndTypesList(), /*reset_columns=*/ true) - , header(header_) + : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) { const auto & global_settings = data_part->storage.getContext()->getSettings(); @@ -37,7 +36,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), data_part_storage, data_part->index_granularity_info, storage_settings, - header.getNamesAndTypesList(), + columns_list_, data_part->getColumnPositions(), metadata_snapshot_, data_part->storage.getVirtualsPtr(), diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index ad3cabe459e..99100d3d8fe 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -17,7 +17,7 @@ public: MergedColumnOnlyOutputStream( const MergeTreeMutableDataPartPtr & data_part, const StorageMetadataPtr & metadata_snapshot_, - const Block & header_, + const NamesAndTypesList & columns_list_, CompressionCodecPtr default_codec_, const MergeTreeIndices & indices_to_recalc_, const Statistics & stats_to_recalc_, @@ -25,16 +25,12 @@ public: const MergeTreeIndexGranularity & index_granularity = {}, const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr); - Block getHeader() const { return header; } void write(const Block & block) override; MergeTreeData::DataPart::Checksums fillChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums); void finish(bool sync); - -private: - Block header; }; using MergedColumnOnlyOutputStreamPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 43238c5bcbc..0b50ad4066b 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1901,7 +1901,7 @@ private: ctx->out = std::make_shared( ctx->new_data_part, ctx->metadata_snapshot, - ctx->updated_header, + ctx->updated_header.getNamesAndTypesList(), ctx->compression_codec, std::vector(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), Statistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()), diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql index 656694242b7..e29653a2e77 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql @@ -11,8 +11,8 @@ CREATE TABLE t_ind_merge_2 ( INDEX idx_b b TYPE minmax, INDEX idx_cd c * d TYPE minmax, INDEX idx_d1 d TYPE minmax, - INDEX idx_d2 d TYPE set(3), - INDEX idx_e e TYPE set(3)) + INDEX idx_d2 d + 7 TYPE set(3), + INDEX idx_e e * 3 TYPE set(3)) ENGINE = MergeTree ORDER BY a SETTINGS index_granularity = 64, From c42dd51f603691fb58c9574f999f1a5ed72fd960 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 May 2024 20:04:26 +0200 Subject: [PATCH 213/856] Fix style check --- src/Common/NamedCollections/NamedCollectionsFactory.h | 1 + src/Common/NamedCollections/NamedCollectionsMetadata.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.h b/src/Common/NamedCollections/NamedCollectionsFactory.h index f8e5f1d41a4..20681d6c752 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.h +++ b/src/Common/NamedCollections/NamedCollectionsFactory.h @@ -1,3 +1,4 @@ +#pragma once #include #include diff --git a/src/Common/NamedCollections/NamedCollectionsMetadata.cpp b/src/Common/NamedCollections/NamedCollectionsMetadata.cpp index c60be72c741..842c913aaba 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadata.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadata.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int NAMED_COLLECTION_ALREADY_EXISTS; extern const int NAMED_COLLECTION_DOESNT_EXIST; extern const int INVALID_CONFIG_PARAMETER; + extern const int BAD_ARGUMENTS; } namespace From 5a09dce95d0c1be55e9283dcf49b8fb5592f923d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 18:06:11 +0000 Subject: [PATCH 214/856] 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 215/856] 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 216/856] [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 35d3cddf993822b31a9cdf21418295f9289be961 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 30 May 2024 21:36:12 +0200 Subject: [PATCH 217/856] Added condition for single block commit --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index ebb3fc4a82d..8333bf45e67 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -127,7 +127,7 @@ void WriteBufferFromAzureBlobStorage::preFinalize() /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (detached_part_data.size() == 1) + if (detached_part_data.size() == 1 && block_ids.empty()) { if (detached_part_data.front().data_size <= max_single_part_upload_size) { @@ -168,6 +168,11 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() void WriteBufferFromAzureBlobStorage::nextImpl() { + if (is_prefinalized) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot write to prefinalized buffer for Azure Blob Storage, the file could have been created"); + task_tracker->waitIfAny(); hidePartialData(); From a7aa24a968fec07b72872881470878949c329738 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Thu, 30 May 2024 21:19:12 +0000 Subject: [PATCH 218/856] 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 219/856] 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 6afe2b16eeaa04cedf931cab3ccfe24c181454df Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 31 May 2024 10:13:06 +0800 Subject: [PATCH 220/856] disable random settings --- .../0_stateless/03164_adapting_parquet_reader_output_size.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql index 330e1928241..fa098b64702 100644 --- a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel +-- Tags: no-fasttest, no-parallel, no-random-settings set max_insert_threads=1; From dea7f776cbf519501935bc9962c7d4aefbfa649f Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 31 May 2024 10:31:14 +0800 Subject: [PATCH 221/856] add doc --- docs/en/interfaces/formats.md | 2 ++ docs/en/operations/settings/settings-formats.md | 11 +++++++++++ src/Core/SettingsChangesHistory.h | 2 +- 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 66d5bd2e574..3de416ae64d 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2165,6 +2165,8 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t - [output_format_parquet_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_fixed_string_as_fixed_byte_array) - use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. Default value - `true`. - [output_format_parquet_version](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_version) - The version of Parquet format used in output format. Default value - `2.latest`. - [output_format_parquet_compression_method](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_compression_method) - compression method used in output Parquet format. Default value - `lz4`. +- [input_format_parquet_max_block_size](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_max_block_size) - Max block row size for parquet reader. Default value - `65409`. +- [input_format_parquet_prefer_block_bytes](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_prefer_block_bytes) - Average block bytes output by parquet reader. Default value - `16744704`. ## ParquetMetadata {data-format-parquet-metadata} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1a27b350652..6aae1ea62e5 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1417,6 +1417,17 @@ Compression method used in output Parquet format. Supported codecs: `snappy`, `l Default value: `lz4`. +### input_format_parquet_max_block_size {#input_format_parquet_max_block_size} +Max block row size for parquet reader. By controlling the number of rows in each block, you can control the memory usage, +and in some operators that cache blocks, you can improve the accuracy of the operator's memory control。 + +Default value: `65409`. + +### input_format_parquet_prefer_block_bytes {#input_format_parquet_prefer_block_bytes} +Average block bytes output by parquet reader. Lowering the configuration in the case of reading some high compression parquet relieves the memory pressure. + +Default value: `65409 * 256 = 16744704` + ## Hive format settings {#hive-format-settings} ### input_format_hive_text_fields_delimiter {#input_format_hive_text_fields_delimiter} diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index e6fb628809a..5922c58525d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,7 +92,7 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Max block size for parquet reader."}, + {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, From d400d69283bd1ed0c8a783678ab07daf301dc5ff Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 31 May 2024 13:52:24 +0800 Subject: [PATCH 222/856] try --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 0c1b8e5c3d5..c2c709b9b52 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -427,8 +427,8 @@ void ParquetBlockInputFormat::initializeIfNeeded() total_size += row_group_meta->ColumnChunk(column_index)->total_uncompressed_size(); } if (!total_size || !format_settings.parquet.prefer_block_bytes) return 0; - auto average_row_bytes = static_cast(total_size) / row_group_meta->num_rows(); - const size_t preferred_num_rows = static_cast(format_settings.parquet.prefer_block_bytes/average_row_bytes); + auto average_row_bytes = floor(static_cast(total_size) / row_group_meta->num_rows()); + const size_t preferred_num_rows = static_cast(floor(format_settings.parquet.prefer_block_bytes/average_row_bytes)); const size_t MIN_ROW_NUM = 128; // size_t != UInt64 in darwin return std::min(std::max(preferred_num_rows, MIN_ROW_NUM), static_cast(format_settings.parquet.max_block_size)); From b5d9ff291fb803a24a14698c1bb8b4950013e6e8 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Fri, 31 May 2024 15:25:32 +0800 Subject: [PATCH 223/856] Update 03164_adapting_parquet_reader_output_size.reference --- .../03164_adapting_parquet_reader_output_size.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference index 900342c13b3..c6250049d47 100644 --- a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference @@ -1,4 +1,4 @@ 65409 16 128 -2235 +2233 From 216c67c9f8cb8dcb79fd061f92bc3e9576be1fd9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 31 May 2024 10:05:15 +0200 Subject: [PATCH 224/856] Fixed style check --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 8333bf45e67..16bd70cf2e5 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -19,6 +19,11 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + struct WriteBufferFromAzureBlobStorage::PartData { Memory<> memory; From 13a9c4d38e08648cf0aa98f8bc1cfbd9ed39b699 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 31 May 2024 16:49:24 +0800 Subject: [PATCH 225/856] fix test --- .../03164_adapting_parquet_reader_output_size.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference index c6250049d47..ef9b07ba955 100644 --- a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference @@ -1,4 +1,4 @@ 65409 16 128 -2233 +2363 From 3f74783302f545971b0ec7bfec954e91209dc0b6 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 31 May 2024 09:11:58 +0000 Subject: [PATCH 226/856] adjust settings history changes --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4c087060179..ecb4960a06a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,6 +92,7 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + {"min_untracked_memory", 4_MiB, 4_KiB, "A new setting."}, }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, @@ -106,7 +107,6 @@ static std::map sett {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - {"min_untracked_memory", 4_MiB, 4_KiB, "A new setting."}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, From 76ee999cae00b1644980cb45aa20502efdd0cd7a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 31 May 2024 13:57:58 +0200 Subject: [PATCH 227/856] Add an example --- docs/en/operations/analyzer.md | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 05f7856c5e3..62bb45eeb7d 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -20,10 +20,7 @@ Optimizations could rewrite the initial query so it becomes valid and can be exe In the new infrastructure, query validation takes place before the optimization step. This means that invalid queries that were possible to execute before are now unsupported. -**Example:** - -The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. -In the old infrastructure, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. +**Example 1:** ```sql SELECT number @@ -31,7 +28,24 @@ FROM numbers(1) GROUP BY toString(number) ``` -### Known incompatibilities for JOIN clause +The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. +In the old infrastructure, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. + +**Example 2:** + +```sql +SELECT + number % 2 AS n, + sum(number) +FROM numbers(10) +GROUP BY n +HAVING number > 5 +``` + +The same problem occurs in this query: column `number` is used after aggregation with another key. +The previous query analyzer fixed this query by moving the `number > 5` filter from the `HAVING` clause to the `WHERE` clause. + +### Known incompatibilities of JOIN clause * Using expression from `SELECT` list in `JOIN` key as an expression from LEFT table. Example. Fix (best effort, should be under compatibility flag). * Similar issue ^. Alias for column (in select list) now applied to JOIN result (and not to left table). Example from Denny Crane. New behavior is the correct one. Will try to add best-effort compatibility setting. @@ -59,8 +73,6 @@ SELECT SETTINGS allow_experimental_analyzer = 1 FORMAT PrettyCompact -Query id: 2a5e39a3-3b64-49fd-bad3-0e351931ac99 - ┌─x─┬─plus(x, 1)─┐ 1. │ 2 │ 3 │ └───┴────────────┘ From 549c6b524e5c7dfb5bdd305c2f880fdcbd4f6480 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 31 May 2024 14:45:49 +0200 Subject: [PATCH 228/856] Implement background update --- .../NamedCollectionsFactory.cpp | 270 ++++++++++++------ .../NamedCollectionsFactory.h | 88 +++--- ...pp => NamedCollectionsMetadataStorage.cpp} | 117 +++++--- ...ta.h => NamedCollectionsMetadataStorage.h} | 22 +- src/Common/tests/gtest_named_collections.cpp | 63 ++-- src/Interpreters/Context.cpp | 3 + .../InterpreterAlterNamedCollectionQuery.cpp | 2 +- .../InterpreterCreateNamedCollectionQuery.cpp | 2 +- .../InterpreterDropNamedCollectionQuery.cpp | 2 +- .../named_collections_with_zookeeper.xml | 1 + .../test_named_collections/test.py | 134 +++++++++ 11 files changed, 503 insertions(+), 201 deletions(-) rename src/Common/NamedCollections/{NamedCollectionsMetadata.cpp => NamedCollectionsMetadataStorage.cpp} (77%) rename src/Common/NamedCollections/{NamedCollectionsMetadata.h => NamedCollectionsMetadataStorage.h} (66%) diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.cpp b/src/Common/NamedCollections/NamedCollectionsFactory.cpp index 5b16a076811..e46fa26896d 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.cpp +++ b/src/Common/NamedCollections/NamedCollectionsFactory.cpp @@ -1,6 +1,7 @@ #include #include -#include +#include +#include namespace DB { @@ -18,23 +19,29 @@ NamedCollectionFactory & NamedCollectionFactory::instance() return instance; } +NamedCollectionFactory::~NamedCollectionFactory() +{ + shutdown(); +} + +void NamedCollectionFactory::shutdown() +{ + shutdown_called = true; + if (update_task) + update_task->deactivate(); + metadata_storage.reset(); +} + bool NamedCollectionFactory::exists(const std::string & collection_name) const { std::lock_guard lock(mutex); - return existsUnlocked(collection_name, lock); -} - -bool NamedCollectionFactory::existsUnlocked( - const std::string & collection_name, - std::lock_guard & /* lock */) const -{ - return loaded_named_collections.contains(collection_name); + return exists(collection_name, lock); } NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_name) const { std::lock_guard lock(mutex); - auto collection = tryGetUnlocked(collection_name, lock); + auto collection = tryGet(collection_name, lock); if (!collection) { throw Exception( @@ -48,14 +55,35 @@ NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_na NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection_name) const { std::lock_guard lock(mutex); - return tryGetUnlocked(collection_name, lock); + return tryGet(collection_name, lock); +} + +NamedCollectionsMap NamedCollectionFactory::getAll() const +{ + std::lock_guard lock(mutex); + return loaded_named_collections; +} + +bool NamedCollectionFactory::exists(const std::string & collection_name, std::lock_guard &) const +{ + return loaded_named_collections.contains(collection_name); +} + +MutableNamedCollectionPtr NamedCollectionFactory::tryGet( + const std::string & collection_name, + std::lock_guard &) const +{ + auto it = loaded_named_collections.find(collection_name); + if (it == loaded_named_collections.end()) + return nullptr; + return it->second; } MutableNamedCollectionPtr NamedCollectionFactory::getMutable( - const std::string & collection_name) const + const std::string & collection_name, + std::lock_guard & lock) const { - std::lock_guard lock(mutex); - auto collection = tryGetUnlocked(collection_name, lock); + auto collection = tryGet(collection_name, lock); if (!collection) { throw Exception( @@ -74,35 +102,10 @@ MutableNamedCollectionPtr NamedCollectionFactory::getMutable( return collection; } -MutableNamedCollectionPtr NamedCollectionFactory::tryGetUnlocked( - const std::string & collection_name, - std::lock_guard & /* lock */) const -{ - auto it = loaded_named_collections.find(collection_name); - if (it == loaded_named_collections.end()) - return nullptr; - return it->second; -} - void NamedCollectionFactory::add( - const std::string & collection_name, - MutableNamedCollectionPtr collection) -{ - std::lock_guard lock(mutex); - addUnlocked(collection_name, collection, lock); -} - -void NamedCollectionFactory::add(NamedCollectionsMap collections) -{ - std::lock_guard lock(mutex); - for (const auto & [collection_name, collection] : collections) - addUnlocked(collection_name, collection, lock); -} - -void NamedCollectionFactory::addUnlocked( const std::string & collection_name, MutableNamedCollectionPtr collection, - std::lock_guard & /* lock */) + std::lock_guard &) { auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection); if (!inserted) @@ -114,10 +117,15 @@ void NamedCollectionFactory::addUnlocked( } } -void NamedCollectionFactory::remove(const std::string & collection_name) +void NamedCollectionFactory::add(NamedCollectionsMap collections, std::lock_guard & lock) { - std::lock_guard lock(mutex); - bool removed = removeIfExistsUnlocked(collection_name, lock); + for (const auto & [collection_name, collection] : collections) + add(collection_name, collection, lock); +} + +void NamedCollectionFactory::remove(const std::string & collection_name, std::lock_guard & lock) +{ + bool removed = removeIfExists(collection_name, lock); if (!removed) { throw Exception( @@ -127,17 +135,11 @@ void NamedCollectionFactory::remove(const std::string & collection_name) } } -void NamedCollectionFactory::removeIfExists(const std::string & collection_name) -{ - std::lock_guard lock(mutex); - removeIfExistsUnlocked(collection_name, lock); // NOLINT -} - -bool NamedCollectionFactory::removeIfExistsUnlocked( +bool NamedCollectionFactory::removeIfExists( const std::string & collection_name, std::lock_guard & lock) { - auto collection = tryGetUnlocked(collection_name, lock); + auto collection = tryGet(collection_name, lock); if (!collection) return false; @@ -153,20 +155,13 @@ bool NamedCollectionFactory::removeIfExistsUnlocked( return true; } -void NamedCollectionFactory::removeById(NamedCollection::SourceId id) +void NamedCollectionFactory::removeById(NamedCollection::SourceId id, std::lock_guard &) { - std::lock_guard lock(mutex); std::erase_if( loaded_named_collections, [&](const auto & value) { return value.second->getSourceId() == id; }); } -NamedCollectionsMap NamedCollectionFactory::getAll() const -{ - std::lock_guard lock(mutex); - return loaded_named_collections; -} - namespace { constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; @@ -223,37 +218,67 @@ namespace } } -void NamedCollectionFactory::loadFromConfig(const Poco::Util::AbstractConfiguration & config) +void NamedCollectionFactory::loadIfNot() { - add(getNamedCollections(config)); + std::lock_guard lock(mutex); + loadIfNot(lock); +} + +bool NamedCollectionFactory::loadIfNot(std::lock_guard & lock) +{ + if (loaded) + return false; + + auto context = Context::getGlobalContextInstance(); + metadata_storage = NamedCollectionsMetadataStorage::create(context); + + loadFromConfig(context->getConfigRef(), lock); + loadFromSQL(lock); + + if (metadata_storage->requiresPeriodicUpdate()) + { + update_task = context->getMessageBrokerSchedulePool().createTask("NamedCollectionsMetadataStorage", [this]{ updateFunc(); }); + update_task->activate(); + update_task->schedule(); + } + + loaded = true; + return true; +} + +void NamedCollectionFactory::loadFromConfig(const Poco::Util::AbstractConfiguration & config, std::lock_guard & lock) +{ + auto collections = getNamedCollections(config); + LOG_TEST(log, "Loaded {} collections from config", collections.size()); + add(std::move(collections), lock); } void NamedCollectionFactory::reloadFromConfig(const Poco::Util::AbstractConfiguration & config) { - auto collections = getNamedCollections(config); - removeById(NamedCollection::SourceId::CONFIG); - add(collections); -} - -void NamedCollectionFactory::loadFromSQL(const ContextPtr & context) -{ - add(NamedCollectionsMetadata::create(context)->getAll()); -} - -void NamedCollectionFactory::loadIfNot() -{ - if (loaded) + std::lock_guard lock(mutex); + if (loadIfNot(lock)) return; - auto global_context = Context::getGlobalContextInstance(); - loadFromConfig(global_context->getConfigRef()); - loadFromSQL(global_context); - loaded = true; + + auto collections = getNamedCollections(config); + LOG_TEST(log, "Loaded {} collections from config", collections.size()); + + removeById(NamedCollection::SourceId::CONFIG, lock); + add(std::move(collections), lock); } -void NamedCollectionFactory::createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context) +void NamedCollectionFactory::loadFromSQL(std::lock_guard & lock) { - loadIfNot(); - if (exists(query.collection_name)) + auto collections = metadata_storage->getAll(); + LOG_TEST(log, "Loaded {} collections from sql", collections.size()); + add(std::move(collections), lock); +} + +void NamedCollectionFactory::createFromSQL(const ASTCreateNamedCollectionQuery & query) +{ + std::lock_guard lock(mutex); + loadIfNot(lock); + + if (exists(query.collection_name, lock)) { if (query.if_not_exists) return; @@ -263,13 +288,16 @@ void NamedCollectionFactory::createFromSQL(const ASTCreateNamedCollectionQuery & "A named collection `{}` already exists", query.collection_name); } - add(query.collection_name, NamedCollectionsMetadata::create(context)->create(query)); + + add(query.collection_name, metadata_storage->create(query), lock); } -void NamedCollectionFactory::removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context) +void NamedCollectionFactory::removeFromSQL(const ASTDropNamedCollectionQuery & query) { - loadIfNot(); - if (!exists(query.collection_name)) + std::lock_guard lock(mutex); + loadIfNot(lock); + + if (!exists(query.collection_name, lock)) { if (query.if_exists) return; @@ -279,14 +307,17 @@ void NamedCollectionFactory::removeFromSQL(const ASTDropNamedCollectionQuery & q "Cannot remove collection `{}`, because it doesn't exist", query.collection_name); } - NamedCollectionsMetadata::create(context)->remove(query.collection_name); - remove(query.collection_name); + + metadata_storage->remove(query.collection_name); + remove(query.collection_name, lock); } -void NamedCollectionFactory::updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context) +void NamedCollectionFactory::updateFromSQL(const ASTAlterNamedCollectionQuery & query) { - loadIfNot(); - if (!exists(query.collection_name)) + std::lock_guard lock(mutex); + loadIfNot(lock); + + if (!exists(query.collection_name, lock)) { if (query.if_exists) return; @@ -296,9 +327,10 @@ void NamedCollectionFactory::updateFromSQL(const ASTAlterNamedCollectionQuery & "Cannot remove collection `{}`, because it doesn't exist", query.collection_name); } - NamedCollectionsMetadata::create(context)->update(query); - auto collection = getMutable(query.collection_name); + metadata_storage->update(query); + + auto collection = getMutable(query.collection_name, lock); auto collection_lock = collection->lock(); for (const auto & [name, value] : query.changes) @@ -314,4 +346,58 @@ void NamedCollectionFactory::updateFromSQL(const ASTAlterNamedCollectionQuery & collection->remove(key); } +void NamedCollectionFactory::reloadFromSQL() +{ + std::lock_guard lock(mutex); + if (loadIfNot(lock)) + return; + + auto collections = metadata_storage->getAll(); + removeById(NamedCollection::SourceId::SQL, lock); + add(std::move(collections), lock); +} + +void NamedCollectionFactory::updateFunc() +{ + LOG_TRACE(log, "Named collections background updating thread started"); + + while (!shutdown_called.load()) + { + NamedCollectionsMap collections; + try + { + reloadFromSQL(); + } + catch (const Coordination::Exception & e) + { + if (Coordination::isHardwareError(e.code)) + { + LOG_INFO(log, "Lost ZooKeeper connection, will try to connect again: {}", + DB::getCurrentExceptionMessage(true)); + + sleepForSeconds(1); + } + else + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); + } + continue; + } + catch (...) + { + DB::tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); + continue; + } + + if (shutdown_called.load()) + break; + + metadata_storage->waitUpdate(); + } + + LOG_TRACE(log, "Named collections background updating thread finished"); +} + } diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.h b/src/Common/NamedCollections/NamedCollectionsFactory.h index 20681d6c752..6ee5940e686 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.h +++ b/src/Common/NamedCollections/NamedCollectionsFactory.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include namespace DB @@ -13,67 +14,70 @@ class NamedCollectionFactory : boost::noncopyable public: static NamedCollectionFactory & instance(); + ~NamedCollectionFactory(); + bool exists(const std::string & collection_name) const; NamedCollectionPtr get(const std::string & collection_name) const; NamedCollectionPtr tryGet(const std::string & collection_name) const; - MutableNamedCollectionPtr getMutable(const std::string & collection_name) const; - - void add(const std::string & collection_name, MutableNamedCollectionPtr collection); - - void add(NamedCollectionsMap collections); - - void update(NamedCollectionsMap collections); - - void remove(const std::string & collection_name); - - void removeIfExists(const std::string & collection_name); - - void removeById(NamedCollection::SourceId id); - NamedCollectionsMap getAll() const; - void loadIfNot(); - void reloadFromConfig(const Poco::Util::AbstractConfiguration & config); - void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context); + void reloadFromSQL(); - void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context); + void createFromSQL(const ASTCreateNamedCollectionQuery & query); - void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context); + void removeFromSQL(const ASTDropNamedCollectionQuery & query); - /// This method is public only for unit tests. - void loadFromConfig(const Poco::Util::AbstractConfiguration & config); + void updateFromSQL(const ASTAlterNamedCollectionQuery & query); -private: - bool existsUnlocked( - const std::string & collection_name, - std::lock_guard & lock) const; + void loadIfNot(); - MutableNamedCollectionPtr tryGetUnlocked( - const std::string & collection_name, - std::lock_guard & lock) const; - - void addUnlocked( - const std::string & collection_name, - MutableNamedCollectionPtr collection, - std::lock_guard & lock); - - bool removeIfExistsUnlocked( - const std::string & collection_name, - std::lock_guard & lock); + void shutdown(); +protected: mutable NamedCollectionsMap loaded_named_collections; - - LoggerPtr log = getLogger("NamedCollectionFactory"); mutable std::mutex mutex; - bool is_initialized = false; - bool loaded = false; - void loadFromSQL(const ContextPtr & context); + const LoggerPtr log = getLogger("NamedCollectionFactory"); + + bool loaded = false; + std::atomic shutdown_called = false; + std::unique_ptr metadata_storage; + BackgroundSchedulePool::TaskHolder update_task; + + bool loadIfNot(std::lock_guard & lock); + + bool exists( + const std::string & collection_name, + std::lock_guard & lock) const; + + MutableNamedCollectionPtr getMutable(const std::string & collection_name, std::lock_guard & lock) const; + + void add(const std::string & collection_name, MutableNamedCollectionPtr collection, std::lock_guard & lock); + + void add(NamedCollectionsMap collections, std::lock_guard & lock); + + void update(NamedCollectionsMap collections, std::lock_guard & lock); + + void remove(const std::string & collection_name, std::lock_guard & lock); + + bool removeIfExists(const std::string & collection_name, std::lock_guard & lock); + + MutableNamedCollectionPtr tryGet(const std::string & collection_name, std::lock_guard & lock) const; + + void removeById(NamedCollection::SourceId id, std::lock_guard & lock); + + void loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + std::lock_guard & lock); + + void loadFromSQL(std::lock_guard & lock); + + void updateFunc(); }; } diff --git a/src/Common/NamedCollections/NamedCollectionsMetadata.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp similarity index 77% rename from src/Common/NamedCollections/NamedCollectionsMetadata.cpp rename to src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index 842c913aaba..a82c0110023 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadata.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -26,6 +26,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +static const std::string named_collections_storage_config_path = "named_collections_storage"; + namespace { MutableNamedCollectionPtr createNamedCollectionFromAST(const ASTCreateNamedCollectionQuery & query) @@ -47,7 +49,7 @@ namespace } } -class NamedCollectionsMetadata::INamedCollectionsStorage +class NamedCollectionsMetadataStorage::INamedCollectionsStorage { public: virtual ~INamedCollectionsStorage() = default; @@ -63,10 +65,12 @@ public: virtual void remove(const std::string & path) = 0; virtual bool removeIfExists(const std::string & path) = 0; + + virtual void waitUpdate(size_t /* timeout */) {} }; -class NamedCollectionsMetadata::LocalStorage : public INamedCollectionsStorage, private WithContext +class NamedCollectionsMetadataStorage::LocalStorage : public INamedCollectionsStorage, private WithContext { private: std::string root_path; @@ -181,11 +185,12 @@ private: }; -class NamedCollectionsMetadata::ZooKeeperStorage : public INamedCollectionsStorage, private WithContext +class NamedCollectionsMetadataStorage::ZooKeeperStorage : public INamedCollectionsStorage, private WithContext { private: std::string root_path; mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; + mutable zkutil::EventPtr wait_event; public: ZooKeeperStorage(ContextPtr context_, const std::string & path_) @@ -210,9 +215,17 @@ public: ~ZooKeeperStorage() override = default; + void waitUpdate(size_t timeout) override + { + if (wait_event) + wait_event->tryWait(timeout); + } + std::vector list() const override { - return getClient()->getChildren(root_path); + if (!wait_event) + wait_event = std::make_shared(); + return getClient()->getChildren(root_path, nullptr, wait_event); } bool exists(const std::string & path) const override @@ -277,40 +290,21 @@ private: } }; -std::unique_ptr NamedCollectionsMetadata::create(const ContextPtr & context_) +NamedCollectionsMetadataStorage::NamedCollectionsMetadataStorage( + std::shared_ptr storage_, + ContextPtr context_) + : WithContext(context_) + , storage(std::move(storage_)) { - static const std::string storage_config_path = "named_collections_storage"; - - const auto & config = context_->getConfigRef(); - const auto storage_type = config.getString(storage_config_path + ".type", "local"); - - if (storage_type == "local") - { - const auto path = config.getString( - storage_config_path + ".path", - std::filesystem::path(context_->getPath()) / "named_collections"); - - auto local_storage = std::make_unique(context_, path); - return std::make_unique(std::move(local_storage), context_); - } - if (storage_type == "zookeeper") - { - auto zk_storage = std::make_unique(context_, config.getString(storage_config_path + ".path")); - return std::make_unique(std::move(zk_storage), context_); - } - - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, - "Unknown storage for named collections: {}", storage_type); } -MutableNamedCollectionPtr NamedCollectionsMetadata::get(const std::string & collection_name) const +MutableNamedCollectionPtr NamedCollectionsMetadataStorage::get(const std::string & collection_name) const { const auto query = readCreateQuery(collection_name); return createNamedCollectionFromAST(query); } -NamedCollectionsMap NamedCollectionsMetadata::getAll() const +NamedCollectionsMap NamedCollectionsMetadataStorage::getAll() const { NamedCollectionsMap result; for (const auto & collection_name : listCollections()) @@ -327,23 +321,23 @@ NamedCollectionsMap NamedCollectionsMetadata::getAll() const return result; } -MutableNamedCollectionPtr NamedCollectionsMetadata::create(const ASTCreateNamedCollectionQuery & query) +MutableNamedCollectionPtr NamedCollectionsMetadataStorage::create(const ASTCreateNamedCollectionQuery & query) { writeCreateQuery(query); return createNamedCollectionFromAST(query); } -void NamedCollectionsMetadata::remove(const std::string & collection_name) +void NamedCollectionsMetadataStorage::remove(const std::string & collection_name) { storage->remove(getFileName(collection_name)); } -bool NamedCollectionsMetadata::removeIfExists(const std::string & collection_name) +bool NamedCollectionsMetadataStorage::removeIfExists(const std::string & collection_name) { return storage->removeIfExists(getFileName(collection_name)); } -void NamedCollectionsMetadata::update(const ASTAlterNamedCollectionQuery & query) +void NamedCollectionsMetadataStorage::update(const ASTAlterNamedCollectionQuery & query) { auto create_query = readCreateQuery(query.collection_name); @@ -403,7 +397,7 @@ void NamedCollectionsMetadata::update(const ASTAlterNamedCollectionQuery & query writeCreateQuery(create_query, true); } -std::vector NamedCollectionsMetadata::listCollections() const +std::vector NamedCollectionsMetadataStorage::listCollections() const { auto paths = storage->list(); std::vector collections; @@ -413,7 +407,7 @@ std::vector NamedCollectionsMetadata::listCollections() const return collections; } -ASTCreateNamedCollectionQuery NamedCollectionsMetadata::readCreateQuery(const std::string & collection_name) const +ASTCreateNamedCollectionQuery NamedCollectionsMetadataStorage::readCreateQuery(const std::string & collection_name) const { const auto path = getFileName(collection_name); auto query = storage->read(path); @@ -425,7 +419,7 @@ ASTCreateNamedCollectionQuery NamedCollectionsMetadata::readCreateQuery(const st return create_query; } -void NamedCollectionsMetadata::writeCreateQuery(const ASTCreateNamedCollectionQuery & query, bool replace) +void NamedCollectionsMetadataStorage::writeCreateQuery(const ASTCreateNamedCollectionQuery & query, bool replace) { auto normalized_query = query.clone(); auto & changes = typeid_cast(normalized_query.get())->changes; @@ -436,4 +430,51 @@ void NamedCollectionsMetadata::writeCreateQuery(const ASTCreateNamedCollectionQu storage->write(getFileName(query.collection_name), serializeAST(*normalized_query), replace); } +bool NamedCollectionsMetadataStorage::requiresPeriodicUpdate() const +{ + const auto & config = Context::getGlobalContextInstance()->getConfigRef(); + return config.has(named_collections_storage_config_path + ".update_timeout_ms"); +} + +void NamedCollectionsMetadataStorage::waitUpdate() +{ + const auto & config = Context::getGlobalContextInstance()->getConfigRef(); + storage->waitUpdate(config.getUInt(named_collections_storage_config_path + ".update_timeout_ms")); +} + +std::unique_ptr NamedCollectionsMetadataStorage::create(const ContextPtr & context_) +{ + const auto & config = context_->getConfigRef(); + const auto storage_type = config.getString(named_collections_storage_config_path + ".type", "local"); + + if (storage_type == "local") + { + const auto path = config.getString( + named_collections_storage_config_path + ".path", + std::filesystem::path(context_->getPath()) / "named_collections"); + + LOG_TRACE(getLogger("NamedCollectionsMetadataStorage"), + "Using local storage for named collections at path: {}", path); + + auto local_storage = std::make_unique(context_, path); + return std::unique_ptr( + new NamedCollectionsMetadataStorage(std::move(local_storage), context_)); + } + if (storage_type == "zookeeper" || storage_type == "keeper") + { + const auto path = config.getString(named_collections_storage_config_path + ".path"); + auto zk_storage = std::make_unique(context_, path); + + LOG_TRACE(getLogger("NamedCollectionsMetadataStorage"), + "Using zookeeper storage for named collections at path: {}", path); + + return std::unique_ptr( + new NamedCollectionsMetadataStorage(std::move(zk_storage), context_)); + } + + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "Unknown storage for named collections: {}", storage_type); +} + } diff --git a/src/Common/NamedCollections/NamedCollectionsMetadata.h b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h similarity index 66% rename from src/Common/NamedCollections/NamedCollectionsMetadata.h rename to src/Common/NamedCollections/NamedCollectionsMetadataStorage.h index 4a432b14b96..059a3c7cf76 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadata.h +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h @@ -3,16 +3,17 @@ #include #include #include +#include namespace DB { +class NamedCollectionsMetadataStorage; +std::unique_ptr checkKek(const ContextPtr & context); -class NamedCollectionsMetadata : private WithContext +class NamedCollectionsMetadataStorage : private WithContext { public: - static std::unique_ptr create(const ContextPtr & context); - - ~NamedCollectionsMetadata() = default; + static std::unique_ptr create(const ContextPtr & context); NamedCollectionsMap getAll() const; @@ -26,18 +27,21 @@ public: void update(const ASTAlterNamedCollectionQuery & query); - class INamedCollectionsStorage; - NamedCollectionsMetadata(std::shared_ptr storage_, ContextPtr context_) - : WithContext(context_) - , storage(std::move(storage_)) {} - /// FIXME: It should be a protected constructor, but I failed make create() method a proper friend. + void shutdown(); + + void waitUpdate(); + + bool requiresPeriodicUpdate() const; private: + class INamedCollectionsStorage; class LocalStorage; class ZooKeeperStorage; std::shared_ptr storage; + NamedCollectionsMetadataStorage(std::shared_ptr storage_, ContextPtr context_); + std::vector listCollections() const; ASTCreateNamedCollectionQuery readCreateQuery(const std::string & collection_name) const; diff --git a/src/Common/tests/gtest_named_collections.cpp b/src/Common/tests/gtest_named_collections.cpp index 0320f825764..8d9aa2bc213 100644 --- a/src/Common/tests/gtest_named_collections.cpp +++ b/src/Common/tests/gtest_named_collections.cpp @@ -6,6 +6,35 @@ using namespace DB; +/// A class which allows to test private methods of NamedCollectionFactory. +class NamedCollectionFactoryFriend : public NamedCollectionFactory +{ +public: + static NamedCollectionFactoryFriend & instance() + { + static NamedCollectionFactoryFriend instance; + return instance; + } + + void loadFromConfig(const Poco::Util::AbstractConfiguration & config) + { + std::lock_guard lock(mutex); + NamedCollectionFactory::loadFromConfig(config, lock); + } + + void add(const std::string & collection_name, MutableNamedCollectionPtr collection) + { + std::lock_guard lock(mutex); + NamedCollectionFactory::add(collection_name, collection, lock); + } + + void remove(const std::string & collection_name) + { + std::lock_guard lock(mutex); + NamedCollectionFactory::remove(collection_name, lock); + } +}; + TEST(NamedCollections, SimpleConfig) { std::string xml(R"CONFIG( @@ -28,13 +57,13 @@ TEST(NamedCollections, SimpleConfig) Poco::AutoPtr document = dom_parser.parseString(xml); Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); - NamedCollectionFactory::instance().loadFromConfig(*config); + NamedCollectionFactoryFriend::instance().loadFromConfig(*config); - ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); - ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2")); - ASSERT_TRUE(NamedCollectionFactory::instance().tryGet("collection3") == nullptr); + ASSERT_TRUE(NamedCollectionFactoryFriend::instance().exists("collection1")); + ASSERT_TRUE(NamedCollectionFactoryFriend::instance().exists("collection2")); + ASSERT_TRUE(NamedCollectionFactoryFriend::instance().tryGet("collection3") == nullptr); - auto collections = NamedCollectionFactory::instance().getAll(); + auto collections = NamedCollectionFactoryFriend::instance().getAll(); ASSERT_EQ(collections.size(), 2); ASSERT_TRUE(collections.contains("collection1")); ASSERT_TRUE(collections.contains("collection2")); @@ -46,7 +75,7 @@ key3: 3.3 key4: -4 )CONFIG"); - auto collection1 = NamedCollectionFactory::instance().get("collection1"); + auto collection1 = NamedCollectionFactoryFriend::instance().get("collection1"); ASSERT_TRUE(collection1 != nullptr); ASSERT_TRUE(collection1->get("key1") == "value1"); @@ -60,7 +89,7 @@ key5: 5 key6: 6.6 )CONFIG"); - auto collection2 = NamedCollectionFactory::instance().get("collection2"); + auto collection2 = NamedCollectionFactoryFriend::instance().get("collection2"); ASSERT_TRUE(collection2 != nullptr); ASSERT_TRUE(collection2->get("key4") == "value4"); @@ -68,9 +97,9 @@ key6: 6.6 ASSERT_TRUE(collection2->get("key6") == 6.6); auto collection2_copy = collections["collection2"]->duplicate(); - NamedCollectionFactory::instance().add("collection2_copy", collection2_copy); - ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2_copy")); - ASSERT_EQ(NamedCollectionFactory::instance().get("collection2_copy")->dumpStructure(), + NamedCollectionFactoryFriend::instance().add("collection2_copy", collection2_copy); + ASSERT_TRUE(NamedCollectionFactoryFriend::instance().exists("collection2_copy")); + ASSERT_EQ(NamedCollectionFactoryFriend::instance().get("collection2_copy")->dumpStructure(), R"CONFIG(key4: value4 key5: 5 key6: 6.6 @@ -87,8 +116,8 @@ key6: 6.6 collection2_copy->setOrUpdate("key4", "value45", {}); ASSERT_EQ(collection2_copy->getOrDefault("key4", "N"), "value45"); - NamedCollectionFactory::instance().remove("collection2_copy"); - ASSERT_FALSE(NamedCollectionFactory::instance().exists("collection2_copy")); + NamedCollectionFactoryFriend::instance().remove("collection2_copy"); + ASSERT_FALSE(NamedCollectionFactoryFriend::instance().exists("collection2_copy")); config.reset(); } @@ -118,11 +147,11 @@ TEST(NamedCollections, NestedConfig) Poco::AutoPtr document = dom_parser.parseString(xml); Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); - NamedCollectionFactory::instance().loadFromConfig(*config); + NamedCollectionFactoryFriend::instance().loadFromConfig(*config); - ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection3")); + ASSERT_TRUE(NamedCollectionFactoryFriend::instance().exists("collection3")); - auto collection = NamedCollectionFactory::instance().get("collection3"); + auto collection = NamedCollectionFactoryFriend::instance().get("collection3"); ASSERT_TRUE(collection != nullptr); ASSERT_EQ(collection->dumpStructure(), @@ -170,8 +199,8 @@ TEST(NamedCollections, NestedConfigDuplicateKeys) Poco::AutoPtr document = dom_parser.parseString(xml); Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); - NamedCollectionFactory::instance().loadFromConfig(*config); - auto collection = NamedCollectionFactory::instance().get("collection"); + NamedCollectionFactoryFriend::instance().loadFromConfig(*config); + auto collection = NamedCollectionFactoryFriend::instance().get("collection"); auto keys = collection->getKeys(); ASSERT_EQ(keys.size(), 6); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5c9ae4716b9..83ccb22dc7f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -610,6 +611,8 @@ struct ContextSharedPart : boost::noncopyable LOG_TRACE(log, "Shutting down database catalog"); DatabaseCatalog::shutdown(); + NamedCollectionFactory::instance().shutdown(); + delete_async_insert_queue.reset(); SHUTDOWN(log, "merges executor", merge_mutate_executor, wait()); diff --git a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp index 30935b3b89d..79a17fd1844 100644 --- a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp @@ -23,7 +23,7 @@ BlockIO InterpreterAlterNamedCollectionQuery::execute() return executeDDLQueryOnCluster(query_ptr, current_context, params); } - NamedCollectionFactory::instance().updateFromSQL(query, current_context); + NamedCollectionFactory::instance().updateFromSQL(query); return {}; } diff --git a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp index 91c19042389..c71441daa8c 100644 --- a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp @@ -23,7 +23,7 @@ BlockIO InterpreterCreateNamedCollectionQuery::execute() return executeDDLQueryOnCluster(query_ptr, current_context, params); } - NamedCollectionFactory::instance().createFromSQL(query, current_context); + NamedCollectionFactory::instance().createFromSQL(query); return {}; } diff --git a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp index 62fffc82049..2edaef1b2f2 100644 --- a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp @@ -23,7 +23,7 @@ BlockIO InterpreterDropNamedCollectionQuery::execute() return executeDDLQueryOnCluster(query_ptr, current_context, params); } - NamedCollectionFactory::instance().removeFromSQL(query, current_context); + NamedCollectionFactory::instance().removeFromSQL(query); return {}; } diff --git a/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml index 4111bdec4eb..2d7946d1587 100644 --- a/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml +++ b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml @@ -2,6 +2,7 @@ zookeeper /named_collections_path/ + 5000 diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 3460ed2e564..dbc502236c0 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -37,6 +37,17 @@ def cluster(): stay_alive=True, with_zookeeper=True, ) + cluster.add_instance( + "node_with_keeper_2", + main_configs=[ + "configs/config.d/named_collections_with_zookeeper.xml", + ], + user_configs=[ + "configs/users.d/users.xml", + ], + stay_alive=True, + with_zookeeper=True, + ) cluster.add_instance( "node_only_named_collection_control", main_configs=[ @@ -576,6 +587,7 @@ def test_sql_commands(cluster, with_keeper): node.query( "ALTER NAMED COLLECTION collection2 SET key3=3, key4='value4' DELETE key1" ) + time.sleep(2) def check_altered_and_deleted(): assert ( @@ -627,3 +639,125 @@ def test_sql_commands(cluster, with_keeper): check_dropped() node.restart_clickhouse() check_dropped() + + +def test_keeper_storage(cluster): + node1 = cluster.instances["node_with_keeper"] + node2 = cluster.instances["node_with_keeper_2"] + zk = cluster.get_kazoo_client("zoo1") + + assert "1" == node1.query("select count() from system.named_collections").strip() + assert "1" == node2.query("select count() from system.named_collections").strip() + + node1.query("CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'") + + def check_created(node): + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections").strip() + ) + + assert ( + "['key1','key2']" + == node.query( + "select mapKeys(collection) from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "1" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "value2" + == node.query( + "select collection['key2'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 1, key2 = 'value2'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + + check_created(node1) + check_created(node2) + + node1.restart_clickhouse() + node2.restart_clickhouse() + + check_created(node1) + check_created(node2) + + node2.query("ALTER NAMED COLLECTION collection2 SET key1=4, key3='value3'") + + time.sleep(5) + + def check_altered(node): + assert ( + "['key1','key2','key3']" + == node.query( + "select mapKeys(collection) from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "4" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "value3" + == node.query( + "select collection['key3'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 4, key2 = 'value2', key3 = 'value3'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + + check_altered(node2) + check_altered(node1) + + node1.restart_clickhouse() + node2.restart_clickhouse() + + check_altered(node1) + check_altered(node2) + + node1.query("DROP NAMED COLLECTION collection2") + + time.sleep(5) + + def check_dropped(node): + assert "1" == node.query("select count() from system.named_collections").strip() + assert ( + "collection1" + == node.query("select name from system.named_collections").strip() + ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 0 == len(children) + + check_dropped(node1) + check_dropped(node2) + + node1.restart_clickhouse() + node2.restart_clickhouse() + + check_dropped(node1) + check_dropped(node2) From c9ce1362a19fd344403100d1fca2351e9ec700b4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 31 May 2024 12:50:29 +0000 Subject: [PATCH 229/856] Fix early constant folding for isNull/isNotNul and analyzer. --- src/DataTypes/DataTypeNullable.cpp | 14 +++++++++ src/DataTypes/DataTypeNullable.h | 2 ++ src/Functions/isNotNull.cpp | 12 ++++++++ src/Functions/isNull.cpp | 12 ++++++++ src/Functions/isNullable.cpp | 10 +++++++ src/Planner/PlannerExpressionAnalysis.cpp | 9 ++++-- ..._early_constant_folding_analyzer.reference | 1 + .../03164_early_constant_folding_analyzer.sql | 30 +++++++++++++++++++ 8 files changed, 88 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03164_early_constant_folding_analyzer.reference create mode 100644 tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index db252659d41..d97edef5c39 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -174,4 +175,17 @@ DataTypePtr removeNullableOrLowCardinalityNullable(const DataTypePtr & type) } +bool canContainNull(const IDataType & type) + { + if (type.isNullable() || type.isLowCardinalityNullable() || isDynamic(type)) + return true; + + if (const auto * variant = typeid_cast(&type)) + for (const auto & elem : variant->getVariants()) + if (canContainNull(*elem)) + return true; + + return false; + } + } diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 71abe48c151..7a8a54fdf3a 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -62,4 +62,6 @@ DataTypePtr makeNullableOrLowCardinalityNullableSafe(const DataTypePtr & type); /// Nullable(T) -> T, LowCardinality(Nullable(T)) -> T DataTypePtr removeNullableOrLowCardinalityNullable(const DataTypePtr & type); +bool canContainNull(const IDataType & type); + } diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index ea95a5c2b1c..a10e7ebd40c 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -29,6 +29,18 @@ public: return name; } + ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override + { + const ColumnWithTypeAndName & elem = arguments[0]; + if (elem.type->onlyNull()) + return result_type->createColumnConst(1, UInt8(0)); + + if (canContainNull(*elem.type)) + return nullptr; + + return result_type->createColumnConst(1, UInt8(1)); + } + size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index a98ff2ab8e8..95d659b103b 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -31,6 +31,18 @@ public: return name; } + ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override + { + const ColumnWithTypeAndName & elem = arguments[0]; + if (elem.type->onlyNull()) + return result_type->createColumnConst(1, UInt8(1)); + + if (canContainNull(*elem.type)) + return nullptr; + + return result_type->createColumnConst(1, UInt8(0)); + } + size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } diff --git a/src/Functions/isNullable.cpp b/src/Functions/isNullable.cpp index 14874487f40..b24ee4f5e73 100644 --- a/src/Functions/isNullable.cpp +++ b/src/Functions/isNullable.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -23,6 +24,15 @@ public: return name; } + ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override + { + const ColumnWithTypeAndName & elem = arguments[0]; + if (elem.type->onlyNull() || canContainNull(*elem.type)) + return result_type->createColumnConst(1, UInt8(1)); + + return result_type->createColumnConst(1, UInt8(0)); + } + bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForNothing() const override { return false; } diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 2a95234057c..c9913005b5d 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -37,7 +38,7 @@ namespace * Actions before filter are added into into actions chain. * It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized. */ -FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, +std::optional analyzeFilter(const QueryTreeNodePtr & filter_expression_node, const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) @@ -45,7 +46,11 @@ 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; + const auto * output = result.filter_actions->getOutputs().at(0); + if (output->column && ConstantFilterDescription(*output->column).always_true) + return {}; + + result.filter_column_name = output->result_name; actions_chain.addStep(std::make_unique(result.filter_actions)); return result; diff --git a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.reference b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.reference new file mode 100644 index 00000000000..227b118bb7f --- /dev/null +++ b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.reference @@ -0,0 +1 @@ +ReadFromPreparedSource (Optimized trivial count) diff --git a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql new file mode 100644 index 00000000000..37675c96dd1 --- /dev/null +++ b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql @@ -0,0 +1,30 @@ +CREATE TABLE checks +( + `pull_request_number` UInt32, + `commit_sha` LowCardinality(String), + `check_name` LowCardinality(String), + `check_status` LowCardinality(String), + `check_duration_ms` UInt64, + `check_start_time` DateTime, + `test_name` LowCardinality(String), + `test_status` LowCardinality(String), + `test_duration_ms` UInt64, + `report_url` String, + `pull_request_url` String, + `commit_url` String, + `task_url` String, + `base_ref` String, + `base_repo` String, + `head_ref` String, + `head_repo` String, + `test_context_raw` String, + `instance_type` LowCardinality(String), + `instance_id` String, + `date` Date MATERIALIZED toDate(check_start_time) +) +ENGINE = MergeTree ORDER BY (date, pull_request_number, commit_sha, check_name, test_name, check_start_time); + +insert into checks select * from generateRandom() limit 1; + + +select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL SETTINGS allow_experimental_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 0) where explain like '%ReadFromPreparedSource%'; \ No newline at end of file From c857099c3d19cce02e3ff80f3bf3faa7656eac28 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 31 May 2024 13:42:16 +0000 Subject: [PATCH 230/856] 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 779eaa01ec54ad06a8ceff4b9efc125439a9671c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 31 May 2024 17:51:00 +0200 Subject: [PATCH 231/856] Fix tests --- src/Backups/BackupIO_S3.cpp | 34 ++++++++++------ .../ObjectStorages/S3/S3ObjectStorage.cpp | 9 ++++- src/Disks/ObjectStorages/S3/diskSettings.cpp | 3 +- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- src/IO/S3Common.cpp | 39 +------------------ src/IO/S3Common.h | 12 +----- src/IO/S3Settings.cpp | 10 +++-- .../ObjectStorage/S3/Configuration.cpp | 9 +++-- 8 files changed, 49 insertions(+), 69 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 2d8ae7f1b00..273d4b4ebe8 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -127,12 +127,18 @@ BackupReaderS3::BackupReaderS3( : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderS3")) , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} - , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup).value_or(S3Settings{})) { - auto & request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); - request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint - request_settings.allow_native_copy = allow_s3_native_copy; + auto endpoint_settings = context_->getStorageS3Settings().getSettings( + s3_uri.uri.toString(), + context_->getUserName(), + /*ignore_user=*/is_internal_backup); + + if (endpoint_settings.has_value()) + s3_settings = endpoint_settings.value(); + + s3_settings.request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); + s3_settings.request_settings.allow_native_copy = allow_s3_native_copy; + client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); if (auto blob_storage_system_log = context_->getBlobStorageLog()) @@ -219,13 +225,19 @@ BackupWriterS3::BackupWriterS3( : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3")) , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} - , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup).value_or(S3Settings{})) { - auto & request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); - request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint - request_settings.allow_native_copy = allow_s3_native_copy; - request_settings.storage_class_name = storage_class_name; + auto endpoint_settings = context_->getStorageS3Settings().getSettings( + s3_uri.uri.toString(), + context_->getUserName(), + /*ignore_user=*/is_internal_backup); + + if (endpoint_settings.has_value()) + s3_settings = endpoint_settings.value(); + + s3_settings.request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); + s3_settings.request_settings.allow_native_copy = allow_s3_native_copy; + s3_settings.request_settings.storage_class_name = storage_class_name; + client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); if (auto blob_storage_system_log = context_->getBlobStorageLog()) { diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 9b862d2c0b3..56cf7a1b9e6 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -262,9 +262,11 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN /// NOTE: For background operations settings are not propagated from session or query. They are taken from /// default user's .xml config. It's obscure and unclear behavior. For them it's always better /// to rely on settings from disk. - if (auto query_context = CurrentThread::getQueryContext(); query_context && !query_context->isBackgroundOperationContext()) + if (auto query_context = CurrentThread::getQueryContext(); + query_context && !query_context->isBackgroundOperationContext()) { - request_settings.updateFromSettings(query_context->getSettingsRef(), /* if_changed */true); + const auto & settings = query_context->getSettingsRef(); + request_settings.updateFromSettings(settings, /* if_changed */true, settings.s3_validate_request_settings); } ThreadPoolCallbackRunnerUnsafe scheduler; @@ -586,7 +588,10 @@ void S3ObjectStorage::applyNewSettings( modified_settings->request_settings.updateIfChanged(settings_from_config->request_settings); if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString(), context->getUserName())) + { modified_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); + modified_settings->request_settings.updateIfChanged(endpoint_settings->request_settings); + } auto current_settings = s3_settings.get(); if (options.allow_client_change diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index bdaacbf62cd..62df98f51e6 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -38,8 +38,9 @@ std::unique_ptr getSettings( bool validate_settings) { const auto & settings = context->getSettingsRef(); + auto auth_settings = S3::AuthSettings(config, settings, config_prefix); - auto request_settings = S3::RequestSettings(config, settings, config_prefix, validate_settings); + auto request_settings = S3::RequestSettings(config, settings, config_prefix, "s3_", validate_settings); request_settings.proxy_resolver = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( ProxyConfiguration::protocolFromString(S3::URI(endpoint).uri.getScheme()), config_prefix, config); diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 7b2762613b6..fd43f31a009 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -47,7 +47,7 @@ namespace auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { auto disk = DiskFactory::instance().create( - disk_name, *config, "", context, disks_map, /* attach */attach, /* custom_disk */true); + disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true); /// Mark that disk can be used without storage policy. disk->markDiskAsCustom(); return disk; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index a545d12aade..54881227d13 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -127,22 +127,13 @@ static bool setValueFromConfig( AuthSettings::AuthSettings( const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, - const std::string & config_prefix, - const std::string & fallback_config_prefix) + const std::string & config_prefix) { - if (config_prefix.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config path cannot be empty"); - for (auto & field : allMutable()) { auto path = fmt::format("{}.{}", config_prefix, field.getName()); - auto fallback_path = fallback_config_prefix.empty() ? "" : fmt::format("{}.{}", fallback_config_prefix, field.getName()); bool updated = setValueFromConfig(config, path, field); - - if (!updated && !fallback_path.empty()) - updated = setValueFromConfig(config, fallback_path, field); - if (!updated) { auto setting_name = "s3_" + field.getName(); @@ -208,40 +199,14 @@ RequestSettings::RequestSettings( const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, const std::string & config_prefix, - bool validate_settings) - : RequestSettings( - config, - settings, - config_prefix, - validate_settings, - config_prefix == "s3" ? "" : "s3_", /* setting_name_prefix */ - config_prefix == "s3" ? "" : "s3", /* fallback_config_prefix */ - "") /* fallback_setting_name_prefix */ -{ -} - -RequestSettings::RequestSettings( - const Poco::Util::AbstractConfiguration & config, - const DB::Settings & settings, - const std::string & config_prefix, - bool validate_settings, const std::string & setting_name_prefix, - const std::string & fallback_config_prefix, - const std::string & fallback_setting_name_prefix) + bool validate_settings) { - if (config_prefix.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config path cannot be empty"); - for (auto & field : allMutable()) { auto path = fmt::format("{}.{}{}", config_prefix, setting_name_prefix, field.getName()); - auto fallback_path = fallback_config_prefix.empty() ? "" : fmt::format("{}.{}{}", fallback_config_prefix, fallback_setting_name_prefix, field.getName()); bool updated = setValueFromConfig(config, path, field); - - if (!updated && !fallback_path.empty()) - updated = setValueFromConfig(config, fallback_path, field); - if (!updated) { auto setting_name = "s3_" + field.getName(); diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 6391075a5ab..68d44b0ed01 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -144,8 +144,7 @@ struct AuthSettings : public BaseSettings AuthSettings( const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, - const std::string & config_prefix, - const std::string & fallback_config_prefix = ""); + const std::string & config_prefix); AuthSettings(const DB::Settings & settings); @@ -170,6 +169,7 @@ struct RequestSettings : public BaseSettings const Poco::Util::AbstractConfiguration & config, const DB::Settings & settings, const std::string & config_prefix, + const std::string & setting_name_prefix = "", bool validate_settings = true); /// Create request settings from DB::Settings. @@ -190,14 +190,6 @@ struct RequestSettings : public BaseSettings std::shared_ptr proxy_resolver; private: - RequestSettings( - const Poco::Util::AbstractConfiguration & config, - const DB::Settings & settings, - const std::string & config_prefix, - bool validate_settings, - const std::string & setting_name_prefix, - const std::string & fallback_config_prefix, - const std::string & fallback_setting_name_prefix); void finishInit(const DB::Settings & settings, bool validate_settings); void normalizeSettings(); }; diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp index 0323da53117..396eb8f5ec7 100644 --- a/src/IO/S3Settings.cpp +++ b/src/IO/S3Settings.cpp @@ -27,10 +27,12 @@ void S3SettingsByEndpoint::loadFromConfig( const auto endpoint_path = key_path + ".endpoint"; if (config.has(endpoint_path)) { - auto endpoint = config.getString(endpoint_path); - auto auth_settings = S3::AuthSettings(config, settings, config_prefix); - auto request_settings = S3::RequestSettings(config, settings, config_prefix, settings.s3_validate_request_settings); - s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); + auto auth_settings = S3::AuthSettings(config, settings, key_path); + auto request_settings = S3::RequestSettings(config, settings, key_path, "", settings.s3_validate_request_settings); + + s3_settings.emplace( + config.getString(endpoint_path), + S3Settings{std::move(auth_settings), std::move(request_settings)}); } } } diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 244d233b302..8720e6440da 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -110,9 +110,6 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, auto s3_settings = getSettings( config, "s3"/* config_prefix */, context, url.uri_str, settings.s3_validate_request_settings); - s3_settings->auth_settings.updateIfChanged(auth_settings); - s3_settings->request_settings.updateIfChanged(request_settings); - if (!headers_from_ast.empty()) { s3_settings->auth_settings.headers.insert( @@ -121,7 +118,13 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, } if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) + { s3_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); + s3_settings->request_settings.updateIfChanged(endpoint_settings->request_settings); + } + + s3_settings->auth_settings.updateIfChanged(auth_settings); + s3_settings->request_settings.updateIfChanged(request_settings); auto client = getClient(url, *s3_settings, context, /* for_disk_s3 */false); auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); From 45720ca086b39630924f7bbc30008ac0d7215239 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 31 May 2024 20:01:42 +0000 Subject: [PATCH 232/856] Comments --- src/Storages/MergeTree/MergeTreeData.cpp | 1 + src/Storages/StorageMergeTree.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 1 + 3 files changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f32ba3ee0bc..d141b856c30 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7052,6 +7052,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( const StorageSnapshotPtr &, SelectQueryInfo &) const { + /// with new analyzer, Planner make decision regarding parallel replicas usage, and so about processing stage on reading if (!query_context->getSettingsRef().allow_experimental_analyzer) { if (query_context->getClientInfo().collaborate_with_initiator) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 092c686f682..34ff332f12c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -214,6 +214,7 @@ void StorageMergeTree::read( size_t num_streams) { const auto & settings = local_context->getSettingsRef(); + /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here if (local_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_for_non_replicated_merge_tree && !settings.allow_experimental_analyzer) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c8e22106657..dd4b38cdad3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5459,6 +5459,7 @@ void StorageReplicatedMergeTree::read( /// For this you have to synchronously go to ZooKeeper. if (settings.select_sequential_consistency) readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); + /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here else if (local_context->canUseParallelReplicasOnInitiator() && !settings.allow_experimental_analyzer) readParallelReplicasImpl(query_plan, column_names, query_info, local_context, processed_stage); else From 5246c56a2aae742d498028a7d5d2b7f9aa124baf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 30 May 2024 16:26:49 +0200 Subject: [PATCH 233/856] Fix type inference for float (in case of small buffer) In case of small buffer (i.e. --max_read_buffer_size 1) the pos() will be always point to this one byte, so, comparing pos() will be always evaluated to true. And we cannot use count() as well, since in case of big buffer it will be the same, plus, in case of reading extra byte for checking for '.' the count() will be different, but it does not mean that the byte had been interpreted (and allowing 1 byte of difference will not work almost always, since it will read max_read_buffer_size bytes). So instead, expose the has_fractional flag from the read helpers for float, via two new methods: - tryReadFloatTextExt - tryReadFloatTextExtNoExponent Where "ext" stands for "extended", which means expose extra information. v2: consider number as float if it has '.' or 'e' (previously only if it has some signs after those two it had been considered as float) Signed-off-by: Azat Khuzhin --- src/Formats/SchemaInferenceUtils.cpp | 30 +++++++------- src/IO/readFloatText.h | 39 +++++++++++++++++-- ...oat_schema_inference_small_block.reference | 15 +++++++ ...3170_float_schema_inference_small_block.sh | 32 +++++++++++++++ 4 files changed, 95 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/03170_float_schema_inference_small_block.reference create mode 100755 tests/queries/0_stateless/03170_float_schema_inference_small_block.sh diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 02c0aa6dd77..80a467a1145 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -879,11 +879,11 @@ namespace } template - bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings) + bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings, bool & has_fractional) { if (is_json || settings.try_infer_exponent_floats) - return tryReadFloatText(value, buf); - return tryReadFloatTextNoExponent(value, buf); + return tryReadFloatTextExt(value, buf, has_fractional); + return tryReadFloatTextExtNoExponent(value, buf, has_fractional); } template @@ -893,6 +893,7 @@ namespace return nullptr; Float64 tmp_float; + bool has_fractional; if (settings.try_infer_integers) { /// If we read from String, we can do it in a more efficient way. @@ -906,12 +907,10 @@ namespace if (buf.eof()) return read_int ? std::make_shared() : nullptr; - char * int_end = buf.position(); /// We can safely get back to the start of the number, because we read from a string and we didn't reach eof. buf.position() = number_start; bool read_uint = false; - char * uint_end = nullptr; /// In case of Int64 overflow we can try to infer UInt64. if (!read_int) { @@ -921,15 +920,14 @@ namespace if (buf.eof()) return read_uint ? std::make_shared() : nullptr; - uint_end = buf.position(); buf.position() = number_start; } - if (tryReadFloat(tmp_float, buf, settings)) + if (tryReadFloat(tmp_float, buf, settings, has_fractional)) { - if (read_int && buf.position() == int_end) + if (read_int && !has_fractional) return std::make_shared(); - if (read_uint && buf.position() == uint_end) + if (read_uint && !has_fractional) return std::make_shared(); return std::make_shared(); } @@ -944,34 +942,31 @@ namespace PeekableReadBufferCheckpoint checkpoint(peekable_buf); Int64 tmp_int; bool read_int = tryReadIntText(tmp_int, peekable_buf); - auto * int_end = peekable_buf.position(); peekable_buf.rollbackToCheckpoint(true); bool read_uint = false; - char * uint_end = nullptr; /// In case of Int64 overflow we can try to infer UInt64. if (!read_int) { PeekableReadBufferCheckpoint new_checkpoint(peekable_buf); UInt64 tmp_uint; read_uint = tryReadIntText(tmp_uint, peekable_buf); - uint_end = peekable_buf.position(); peekable_buf.rollbackToCheckpoint(true); } - if (tryReadFloat(tmp_float, peekable_buf, settings)) + if (tryReadFloat(tmp_float, peekable_buf, settings, has_fractional)) { /// Float parsing reads no fewer bytes than integer parsing, /// so position of the buffer is either the same, or further. /// If it's the same, then it's integer. - if (read_int && peekable_buf.position() == int_end) + if (read_int && !has_fractional) return std::make_shared(); - if (read_uint && peekable_buf.position() == uint_end) + if (read_uint && !has_fractional) return std::make_shared(); return std::make_shared(); } } - else if (tryReadFloat(tmp_float, buf, settings)) + else if (tryReadFloat(tmp_float, buf, settings, has_fractional)) { return std::make_shared(); } @@ -1004,7 +999,8 @@ namespace buf.position() = buf.buffer().begin(); Float64 tmp; - if (tryReadFloat(tmp, buf, settings) && buf.eof()) + bool has_fractional; + if (tryReadFloat(tmp, buf, settings, has_fractional) && buf.eof()) return std::make_shared(); return nullptr; diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index 3a21d7201a9..215bb1a3270 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -320,11 +320,13 @@ static inline void readUIntTextUpToNSignificantDigits(T & x, ReadBuffer & buf) template -ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) +ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in, bool & has_fractional) { static_assert(std::is_same_v || std::is_same_v, "Argument for readFloatTextImpl must be float or double"); static_assert('a' > '.' && 'A' > '.' && '\n' < '.' && '\t' < '.' && '\'' < '.' && '"' < '.', "Layout of char is not like ASCII"); + has_fractional = false; + static constexpr bool throw_exception = std::is_same_v; bool negative = false; @@ -377,6 +379,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) if (checkChar('.', in)) { + has_fractional = true; auto after_point_count = in.count(); while (!in.eof() && *in.position() == '0') @@ -394,6 +397,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) { if (checkChar('e', in) || checkChar('E', in)) { + has_fractional = true; if (in.eof()) { if constexpr (throw_exception) @@ -420,10 +424,14 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) } if (after_point) + { x += static_cast(shift10(after_point, after_point_exponent)); + } if (exponent) + { x = static_cast(shift10(x, exponent)); + } if (negative) x = -x; @@ -590,8 +598,16 @@ ReturnType readFloatTextSimpleImpl(T & x, ReadBuffer & buf) template void readFloatTextPrecise(T & x, ReadBuffer & in) { readFloatTextPreciseImpl(x, in); } template bool tryReadFloatTextPrecise(T & x, ReadBuffer & in) { return readFloatTextPreciseImpl(x, in); } -template void readFloatTextFast(T & x, ReadBuffer & in) { readFloatTextFastImpl(x, in); } -template bool tryReadFloatTextFast(T & x, ReadBuffer & in) { return readFloatTextFastImpl(x, in); } +template void readFloatTextFast(T & x, ReadBuffer & in) +{ + bool has_fractional; + readFloatTextFastImpl(x, in, has_fractional); +} +template bool tryReadFloatTextFast(T & x, ReadBuffer & in) +{ + bool has_fractional; + return readFloatTextFastImpl(x, in, has_fractional); +} template void readFloatTextSimple(T & x, ReadBuffer & in) { readFloatTextSimpleImpl(x, in); } template bool tryReadFloatTextSimple(T & x, ReadBuffer & in) { return readFloatTextSimpleImpl(x, in); } @@ -603,6 +619,21 @@ template void readFloatText(T & x, ReadBuffer & in) { readFloatText template bool tryReadFloatText(T & x, ReadBuffer & in) { return tryReadFloatTextFast(x, in); } /// Don't read exponent part of the number. -template bool tryReadFloatTextNoExponent(T & x, ReadBuffer & in) { return readFloatTextFastImpl(x, in); } +template bool tryReadFloatTextNoExponent(T & x, ReadBuffer & in) +{ + bool has_fractional; + return readFloatTextFastImpl(x, in, has_fractional); +} + +/// With a @has_fractional flag +/// Used for input_format_try_infer_integers +template bool tryReadFloatTextExt(T & x, ReadBuffer & in, bool & has_fractional) +{ + return readFloatTextFastImpl(x, in, has_fractional); +} +template bool tryReadFloatTextExtNoExponent(T & x, ReadBuffer & in, bool & has_fractional) +{ + return readFloatTextFastImpl(x, in, has_fractional); +} } diff --git a/tests/queries/0_stateless/03170_float_schema_inference_small_block.reference b/tests/queries/0_stateless/03170_float_schema_inference_small_block.reference new file mode 100644 index 00000000000..9ee16da8728 --- /dev/null +++ b/tests/queries/0_stateless/03170_float_schema_inference_small_block.reference @@ -0,0 +1,15 @@ +Int64 +x Nullable(Int64) +x Nullable(Int64) +x Nullable(Int64) +Float64 +x Nullable(Float64) +x Nullable(Float64) +x Nullable(Float64) +x Nullable(Float64) +Float64.explicit File +x Nullable(Float64) +Float64.pipe +x Nullable(Float64) +Float64.default max_read_buffer_size +x Nullable(Float64) diff --git a/tests/queries/0_stateless/03170_float_schema_inference_small_block.sh b/tests/queries/0_stateless/03170_float_schema_inference_small_block.sh new file mode 100755 index 00000000000..88f9bfad7ed --- /dev/null +++ b/tests/queries/0_stateless/03170_float_schema_inference_small_block.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# do not fallback to float always +echo "Int64" +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : 1}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : +1}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : -1}' + +echo "Float64" +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : 1.1}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : +1.1}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : 1.111}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : +1.111}' + +# this is requried due to previously clickhouse-local does not interprets +# --max_read_buffer_size for fds [1] +# +# [1]: https://github.com/ClickHouse/ClickHouse/pull/64532 +echo "Float64.explicit File" +tmp_path=$(mktemp "$CUR_DIR/03170_float_schema_inference_small_block.json.XXXXXX") +trap 'rm -f $tmp_path' EXIT +cat > "$tmp_path" <<<'{"x" : 1.111}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' --file "$tmp_path" + +echo "Float64.pipe" +echo '{"x" : 1.1}' | $CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' +echo "Float64.default max_read_buffer_size" +echo '{"x" : 1.1}' | $CLICKHOUSE_LOCAL --storage_file_read_method read --input-format JSONEachRow 'desc "table"' From 5df91804f6559095e115ec417aeeeab368b2975f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Jun 2024 11:51:35 +0200 Subject: [PATCH 234/856] Add probability setting for GWPAsan --- programs/server/Server.cpp | 8 ++++++++ src/Common/Allocator.cpp | 14 +++++++------- src/Common/GWPAsan.cpp | 14 ++++++++++++-- src/Common/GWPAsan.h | 18 ++++++++++++++++-- src/Common/PODArray.h | 22 +++++++++++++--------- src/Common/memory.h | 26 +++++++++++++------------- src/Core/ServerSettings.h | 1 + src/Daemon/BaseDaemon.cpp | 4 ++-- 8 files changed, 72 insertions(+), 35 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..1822c016f4d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1632,6 +1632,10 @@ try if (global_context->isServerCompletelyStarted()) CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings.cannot_allocate_thread_fault_injection_probability); +#if USE_GWP_ASAN + GWPAsan::setForceSampleProbability(new_server_settings.gwp_asan_force_sample_probability); +#endif + ProfileEvents::increment(ProfileEvents::MainConfigLoads); /// Must be the last. @@ -2124,6 +2128,10 @@ try CannotAllocateThreadFaultInjector::setFaultProbability(server_settings.cannot_allocate_thread_fault_injection_probability); +#if USE_GWP_ASAN + GWPAsan::setForceSampleProbability(server_settings.gwp_asan_force_sample_probability); +#endif + try { global_context->startClusterDiscovery(); diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 87075a8c709..bfc85559fe8 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -68,9 +68,9 @@ void * allocNoTrack(size_t size, size_t alignment) { void * buf; #if USE_GWP_ASAN - if (unlikely(Memory::GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) { - if (void * ptr = Memory::GuardedAlloc.allocate(size, alignment)) + if (void * ptr = GWPAsan::GuardedAlloc.allocate(size, alignment)) { if constexpr (clear_memory) memset(ptr, 0, size); @@ -120,10 +120,10 @@ void * allocNoTrack(size_t size, size_t alignment) void freeNoTrack(void * buf) { #if USE_GWP_ASAN - if (unlikely(Memory::GuardedAlloc.pointerIsMine(buf))) + if (unlikely(GWPAsan::GuardedAlloc.pointerIsMine(buf))) { ProfileEvents::increment(ProfileEvents::GWPAsanFree); - Memory::GuardedAlloc.deallocate(buf); + GWPAsan::GuardedAlloc.deallocate(buf); return; } #endif @@ -185,9 +185,9 @@ void * Allocator::realloc(void * buf, size_t old_size, } #if USE_GWP_ASAN - if (unlikely(Memory::GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) { - if (void * ptr = Memory::GuardedAlloc.allocate(new_size, alignment)) + if (void * ptr = GWPAsan::GuardedAlloc.allocate(new_size, alignment)) { auto trace_free = CurrentMemoryTracker::free(old_size); auto trace_alloc = CurrentMemoryTracker::alloc(new_size); @@ -213,7 +213,7 @@ void * Allocator::realloc(void * buf, size_t old_size, } } - if (unlikely(Memory::GuardedAlloc.pointerIsMine(buf))) + if (unlikely(GWPAsan::GuardedAlloc.pointerIsMine(buf))) { /// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods. void * new_buf = alloc(new_size, alignment); diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index ecff097e365..4bda2f7e913 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -12,9 +12,10 @@ # include # include +# include # include -namespace Memory +namespace GWPAsan { namespace @@ -53,6 +54,7 @@ static bool guarded_alloc_initialized = [] auto & opts = gwp_asan::options::getOptions(); opts.Backtrace = getBackTrace; + opts.MaxSimultaneousAllocations = 256; /// for testing GuardedAlloc.init(opts); ///std::cerr << "GwpAsan is initialized, the options are { Enabled: " << opts.Enabled @@ -155,7 +157,7 @@ void printHeader(gwp_asan::Error error, uintptr_t fault_address, const gwp_asan: } -void printGWPAsanReport([[maybe_unused]] uintptr_t fault_address) +void printReport([[maybe_unused]] uintptr_t fault_address) { const auto logger = getLogger("GWPAsan"); const auto * state = GuardedAlloc.getAllocatorState(); @@ -212,5 +214,13 @@ void printGWPAsanReport([[maybe_unused]] uintptr_t fault_address) reinterpret_cast(trace.data()), 0, trace_length, [&](const auto line) { LOG_FATAL(logger, fmt::runtime(line)); }); } +std::atomic force_sample_probability = 0.0; + +void setForceSampleProbability(double value) +{ + force_sample_probability.store(value, std::memory_order_relaxed); } + +} + #endif diff --git a/src/Common/GWPAsan.h b/src/Common/GWPAsan.h index 164c6ee0221..b3215c6157e 100644 --- a/src/Common/GWPAsan.h +++ b/src/Common/GWPAsan.h @@ -5,15 +5,29 @@ #if USE_GWP_ASAN #include +#include -namespace Memory +#include +#include + +namespace GWPAsan { extern gwp_asan::GuardedPoolAllocator GuardedAlloc; bool isGWPAsanError(uintptr_t fault_address); -void printGWPAsanReport(uintptr_t fault_address); +void printReport(uintptr_t fault_address); + +extern std::atomic force_sample_probability; + +void setForceSampleProbability(double value); + +inline bool shouldForceSample() +{ + std::bernoulli_distribution dist(force_sample_probability.load(std::memory_order_relaxed)); + return dist(thread_local_rng); +} } diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index b9246bcdca2..d004e703ac6 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -2,17 +2,19 @@ #include "config.h" -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include + +#include +#include +#include #include #include -#include -#include -#include #ifndef NDEBUG #include @@ -119,7 +121,8 @@ protected: void alloc(size_t bytes, TAllocatorParams &&... allocator_params) { #if USE_GWP_ASAN - gwp_asan::getThreadLocals()->NextSampleCounter = 1; + if (unlikely(GWPAsan::shouldForceSample())) + gwp_asan::getThreadLocals()->NextSampleCounter = 1; #endif char * allocated = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)); @@ -152,7 +155,8 @@ protected: } #if USE_GWP_ASAN - gwp_asan::getThreadLocals()->NextSampleCounter = 1; + if (unlikely(GWPAsan::shouldForceSample())) + gwp_asan::getThreadLocals()->NextSampleCounter = 1; #endif unprotect(); diff --git a/src/Common/memory.h b/src/Common/memory.h index 633994a83e2..73b86c5d3ca 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -37,11 +37,11 @@ requires DB::OptionalArgument inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) { #if USE_GWP_ASAN - if (unlikely(GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) { if constexpr (sizeof...(TAlign) == 1) { - if (void * ptr = GuardedAlloc.allocate(size, alignToSizeT(align...))) + if (void * ptr = GWPAsan::GuardedAlloc.allocate(size, alignToSizeT(align...))) { ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); return ptr; @@ -53,7 +53,7 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) } else { - if (void * ptr = GuardedAlloc.allocate(size)) + if (void * ptr = GWPAsan::GuardedAlloc.allocate(size)) { ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); return ptr; @@ -83,9 +83,9 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept { #if USE_GWP_ASAN - if (unlikely(GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) { - if (void * ptr = GuardedAlloc.allocate(size)) + if (void * ptr = GWPAsan::GuardedAlloc.allocate(size)) { ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); return ptr; @@ -102,9 +102,9 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) noexcept { #if USE_GWP_ASAN - if (unlikely(GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) { - if (void * ptr = GuardedAlloc.allocate(size, alignToSizeT(align))) + if (void * ptr = GWPAsan::GuardedAlloc.allocate(size, alignToSizeT(align))) { ProfileEvents::increment(ProfileEvents::GWPAsanAllocateSuccess); return ptr; @@ -121,10 +121,10 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept { #if USE_GWP_ASAN - if (unlikely(GuardedAlloc.pointerIsMine(ptr))) + if (unlikely(GWPAsan::GuardedAlloc.pointerIsMine(ptr))) { ProfileEvents::increment(ProfileEvents::GWPAsanFree); - GuardedAlloc.deallocate(ptr); + GWPAsan::GuardedAlloc.deallocate(ptr); return; } #endif @@ -141,10 +141,10 @@ inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size, TAlign... al return; #if USE_GWP_ASAN - if (unlikely(GuardedAlloc.pointerIsMine(ptr))) + if (unlikely(GWPAsan::GuardedAlloc.pointerIsMine(ptr))) { ProfileEvents::increment(ProfileEvents::GWPAsanFree); - GuardedAlloc.deallocate(ptr); + GWPAsan::GuardedAlloc.deallocate(ptr); return; } #endif @@ -217,10 +217,10 @@ inline ALWAYS_INLINE size_t untrackMemory(void * ptr [[maybe_unused]], Allocatio std::size_t actual_size = 0; #if USE_GWP_ASAN - if (unlikely(GuardedAlloc.pointerIsMine(ptr))) + if (unlikely(GWPAsan::GuardedAlloc.pointerIsMine(ptr))) { if (!size) - size = GuardedAlloc.getSize(ptr); + size = GWPAsan::GuardedAlloc.getSize(ptr); trace = CurrentMemoryTracker::free(size); return size; } diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 45f235116ab..55d8a8f0ec7 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -146,6 +146,7 @@ namespace DB M(UInt64, global_profiler_real_time_period_ns, 0, "Period for real clock timer of global profiler (in nanoseconds). Set 0 value to turn off the real clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, global_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of global profiler (in nanoseconds). Set 0 value to turn off the CPU clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \ + M(Double, gwp_asan_force_sample_probability, 0.1, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 3f7ad8d7126..dc8cdf7deff 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -159,8 +159,8 @@ static void signalHandler(int sig, siginfo_t * info, void * context) #if USE_GWP_ASAN if (const auto fault_address = reinterpret_cast(info->si_addr); - ::Memory::isGWPAsanError(fault_address)) - ::Memory::printGWPAsanReport(fault_address); + GWPAsan::isGWPAsanError(fault_address)) + GWPAsan::printReport(fault_address); #endif writeBinary(sig, out); From a035e4ed698230117303114a12dd67de6929470c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 Jun 2024 11:52:37 +0200 Subject: [PATCH 235/856] Fix tests --- .../ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/IO/S3Common.cpp | 25 ++++++++++++++++++- src/IO/S3Common.h | 7 ++++-- .../ObjectStorage/S3/Configuration.cpp | 14 +++++------ 4 files changed, 37 insertions(+), 11 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 56cf7a1b9e6..fab3fa357cc 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -597,7 +597,7 @@ void S3ObjectStorage::applyNewSettings( if (options.allow_client_change && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3)) { - auto new_client = getClient(uri, *settings_from_config, context, for_disk_s3); + auto new_client = getClient(uri, *modified_settings, context, for_disk_s3); client.set(std::move(new_client)); } s3_settings.set(std::move(modified_settings)); diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 54881227d13..490bf8c2d0c 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -192,7 +192,10 @@ void AuthSettings::updateIfChanged(const AuthSettings & settings) if (!settings.users.empty()) users.insert(settings.users.begin(), settings.users.end()); - server_side_encryption_kms_config = settings.server_side_encryption_kms_config; + if (settings.server_side_encryption_kms_config.key_id.has_value() + || settings.server_side_encryption_kms_config.encryption_context.has_value() + || settings.server_side_encryption_kms_config.key_id.has_value()) + server_side_encryption_kms_config = settings.server_side_encryption_kms_config; } RequestSettings::RequestSettings( @@ -390,6 +393,26 @@ void RequestSettings::validateUploadSettings() /// We can check that max possible object size is not too small. } +bool operator==(const AuthSettings & left, const AuthSettings & right) +{ + if (left.headers != right.headers) + return false; + + if (left.users != right.users) + return false; + + if (left.server_side_encryption_kms_config != right.server_side_encryption_kms_config) + return false; + + auto l = left.begin(); + for (const auto & r : right) + { + if ((l == left.end()) || (*l != r)) + return false; + ++l; + } + return l == left.end(); +} } IMPLEMENT_SETTINGS_TRAITS(S3::AuthSettingsTraits, CLIENT_SETTINGS_LIST) diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 68d44b0ed01..2dca08871d3 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -146,9 +146,9 @@ struct AuthSettings : public BaseSettings const DB::Settings & settings, const std::string & config_prefix); - AuthSettings(const DB::Settings & settings); + explicit AuthSettings(const DB::Settings & settings); - AuthSettings(const DB::NamedCollection & collection); + explicit AuthSettings(const DB::NamedCollection & collection); void updateFromSettings(const DB::Settings & settings, bool if_changed); bool hasUpdates(const AuthSettings & other) const; @@ -158,8 +158,11 @@ struct AuthSettings : public BaseSettings HTTPHeaderEntries headers; std::unordered_set users; ServerSideEncryptionKMSConfig server_side_encryption_kms_config; + /// Note: if you add any field, do not forget to update operator ==. }; +bool operator==(const AuthSettings & left, const AuthSettings & right); + struct RequestSettings : public BaseSettings { RequestSettings() = default; diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 8720e6440da..b33d55105e9 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -110,13 +110,6 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, auto s3_settings = getSettings( config, "s3"/* config_prefix */, context, url.uri_str, settings.s3_validate_request_settings); - if (!headers_from_ast.empty()) - { - s3_settings->auth_settings.headers.insert( - s3_settings->auth_settings.headers.end(), - headers_from_ast.begin(), headers_from_ast.end()); - } - if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) { s3_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); @@ -126,6 +119,13 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, s3_settings->auth_settings.updateIfChanged(auth_settings); s3_settings->request_settings.updateIfChanged(request_settings); + if (!headers_from_ast.empty()) + { + s3_settings->auth_settings.headers.insert( + s3_settings->auth_settings.headers.end(), + headers_from_ast.begin(), headers_from_ast.end()); + } + auto client = getClient(url, *s3_settings, context, /* for_disk_s3 */false); auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); auto s3_capabilities = S3Capabilities From 918d3849e18e4eb8933fa869e2cf6c534424e6d5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 31 May 2024 15:49:03 +0200 Subject: [PATCH 236/856] Simplify logic for input_format_try_infer_integers Now, when we can be sure that it is a float, parse it as a float, and fallback to int/uint after. But note, that this would break something if tryReadFloat() != tryReadIntText() + parsing of '.'/'e', but for now, it is true. Signed-off-by: Azat Khuzhin --- src/Formats/SchemaInferenceUtils.cpp | 78 ++++++++++------------------ 1 file changed, 27 insertions(+), 51 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 80a467a1145..1dc612728f5 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -900,37 +900,24 @@ namespace if (auto * string_buf = dynamic_cast(&buf)) { /// Remember the pointer to the start of the number to rollback to it. - char * number_start = buf.position(); - Int64 tmp_int; - bool read_int = tryReadIntText(tmp_int, buf); - /// If we reached eof, it cannot be float (it requires no less data than integer) - if (buf.eof()) - return read_int ? std::make_shared() : nullptr; - /// We can safely get back to the start of the number, because we read from a string and we didn't reach eof. - buf.position() = number_start; + char * number_start = buf.position(); - bool read_uint = false; - /// In case of Int64 overflow we can try to infer UInt64. - if (!read_int) - { - UInt64 tmp_uint; - read_uint = tryReadIntText(tmp_uint, buf); - /// If we reached eof, it cannot be float (it requires no less data than integer) - if (buf.eof()) - return read_uint ? std::make_shared() : nullptr; - - buf.position() = number_start; - } - - if (tryReadFloat(tmp_float, buf, settings, has_fractional)) - { - if (read_int && !has_fractional) - return std::make_shared(); - if (read_uint && !has_fractional) - return std::make_shared(); + /// NOTE: it may break parsing of tryReadFloat() != tryReadIntText() + parsing of '.'/'e' + /// But, for now it is true + if (tryReadFloat(tmp_float, buf, settings, has_fractional) && has_fractional) return std::make_shared(); - } + + Int64 tmp_int; + buf.position() = number_start; + if (tryReadIntText(tmp_int, buf)) + return std::make_shared(); + + /// In case of Int64 overflow we can try to infer UInt64. + UInt64 tmp_uint; + buf.position() = number_start; + if (tryReadIntText(tmp_uint, buf)) + return std::make_shared(); return nullptr; } @@ -940,31 +927,20 @@ namespace /// and then as float. PeekableReadBuffer peekable_buf(buf); PeekableReadBufferCheckpoint checkpoint(peekable_buf); - Int64 tmp_int; - bool read_int = tryReadIntText(tmp_int, peekable_buf); - peekable_buf.rollbackToCheckpoint(true); - bool read_uint = false; - /// In case of Int64 overflow we can try to infer UInt64. - if (!read_int) - { - PeekableReadBufferCheckpoint new_checkpoint(peekable_buf); - UInt64 tmp_uint; - read_uint = tryReadIntText(tmp_uint, peekable_buf); - peekable_buf.rollbackToCheckpoint(true); - } - - if (tryReadFloat(tmp_float, peekable_buf, settings, has_fractional)) - { - /// Float parsing reads no fewer bytes than integer parsing, - /// so position of the buffer is either the same, or further. - /// If it's the same, then it's integer. - if (read_int && !has_fractional) - return std::make_shared(); - if (read_uint && !has_fractional) - return std::make_shared(); + if (tryReadFloat(tmp_float, peekable_buf, settings, has_fractional) && has_fractional) return std::make_shared(); - } + peekable_buf.rollbackToCheckpoint(/* drop= */ false); + + Int64 tmp_int; + if (tryReadIntText(tmp_int, peekable_buf)) + return std::make_shared(); + peekable_buf.rollbackToCheckpoint(/* drop= */ true); + + /// In case of Int64 overflow we can try to infer UInt64. + UInt64 tmp_uint; + if (tryReadIntText(tmp_uint, peekable_buf)) + return std::make_shared(); } else if (tryReadFloat(tmp_float, buf, settings, has_fractional)) { From e1bf007bb97f12f379512d0d9a24235a0114f0b3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Jun 2024 14:18:15 +0200 Subject: [PATCH 237/856] Small fix --- src/Common/PODArray.h | 5 ----- src/Common/memory.h | 4 ++-- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index d004e703ac6..92ef0597c7e 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -20,11 +20,6 @@ #include #endif -#if USE_GWP_ASAN -# include - -#endif - /** Whether we can use memcpy instead of a loop with assignment to T from U. * It is Ok if types are the same. And if types are integral and of the same size, * example: char, signed char, unsigned char. diff --git a/src/Common/memory.h b/src/Common/memory.h index 73b86c5d3ca..caa0418fa56 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -162,10 +162,10 @@ requires DB::OptionalArgument inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unused]], TAlign... /* align */) noexcept { #if USE_GWP_ASAN - if (unlikely(GuardedAlloc.pointerIsMine(ptr))) + if (unlikely(GWPAsan::GuardedAlloc.pointerIsMine(ptr))) { ProfileEvents::increment(ProfileEvents::GWPAsanFree); - GuardedAlloc.deallocate(ptr); + GWPAsan::GuardedAlloc.deallocate(ptr); return; } #endif From 79442f226532a31cd8546af7b5fb36263c1a8f7c Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Mon, 3 Jun 2024 13:03:29 +0000 Subject: [PATCH 238/856] 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 239/856] 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 240/856] 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 f716b131d77bd23545d7748e30bddaff49b2bb39 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 3 Jun 2024 14:23:34 +0000 Subject: [PATCH 241/856] fix tests --- src/Storages/MergeTree/MergeTask.cpp | 6 +++--- src/Storages/MergeTree/MergeTask.h | 2 +- .../0_stateless/03166_skip_indexes_vertical_merge_1.sql | 3 +++ 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 2660602d652..0fcb3b0b0e7 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -95,7 +95,7 @@ static void addMissedColumnsToSerializationInfos( } /// PK columns are sorted and merged, ordinary columns are gathered using info from merge step -void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColumns() +void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColumns() const { const auto & sorting_key_expr = global_ctx->metadata_snapshot->getSortingKey().expression; Names sort_key_columns_vec = sorting_key_expr->getRequiredColumns(); @@ -233,13 +233,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->temporary_directory_lock = global_ctx->data->getTemporaryPartDirectoryHolder(local_tmp_part_basename); global_ctx->storage_columns = global_ctx->metadata_snapshot->getColumns().getAllPhysical(); - extractMergingAndGatheringColumns(); auto object_columns = MergeTreeData::getConcreteObjectColumns(global_ctx->future_part->parts, global_ctx->metadata_snapshot->getColumns()); - extendObjectColumns(global_ctx->storage_columns, object_columns, false); global_ctx->storage_snapshot = std::make_shared(*global_ctx->data, global_ctx->metadata_snapshot, std::move(object_columns)); + extractMergingAndGatheringColumns(); + global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; global_ctx->new_data_part->partition.assign(global_ctx->future_part->getPartition()); global_ctx->new_data_part->is_temp = global_ctx->parent_part == nullptr; diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 13de2b9da11..0e13d3aef62 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -259,7 +259,7 @@ private: MergeAlgorithm chooseMergeAlgorithm() const; void createMergedStream(); - void extractMergingAndGatheringColumns(); + void extractMergingAndGatheringColumns() const; void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) override { diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql index a605f9dea81..ba770656532 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql @@ -1,9 +1,12 @@ DROP TABLE IF EXISTS t_ind_merge_1; +SET allow_experimental_analyzer = 1; + CREATE TABLE t_ind_merge_1 (a UInt64, b UInt64, c UInt64, d UInt64, INDEX idx_b b TYPE minmax) ENGINE = MergeTree ORDER BY a SETTINGS index_granularity = 64, + merge_max_block_size = 8192, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1, min_bytes_for_wide_part = 0; From 6a8adb6d487db7789f2c2f4f72103cb5e14b2281 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 3 Jun 2024 16:34:53 +0200 Subject: [PATCH 242/856] Refactor change and add failing test case --- src/Common/StringUtils.h | 12 ++++++++++++ src/Common/UTF8Helpers.cpp | 18 +++++++++++++++--- .../03142_skip_ANSI_in_UTF8_compute_width.sql | 7 ++++++- 3 files changed, 33 insertions(+), 4 deletions(-) diff --git a/src/Common/StringUtils.h b/src/Common/StringUtils.h index fe5fc3c058f..e4c7ab3e80c 100644 --- a/src/Common/StringUtils.h +++ b/src/Common/StringUtils.h @@ -140,6 +140,18 @@ inline bool isPrintableASCII(char c) return uc >= 32 && uc <= 126; /// 127 is ASCII DEL. } +inline bool isCSIParameterByte(char c) +{ + uint8_t uc = c; + return uc >= 0x30 && uc <= 0x3F; /// ASCII 0–9:;<=>? +} + +inline bool isCSIIntermediateByte(char c) +{ + uint8_t uc = c; + return uc >= 0x20 && uc <= 0x2F; /// ASCII !"#$%&'()*+,-./ +} + inline bool isCSIFinalByte(char c) { uint8_t uc = c; diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 8c8c8e8327b..34eba832113 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -147,10 +147,22 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l while (i < size && isPrintableASCII(data[i])) { - if (!isEscapeSequence) + auto isParameterByte = isCSIParameterByte(data[i]); + auto isIntermediateByte = isCSIIntermediateByte(data[i]); + auto ignore_width = isEscapeSequence & (isParameterByte || isIntermediateByte); + + if (ignore_width || (data[i] == '[' && isEscapeSequence)) + { + /// don't count the width + } + else if (isEscapeSequence && isCSIFinalByte(data[i])) + { + isEscapeSequence = false; + } + else + { ++width; - else if (isCSIFinalByte(data[i]) && data[i - 1] != '\x1b') - isEscapeSequence = false; /// end of CSI escape sequence reached + } ++i; } diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql index e37b0db08e9..f4b0bfe5888 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql @@ -1 +1,6 @@ -SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 0) AS x FORMAT Pretty; +SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x; +SELECT format('\x1b[38;2;{0};{1};{2}m█ test \x1b[0m', 255, 128, 128) AS x; +SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test', 255, 128, 128) AS x; +SELECT format('test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x; +SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x; +SELECT visibleWidth('0};{1};{2}m█'); \ No newline at end of file From e449f83cfaa548cec8b17ddfec1123e0a7b27c84 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Jun 2024 15:28:32 +0000 Subject: [PATCH 243/856] 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 642e2fdd7211b8eaa2eb8012050cb2d84b73755b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 Jun 2024 18:17:13 +0200 Subject: [PATCH 244/856] Fix clang-tidy --- src/IO/S3/copyS3File.cpp | 2 +- src/IO/WriteBufferFromS3.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 22572b5fc66..c48d7965ac2 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -185,7 +185,7 @@ namespace request.SetMultipartUpload(multipart_upload); - size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); + size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 7e3ba48c165..ac63281d328 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -610,7 +610,7 @@ void WriteBufferFromS3::completeMultipartUpload() req.SetMultipartUpload(multipart_upload); - size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); + size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); From 7d0e35bf5fb8ba0a03f34773333877b01462f302 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Jun 2024 16:53:50 +0000 Subject: [PATCH 245/856] 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 d2184fd2a2d76de2de2252b9152c08f7ae269574 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 3 Jun 2024 20:51:09 +0200 Subject: [PATCH 246/856] Update test.py --- tests/integration/test_replicated_database/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index ef86da5af30..a5859960cd9 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -46,7 +46,7 @@ snapshotting_node = cluster.add_instance( ) snapshot_recovering_node = cluster.add_instance( "snapshot_recovering_node", - main_configs=["configs/config2.xml"], + main_configs=["configs/config.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, ) @@ -61,7 +61,7 @@ all_nodes = [ bad_settings_node = cluster.add_instance( "bad_settings_node", - main_configs=["configs/config.xml"], + main_configs=["configs/config2.xml"], user_configs=["configs/inconsistent_settings.xml"], with_zookeeper=True, macros={"shard": 1, "replica": 4}, From acfe2876b57aa4766e15df4a955991c19eb9dc8e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 3 Jun 2024 21:06:02 +0200 Subject: [PATCH 247/856] Fix OrderByLimitByDuplicateEliminationVisitor across subqueries --- ...OrderByLimitByDuplicateEliminationPass.cpp | 7 +--- .../03165_order_by_duplicate.reference | 39 +++++++++++++++++++ .../0_stateless/03165_order_by_duplicate.sql | 16 ++++++++ 3 files changed, 57 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03165_order_by_duplicate.reference create mode 100644 tests/queries/0_stateless/03165_order_by_duplicate.sql diff --git a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp index 26ca5984b49..15919c4a2fe 100644 --- a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp +++ b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp @@ -22,6 +22,7 @@ public: if (query_node->hasOrderBy()) { + QueryTreeNodeConstRawPtrWithHashSet unique_expressions_nodes_set; QueryTreeNodes result_nodes; auto & query_order_by_nodes = query_node->getOrderBy().getNodes(); @@ -45,10 +46,9 @@ public: query_order_by_nodes = std::move(result_nodes); } - unique_expressions_nodes_set.clear(); - if (query_node->hasLimitBy()) { + QueryTreeNodeConstRawPtrWithHashSet unique_expressions_nodes_set; QueryTreeNodes result_nodes; auto & query_limit_by_nodes = query_node->getLimitBy().getNodes(); @@ -63,9 +63,6 @@ public: query_limit_by_nodes = std::move(result_nodes); } } - -private: - QueryTreeNodeConstRawPtrWithHashSet unique_expressions_nodes_set; }; } diff --git a/tests/queries/0_stateless/03165_order_by_duplicate.reference b/tests/queries/0_stateless/03165_order_by_duplicate.reference new file mode 100644 index 00000000000..5d5e7a33f4a --- /dev/null +++ b/tests/queries/0_stateless/03165_order_by_duplicate.reference @@ -0,0 +1,39 @@ +QUERY id: 0 + PROJECTION COLUMNS + id UInt64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test, final: 1 + WHERE + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + QUERY id: 6, is_subquery: 1, is_distinct: 1 + PROJECTION COLUMNS + id UInt64 + PROJECTION + LIST id: 7, nodes: 1 + COLUMN id: 8, column_name: id, result_type: UInt64, source_id: 9 + JOIN TREE + TABLE id: 9, alias: __table1, table_name: default.test, final: 1 + ORDER BY + LIST id: 10, nodes: 1 + SORT id: 11, sort_direction: ASCENDING, with_fill: 0 + EXPRESSION + COLUMN id: 8, column_name: id, result_type: UInt64, source_id: 9 + LIMIT + CONSTANT id: 12, constant_value: UInt64_4, constant_value_type: UInt64 + ORDER BY + LIST id: 13, nodes: 1 + SORT id: 14, sort_direction: ASCENDING, with_fill: 0 + EXPRESSION + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + LIMIT BY LIMIT + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt64 + LIMIT BY + LIST id: 16, nodes: 1 + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/03165_order_by_duplicate.sql b/tests/queries/0_stateless/03165_order_by_duplicate.sql new file mode 100644 index 00000000000..0054cbc36a6 --- /dev/null +++ b/tests/queries/0_stateless/03165_order_by_duplicate.sql @@ -0,0 +1,16 @@ +CREATE TABLE test +ENGINE = ReplacingMergeTree +PRIMARY KEY id +AS SELECT number AS id FROM numbers(100); + +EXPLAIN QUERY TREE SELECT id +FROM test FINAL +WHERE id IN ( + SELECT DISTINCT id + FROM test FINAL + ORDER BY id ASC + LIMIT 4 +) +ORDER BY id ASC +LIMIT 1 BY id +SETTINGS allow_experimental_analyzer = 1; From ac480084a5faf6fa029bff492c09989ac9573388 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 3 Jun 2024 19:40:41 +0000 Subject: [PATCH 248/856] 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 63900550c9adba47034f0f136c45d4ef4d34076f Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 3 Jun 2024 19:16:42 -0400 Subject: [PATCH 249/856] Making use of LoadingStrictnessLevel --- src/Interpreters/InterpreterCreateQuery.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a3eaced0d02..5e07f7a27f2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1082,11 +1082,14 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String current_database = getContext()->getCurrentDatabase(); auto database_name = create.database ? create.getDatabase() : current_database; + bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery(); + auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup); + if (!create.sql_security && create.supportSQLSecurity() && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query) create.sql_security = std::make_shared(); if (create.sql_security) - processSQLSecurityOption(getContext(), create.sql_security->as(), create.is_materialized_view, /* skip_check_permissions= */ is_restore_from_backup || create.attach); + processSQLSecurityOption(getContext(), create.sql_security->as(), create.is_materialized_view, /* skip_check_permissions= */ mode >= LoadingStrictnessLevel::SECONDARY_CREATE); DDLGuardPtr ddl_guard; @@ -1213,9 +1216,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!UserDefinedSQLFunctionFactory::instance().empty()) UserDefinedSQLFunctionVisitor::visit(query_ptr); - bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery(); - auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup); - /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode); From 6c35a1b2e15e4161289ad1fc8870df1fca207447 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Tue, 4 Jun 2024 11:39:55 +0800 Subject: [PATCH 250/856] fix ubsan error --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index c2c709b9b52..be131e8f3a3 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -428,6 +428,8 @@ void ParquetBlockInputFormat::initializeIfNeeded() } if (!total_size || !format_settings.parquet.prefer_block_bytes) return 0; auto average_row_bytes = floor(static_cast(total_size) / row_group_meta->num_rows()); + // avoid inf preferred_num_rows; + if (average_row_bytes < 1) return 0; const size_t preferred_num_rows = static_cast(floor(format_settings.parquet.prefer_block_bytes/average_row_bytes)); const size_t MIN_ROW_NUM = 128; // size_t != UInt64 in darwin From 2d48600e439f5fd83bc194e62061005ec795c56d Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Tue, 4 Jun 2024 11:51:39 +0800 Subject: [PATCH 251/856] Update ParquetBlockInputFormat.cpp --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index be131e8f3a3..9f72e849ed0 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -429,7 +429,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() if (!total_size || !format_settings.parquet.prefer_block_bytes) return 0; auto average_row_bytes = floor(static_cast(total_size) / row_group_meta->num_rows()); // avoid inf preferred_num_rows; - if (average_row_bytes < 1) return 0; + if (average_row_bytes < 1) return 0; const size_t preferred_num_rows = static_cast(floor(format_settings.parquet.prefer_block_bytes/average_row_bytes)); const size_t MIN_ROW_NUM = 128; // size_t != UInt64 in darwin From e6f99266f60a46ec5ae56e55ecff2d753ac49919 Mon Sep 17 00:00:00 2001 From: Lee sungju Date: Tue, 4 Jun 2024 14:59:22 +0900 Subject: [PATCH 252/856] 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 2d72e194fbad9f1b16b5bde929fa9d006aa3ddf1 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 4 Jun 2024 07:59:15 +0000 Subject: [PATCH 253/856] remove bad optimization and re-enable vertical final Signed-off-by: Duc Canh Le --- src/Columns/FilterDescription.h | 7 - src/Core/Settings.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 4 - .../QueryPlan/ReadFromMergeTree.cpp | 8 +- src/Processors/Transforms/FilterTransform.cpp | 154 +----------------- src/Storages/SelectQueryInfo.h | 6 - ...ce => 02893_vertical_final_bugs.reference} | 1 + ...join.sql => 02893_vertical_final_bugs.sql} | 12 ++ 8 files changed, 19 insertions(+), 175 deletions(-) rename tests/queries/0_stateless/{02893_vertical_final_array_join.reference => 02893_vertical_final_bugs.reference} (99%) rename tests/queries/0_stateless/{02893_vertical_final_array_join.sql => 02893_vertical_final_bugs.sql} (65%) diff --git a/src/Columns/FilterDescription.h b/src/Columns/FilterDescription.h index 63457b8b544..b4335a49787 100644 --- a/src/Columns/FilterDescription.h +++ b/src/Columns/FilterDescription.h @@ -23,15 +23,10 @@ struct ConstantFilterDescription struct IFilterDescription { - /// has_one can be pre-compute during creating the filter description in some cases - Int64 has_one = -1; virtual ColumnPtr filter(const IColumn & column, ssize_t result_size_hint) const = 0; virtual size_t countBytesInFilter() const = 0; virtual ~IFilterDescription() = default; - bool hasOne() { return has_one >= 0 ? has_one : hasOneImpl();} protected: - /// Calculate if filter has a non-zero from the filter values, may update has_one - virtual bool hasOneImpl() = 0; }; /// Obtain a filter from non constant Column, that may have type: UInt8, Nullable(UInt8). @@ -45,7 +40,6 @@ struct FilterDescription final : public IFilterDescription ColumnPtr filter(const IColumn & column, ssize_t result_size_hint) const override { return column.filter(*data, result_size_hint); } size_t countBytesInFilter() const override { return DB::countBytesInFilter(*data); } protected: - bool hasOneImpl() override { return data ? (has_one = !memoryIsZero(data->data(), 0, data->size())) : false; } }; struct SparseFilterDescription final : public IFilterDescription @@ -56,7 +50,6 @@ struct SparseFilterDescription final : public IFilterDescription ColumnPtr filter(const IColumn & column, ssize_t) const override { return column.index(*filter_indices, 0); } size_t countBytesInFilter() const override { return filter_indices->size(); } protected: - bool hasOneImpl() override { return filter_indices && !filter_indices->empty(); } }; struct ColumnWithTypeAndName; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b8f5a8b5a75..ec5ee7b8fb4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -396,7 +396,7 @@ class IColumn; M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", 0) \ M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ - M(Bool, enable_vertical_final, false, "Not recommended. If enable, remove duplicated rows during FINAL by marking rows as deleted and filtering them later instead of merging rows", 0) \ + M(Bool, enable_vertical_final, true, "If enable, remove duplicated rows during FINAL by marking rows as deleted and filtering them later instead of merging rows", 0) \ \ \ /** Limits during query execution are part of the settings. \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ffe45d55643..4afd3f26b31 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2599,10 +2599,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc query_info.storage_limits = std::make_shared(storage_limits); query_info.settings_limit_offset_done = options.settings_limit_offset_done; - /// 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); storage->read(query_plan, required_columns, storage_snapshot, query_info, context, processing_stage, max_block_size, max_streams); if (context->hasQueryContext() && !options.is_internal) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 3988ba33d90..1a3f97f2dfb 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1092,8 +1092,7 @@ static void addMergingFinal( MergeTreeData::MergingParams merging_params, Names partition_key_columns, size_t max_block_size_rows, - bool enable_vertical_final, - bool can_merge_final_indices_to_next_step_filter) + bool enable_vertical_final) { const auto & header = pipe.getHeader(); size_t num_outputs = pipe.numOutputPorts(); @@ -1135,7 +1134,7 @@ static void addMergingFinal( }; pipe.addTransform(get_merging_processor()); - if (enable_vertical_final && !can_merge_final_indices_to_next_step_filter) + if (enable_vertical_final) pipe.addSimpleTransform([](const Block & header_) { return std::make_shared(header_); }); } @@ -1323,8 +1322,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( data.merging_params, partition_key_columns, block_size.max_block_size_rows, - enable_vertical_final, - query_info.has_filters_and_no_array_join_before_filter); + enable_vertical_final); merging_pipes.emplace_back(Pipe::unitePipes(std::move(pipes))); } diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index e8e7f99ce53..f3b3d8127d4 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -37,147 +37,6 @@ static void replaceFilterToConstant(Block & block, const String & filter_column_ } } -static std::shared_ptr getSelectByFinalIndices(Chunk & chunk) -{ - if (auto select_final_indices_info = std::dynamic_pointer_cast(chunk.getChunkInfo())) - { - const auto & index_column = select_final_indices_info->select_final_indices; - chunk.setChunkInfo(nullptr); - if (index_column && index_column->size() != chunk.getNumRows()) - return select_final_indices_info; - } - return nullptr; -} - -static void -executeSelectByIndices(Columns & columns, std::shared_ptr & select_final_indices_info, size_t & num_rows) -{ - if (select_final_indices_info) - { - const auto & index_column = select_final_indices_info->select_final_indices; - - for (auto & column : columns) - column = column->index(*index_column, 0); - - num_rows = index_column->size(); - } -} - -static std::unique_ptr combineFilterAndIndices( - std::unique_ptr description, - std::shared_ptr & select_final_indices_info, - size_t num_rows) -{ - if (select_final_indices_info) - { - const auto * index_column = select_final_indices_info->select_final_indices; - - if (description->hasOne()) - { - const auto & selected_by_indices = index_column->getData(); - const auto * selected_by_filter = description->data->data(); - /// We will recompute new has_one - description->has_one = 0; - /// At this point we know that the filter is not constant, just create a new filter - auto mutable_holder = ColumnUInt8::create(num_rows, 0); - auto & data = mutable_holder->getData(); - for (auto idx : selected_by_indices) - { - if (idx >= num_rows) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range {}", idx, num_rows); - data[idx] = 1; - } - - /// AND two filters - auto * begin = data.data(); - const auto * end = begin + num_rows; -#if defined(__AVX2__) - while (end - begin >= 32) - { - _mm256_storeu_si256( - reinterpret_cast<__m256i *>(begin), - _mm256_and_si256( - _mm256_loadu_si256(reinterpret_cast(begin)), - _mm256_loadu_si256(reinterpret_cast(selected_by_filter)))); - description->has_one |= !memoryIsZero(begin, 0, 32); - begin += 32; - selected_by_filter += 32; - } -#elif defined(__SSE2__) - while (end - begin >= 16) - { - _mm_storeu_si128( - reinterpret_cast<__m128i *>(begin), - _mm_and_si128( - _mm_loadu_si128(reinterpret_cast(begin)), - _mm_loadu_si128(reinterpret_cast(selected_by_filter)))); - description->has_one |= !memoryIsZero(begin, 0, 16); - begin += 16; - selected_by_filter += 16; - } -#endif - - while (end - begin >= 8) - { - *reinterpret_cast(begin) &= *reinterpret_cast(selected_by_filter); - description->has_one |= *reinterpret_cast(begin); - begin += 8; - selected_by_filter += 8; - } - - while (end - begin > 0) - { - *begin &= *selected_by_filter; - description->has_one |= *begin; - begin++; - selected_by_filter++; - } - - description->data_holder = std::move(mutable_holder); - description->data = &data; - } - } - return std::move(description); -} - -static std::unique_ptr combineFilterAndIndices( - std::unique_ptr description, - std::shared_ptr & select_final_indices_info, - size_t num_rows) -{ - /// Iterator interface to decorate data from output of std::set_intersection - struct Iterator - { - UInt8 * data; - Int64 & pop_cnt; - explicit Iterator(UInt8 * data_, Int64 & pop_cnt_) : data(data_), pop_cnt(pop_cnt_) {} - Iterator & operator = (UInt64 index) { data[index] = 1; ++pop_cnt; return *this; } - Iterator & operator ++ () { return *this; } - Iterator & operator * () { return *this; } - }; - - if (select_final_indices_info) - { - const auto * index_column = select_final_indices_info->select_final_indices; - - if (description->hasOne()) - { - std::unique_ptr res; - res->has_one = 0; - const auto & selected_by_indices = index_column->getData(); - const auto & selected_by_filter = description->filter_indices->getData(); - auto mutable_holder = ColumnUInt8::create(num_rows, 0); - auto & data = mutable_holder->getData(); - Iterator decorator(data.data(), res->has_one); - std::set_intersection(selected_by_indices.begin(), selected_by_indices.end(), selected_by_filter.begin(), selected_by_filter.end(), decorator); - res->data_holder = std::move(mutable_holder); - res->data = &data; - return res; - } - } - return std::move(description); -} - Block FilterTransform::transformHeader( const Block & header, const ActionsDAG * expression, const String & filter_column_name, bool remove_filter_column) { @@ -267,7 +126,6 @@ void FilterTransform::doTransform(Chunk & chunk) size_t num_rows_before_filtration = chunk.getNumRows(); auto columns = chunk.detachColumns(); DataTypes types; - auto select_final_indices_info = getSelectByFinalIndices(chunk); { Block block = getInputPort().getHeader().cloneWithColumns(columns); @@ -282,7 +140,6 @@ void FilterTransform::doTransform(Chunk & chunk) if (constant_filter_description.always_true || on_totals) { - executeSelectByIndices(columns, select_final_indices_info, num_rows_before_filtration); chunk.setColumns(std::move(columns), num_rows_before_filtration); removeFilterIfNeed(chunk); return; @@ -303,7 +160,6 @@ void FilterTransform::doTransform(Chunk & chunk) if (constant_filter_description.always_true) { - executeSelectByIndices(columns, select_final_indices_info, num_rows_before_filtration); chunk.setColumns(std::move(columns), num_rows_before_filtration); removeFilterIfNeed(chunk); return; @@ -311,15 +167,9 @@ void FilterTransform::doTransform(Chunk & chunk) std::unique_ptr filter_description; if (filter_column->isSparse()) - filter_description = combineFilterAndIndices( - std::make_unique(*filter_column), select_final_indices_info, num_rows_before_filtration); + filter_description = std::make_unique(*filter_column); else - filter_description = combineFilterAndIndices( - std::make_unique(*filter_column), select_final_indices_info, num_rows_before_filtration); - - - if (!filter_description->has_one) - return; + filter_description = std::make_unique(*filter_column); /** Let's find out how many rows will be in result. * To do this, we filter out the first non-constant column diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 11e2a2fc5e7..c3db0f510f9 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -215,12 +215,6 @@ struct SelectQueryInfo /// If query has aggregate functions bool has_aggregates = false; - /// If query has any filter and no arrayJoin before filter. Used by skipping FINAL - /// Skipping FINAL algorithm will output the original chunk and a column indices of - /// selected rows. If query has filter and doesn't have array join before any filter, - /// we can merge the indices with the first filter in FilterTransform later. - bool has_filters_and_no_array_join_before_filter = false; - ClusterPtr getCluster() const { return !optimized_cluster ? cluster : optimized_cluster; } bool settings_limit_offset_done = false; diff --git a/tests/queries/0_stateless/02893_vertical_final_array_join.reference b/tests/queries/0_stateless/02893_vertical_final_bugs.reference similarity index 99% rename from tests/queries/0_stateless/02893_vertical_final_array_join.reference rename to tests/queries/0_stateless/02893_vertical_final_bugs.reference index 27b54a2e42e..ab23116aa5f 100644 --- a/tests/queries/0_stateless/02893_vertical_final_array_join.reference +++ b/tests/queries/0_stateless/02893_vertical_final_bugs.reference @@ -1,3 +1,4 @@ +1 2 b 1 -- { echo ON } SELECT arrayJoin([(k1, v), (k2, v)]) AS row, row.1 as k FROM t FINAL WHERE k1 != 3 AND k = 1 ORDER BY row SETTINGS enable_vertical_final = 0; (1,4) 1 diff --git a/tests/queries/0_stateless/02893_vertical_final_array_join.sql b/tests/queries/0_stateless/02893_vertical_final_bugs.sql similarity index 65% rename from tests/queries/0_stateless/02893_vertical_final_array_join.sql rename to tests/queries/0_stateless/02893_vertical_final_bugs.sql index cc2e37fdc6e..e82ab674c58 100644 --- a/tests/queries/0_stateless/02893_vertical_final_array_join.sql +++ b/tests/queries/0_stateless/02893_vertical_final_bugs.sql @@ -1,3 +1,15 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/64543 +DROP TABLE IF EXISTS foo; +DROP TABLE IF EXISTS bar; +CREATE TABLE foo (id UInt64, seq UInt64) ENGINE = Memory; +CREATE TABLE bar (id UInt64, seq UInt64, name String) ENGINE = ReplacingMergeTree ORDER BY id; +INSERT INTO foo VALUES (1, 1); +INSERT INTO bar VALUES (1, 1, 'a') (2, 2, 'b'); +INSERT INTO bar VALUES (1, 2, 'b') (2, 3, 'c'); +SELECT * FROM bar INNER JOIN foo USING id WHERE bar.seq > foo.seq SETTINGS final = 1; + +-- Same problem possible can happen with array join +DROP TABLE IF EXISTS t; CREATE TABLE t (k1 UInt64, k2 UInt64, v UInt64) ENGINE = ReplacingMergeTree() ORDER BY (k1, k2); SET optimize_on_insert = 0; INSERT INTO t VALUES (1, 2, 3) (1, 2, 4) (2, 3, 4), (2, 3, 5); From 00c352a31c259802866d0992e917d1b64c947583 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 4 Jun 2024 12:28:07 +0200 Subject: [PATCH 254/856] Fix clang-tidy --- src/IO/S3/copyS3File.cpp | 4 ++-- src/IO/WriteBufferFromS3.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index c48d7965ac2..bb654c3f5c9 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -524,7 +524,7 @@ namespace void processPutRequest(S3::PutObjectRequest & request) { - size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); + size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3PutObject); @@ -726,7 +726,7 @@ namespace void processCopyRequest(S3::CopyObjectRequest & request) { - size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); + size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3CopyObject); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index ac63281d328..cd9949862ca 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -693,7 +693,7 @@ void WriteBufferFromS3::makeSinglepartUpload(WriteBufferFromS3::PartData && data auto & request = std::get<0>(*worker_data); size_t content_length = request.GetContentLength(); - size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); + size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries.value, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3PutObject); From 6e9343a0c754ef7a6511f81d40c393ef732f23a5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 10:34:28 +0000 Subject: [PATCH 255/856] 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 8b9bb1d47309c2ca927b9d50026b7dcc9be7b164 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 4 Jun 2024 14:09:32 +0200 Subject: [PATCH 256/856] Fix incorrect width calculation --- src/Common/UTF8Helpers.cpp | 18 ++++++++---------- .../03142_skip_ANSI_in_UTF8_compute_width.sql | 7 ++----- 2 files changed, 10 insertions(+), 15 deletions(-) diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 34eba832113..006ec33c08b 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -116,6 +116,11 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l while (i + 15 < size) { + if (isEscapeSequence) + { + break; + } + __m128i bytes = _mm_loadu_si128(reinterpret_cast(&data[i])); const uint16_t non_regular_width_mask = _mm_movemask_epi8( @@ -132,15 +137,8 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l } else { - if (isEscapeSequence) - { - break; - } - else - { - i += 16; - width += 16; - } + i += 16; + width += 16; } } #endif @@ -149,7 +147,7 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l { auto isParameterByte = isCSIParameterByte(data[i]); auto isIntermediateByte = isCSIIntermediateByte(data[i]); - auto ignore_width = isEscapeSequence & (isParameterByte || isIntermediateByte); + auto ignore_width = isEscapeSequence && (isParameterByte || isIntermediateByte); if (ignore_width || (data[i] == '[' && isEscapeSequence)) { diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql index f4b0bfe5888..812e7124526 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql @@ -1,6 +1,3 @@ SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x; -SELECT format('\x1b[38;2;{0};{1};{2}m█ test \x1b[0m', 255, 128, 128) AS x; -SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test', 255, 128, 128) AS x; -SELECT format('test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x; -SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x; -SELECT visibleWidth('0};{1};{2}m█'); \ No newline at end of file +SELECT 'Hello', format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x +SELECT visibleWidth(format('\x1b[38;2;{0};{1};{2}m█\x1b[0m',255,128,128)); From 54a9daa57007550fc253bd64dce3114331a211fd Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 4 Jun 2024 14:15:14 +0200 Subject: [PATCH 257/856] Update reference file --- .../03142_skip_ANSI_in_UTF8_compute_width.reference | 12 +++++++++++- .../03142_skip_ANSI_in_UTF8_compute_width.sql | 6 +++--- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference index 864f62d3113..fa161970a3d 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference @@ -1,5 +1,15 @@ ┏━━━┓ ┃ x ┃ ┡━━━┩ -1. │ █ │ +1. │ █ │ └───┘ + ┏━━━━━━━━━┳━━━━━━━━━━┓ + ┃ 'Hello' ┃ x ┃ + ┡━━━━━━━━━╇━━━━━━━━━━┩ +1. │ Hello │ █ test █ │ + └─────────┴──────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ visibleWidth(format('[38;2;{0};{1};{2}m█', 255, 128, 128)) ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 22 │ + └─────────────────────────────────────────────────────┘ diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql index 812e7124526..17608655ec5 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql @@ -1,3 +1,3 @@ -SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x; -SELECT 'Hello', format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x -SELECT visibleWidth(format('\x1b[38;2;{0};{1};{2}m█\x1b[0m',255,128,128)); +SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x FORMAT Pretty; +SELECT 'Hello', format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x FORMAT Pretty; +SELECT visibleWidth(format('\x1b[38;2;{0};{1};{2}m█\x1b[0m',255,128,128)) FORMAT Pretty; From edbef6ac9f0e6ae3e94a4732a453de0dcabb3a93 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 4 Jun 2024 14:22:21 +0200 Subject: [PATCH 258/856] Add 'Known incompatibilities of JOIN clause' --- docs/en/operations/analyzer.md | 42 +++++++++++++++++++++++++++++++--- 1 file changed, 39 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 62bb45eeb7d..c559924314b 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -47,9 +47,45 @@ The previous query analyzer fixed this query by moving the `number > 5` filter f ### Known incompatibilities of JOIN clause -* Using expression from `SELECT` list in `JOIN` key as an expression from LEFT table. Example. Fix (best effort, should be under compatibility flag). -* Similar issue ^. Alias for column (in select list) now applied to JOIN result (and not to left table). Example from Denny Crane. New behavior is the correct one. Will try to add best-effort compatibility setting. -* Columns names are changed for some queries. This might breaks some scripts. Example. +#### Join using column from projection + +Alias from the `SELECT` list can not be used as a `JOIN USING` key by default. + +A new setting, `analyzer_compatibility_join_using_top_level_identifier`, when enabled, alters the behavior of `JOIN USING` to prefer to resolve identifiers based on expressions from the projection list of the SELECT query, rather than using the columns from left table directly. + +*Example:* + +```sql +SELECT a + 1 AS b, t2.s +FROM Values('a UInt64, b UInt64', (1, 1)) AS t1 +JOIN Values('b UInt64, s String', (1, 'one'), (2, 'two')) t2 +USING (b); +``` + +With `analyzer_compatibility_join_using_top_level_identifier` set to `true`, the join condition is interpreted as `t1.a + 1 = t2.b`, matching the behavior of earlier versions. So, the result will be `2, 'two'` +When the setting is `false`, the join condition defaults to `t1.b = t2.b`, and the query will return `2, 'one'`. +In case then `b` is not present in `t1`, the query will fail with an error. + +#### Changes in Behavior with `JOIN USING` and `ALIAS/MATERIALIZED` Columns + +In the new analyzer, using `*` in a `JOIN USING` query that involves `ALIAS` or `MATERIALIZED` columns will include that columns in the result set by default. + +*Example:* + +```sql +CREATE TABLE t1 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id; +INSERT INTO t1 VALUES (1), (2); + +CREATE TABLE t2 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id; +INSERT INTO t2 VALUES (2), (3); + +SELECT * FROM t1 +FULL JOIN t2 USING (payload); +``` + +In the new analyzer, the result of this query will include the `payload` column along with `id` from both tables. In contrast, the previous analyzer would only include these `ALIAS` columns if specific settings (`asterisk_include_alias_columns` or `asterisk_include_materialized_columns`) were enabled, and the columns might appear in a different order. + +To ensure consistent and expected results, especially when migrating old queries to the new analyzer, it is advisable to specify columns explicitly in the `SELECT` clause rather than using `*`. ### Projection column names changes From e7541224751137e193941aa06f71a80c44bfd1c3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 4 Jun 2024 14:41:30 +0200 Subject: [PATCH 259/856] Handling of Type Modifiers for columns in `USING` Clause --- docs/en/operations/analyzer.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index c559924314b..f5b900d01f7 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -87,6 +87,23 @@ In the new analyzer, the result of this query will include the `payload` column To ensure consistent and expected results, especially when migrating old queries to the new analyzer, it is advisable to specify columns explicitly in the `SELECT` clause rather than using `*`. +#### Handling of Type Modifiers for columns in `USING` Clause + +In the new version of the analyzer, the rules for determining the common supertype for columns specified in the `USING` clause have been standardized to produce more predictable outcomes, especially when dealing with type modifiers like `LowCardinality` and `Nullable`. + +- `LowCardinality(T)` and `T`: When a column of type `LowCardinality(T)` is joined with a column of type `T`, the resulting common supertype will be `T`, effectively discarding the `LowCardinality` modifier. + +- `Nullable(T)` and `T`: When a column of type `Nullable(T)` is joined with a column of type `T`, the resulting common supertype will be `Nullable(T)`, ensuring that the nullable property is preserved. + +*Example:* + +```sql +SELECT id, toTypeName(id) FROM Values('id LowCardinality(String)', ('a')) AS t1 +FULL OUTER JOIN Values('id String', ('b')) AS t2 +USING (id); +``` + +In this query, the common supertype for `id` is determined as `String`, discarding the `LowCardinality` modifier from `t1`. ### Projection column names changes From 252b5f51c2f8e5a6f41d21245340fae9782445c1 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 4 Jun 2024 15:07:44 +0200 Subject: [PATCH 260/856] update test --- .../03142_skip_ANSI_in_UTF8_compute_width.reference | 5 ----- .../0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql | 1 - 2 files changed, 6 deletions(-) diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference index fa161970a3d..6d375fd471a 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference @@ -8,8 +8,3 @@ ┡━━━━━━━━━╇━━━━━━━━━━┩ 1. │ Hello │ █ test █ │ └─────────┴──────────┘ - ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ - ┃ visibleWidth(format('[38;2;{0};{1};{2}m█', 255, 128, 128)) ┃ - ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -1. │ 22 │ - └─────────────────────────────────────────────────────┘ diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql index 17608655ec5..49f689a4cc5 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql @@ -1,3 +1,2 @@ SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x FORMAT Pretty; SELECT 'Hello', format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x FORMAT Pretty; -SELECT visibleWidth(format('\x1b[38;2;{0};{1};{2}m█\x1b[0m',255,128,128)) FORMAT Pretty; From aa4beb0be97ae87aea960fc642359ff20dac575a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 13:27:29 +0000 Subject: [PATCH 261/856] 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 086fc73f42f7d29d2082b66b24ab769230f47528 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 13:52:10 +0000 Subject: [PATCH 262/856] Remove aliases from fiter condition. --- src/Interpreters/ActionsDAG.cpp | 31 ++++++++++++++++++++++++- src/Interpreters/ActionsDAG.h | 2 ++ src/Processors/QueryPlan/FilterStep.cpp | 2 ++ 3 files changed, 34 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index cfccc835d29..cb88aad5a10 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -596,6 +596,34 @@ void ActionsDAG::removeUnusedActions(const std::unordered_set & us std::erase_if(inputs, [&](const Node * node) { return !visited_nodes.contains(node); }); } + +void ActionsDAG::removeAliasesForFilter(const std::string & filter_name) +{ + const auto & filter_node = findInOutputs(filter_name); + std::stack stack; + stack.push(const_cast(&filter_node)); + + std::unordered_set visited; + visited.insert(stack.top()); + + while (!stack.empty()) + { + auto * node = stack.top(); + stack.pop(); + for (auto & child : node->children) + { + while (child->type == ActionType::ALIAS) + child = child->children.front(); + + if (!visited.contains(child)) + { + stack.push(const_cast(child)); + visited.insert(child); + } + } + } +} + ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) { auto actions = std::make_shared(); @@ -1830,7 +1858,8 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split input_node.result_name = child->result_name; child_data.to_second = &second_nodes.emplace_back(std::move(input_node)); - new_inputs.push_back(child); + if (child->type != ActionType::INPUT) + new_inputs.push_back(child); } } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 8c0e3f0e576..97549088cdf 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -195,6 +195,8 @@ public: /// Remove actions that are not needed to compute output nodes with required names void removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs = true, bool allow_constant_folding = true); + void removeAliasesForFilter(const std::string & filter_name); + /// Transform the current DAG in a way that leaf nodes get folded into their parents. It's done /// because each projection can provide some columns as inputs to substitute certain sub-DAGs /// (expressions). Consider the following example: diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 56b31b2c8ba..72934665b5c 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -50,6 +50,8 @@ FilterStep::FilterStep( , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { + actions_dag = actions_dag->clone(); + actions_dag->removeAliasesForFilter(filter_column_name); } void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) From 73ba23cae7a920a30c444c71e75ed099e2a0ea9b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 15:55:10 +0200 Subject: [PATCH 263/856] 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 c8def45cc2cb232bd2e2eddad5d5fa33cc23bf4f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 14:14:43 +0000 Subject: [PATCH 264/856] Fixing tests. --- src/DataTypes/DataTypeNullable.cpp | 18 +++++++++--------- src/Planner/PlannerExpressionAnalysis.cpp | 21 +++++++++++++++------ 2 files changed, 24 insertions(+), 15 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index d97edef5c39..df9689919cc 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -176,16 +176,16 @@ DataTypePtr removeNullableOrLowCardinalityNullable(const DataTypePtr & type) } bool canContainNull(const IDataType & type) - { - if (type.isNullable() || type.isLowCardinalityNullable() || isDynamic(type)) - return true; +{ + if (type.isNullable() || type.isLowCardinalityNullable() || isDynamic(type)) + return true; - if (const auto * variant = typeid_cast(&type)) - for (const auto & elem : variant->getVariants()) - if (canContainNull(*elem)) - return true; + if (const auto * variant = typeid_cast(&type)) + for (const auto & elem : variant->getVariants()) + if (canContainNull(*elem)) + return true; - return false; - } + return false; +} } diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 983c5d3d5b6..060bbba1c05 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -539,8 +539,11 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (query_node.hasWhere()) { where_analysis_result_optional = analyzeFilter(query_node.getWhere(), current_output_columns, planner_context, actions_chain); - where_action_step_index_optional = actions_chain.getLastStepIndex(); - current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + if (where_analysis_result_optional) + { + where_action_step_index_optional = actions_chain.getLastStepIndex(); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + } } auto aggregation_analysis_result_optional = analyzeAggregation(query_tree, current_output_columns, planner_context, actions_chain); @@ -553,8 +556,11 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (query_node.hasHaving()) { having_analysis_result_optional = analyzeFilter(query_node.getHaving(), current_output_columns, planner_context, actions_chain); - having_action_step_index_optional = actions_chain.getLastStepIndex(); - current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + if (having_analysis_result_optional) + { + having_action_step_index_optional = actions_chain.getLastStepIndex(); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + } } auto window_analysis_result_optional = analyzeWindow(query_tree, current_output_columns, planner_context, actions_chain); @@ -567,8 +573,11 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (query_node.hasQualify()) { qualify_analysis_result_optional = analyzeFilter(query_node.getQualify(), current_output_columns, planner_context, actions_chain); - qualify_action_step_index_optional = actions_chain.getLastStepIndex(); - current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + if (qualify_analysis_result_optional) + { + qualify_action_step_index_optional = actions_chain.getLastStepIndex(); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + } } auto projection_analysis_result = analyzeProjection(query_node, current_output_columns, planner_context, actions_chain); From 830acd3a8927e49c94a4d82817a1364c4dfd0604 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Tue, 4 Jun 2024 15:32:38 +0000 Subject: [PATCH 265/856] Support non-const scale arguments in rounding functions --- .../functions/rounding-functions.md | 8 +- .../functions/rounding-functions.md | 2 +- src/Functions/FunctionsRound.h | 251 +- .../03165_round_scale_as_column.reference | 2165 +++++++++++++++++ .../03165_round_scale_as_column.sql | 125 + 5 files changed, 2475 insertions(+), 76 deletions(-) create mode 100644 tests/queries/0_stateless/03165_round_scale_as_column.reference create mode 100644 tests/queries/0_stateless/03165_round_scale_as_column.sql diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index d18185c5013..55a4b3ce9df 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -9,8 +9,8 @@ sidebar_label: Rounding ## floor(x\[, N\]) Returns the largest round number that is less than or equal to `x`. A round number is a multiple of 1/10N, or the nearest number of the appropriate data type if 1 / 10N isn’t exact. -‘N’ is an integer constant, optional parameter. By default it is zero, which means to round to an integer. -‘N’ may be negative. +`N` is an integer, optional parameter. By default it is zero, which means to round to an integer. +`N` may be negative. Examples: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` @@ -70,7 +70,7 @@ round(expression [, decimal_places]) **Arguments** - `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types). -- `decimal-places` — An integer value. +- `decimal-places` — The number of decimal places to round to. - If `decimal-places > 0` then the function rounds the value to the right of the decimal point. - If `decimal-places < 0` then the function rounds the value to the left of the decimal point. - If `decimal-places = 0` then the function rounds the value to integer. In this case the argument can be omitted. @@ -264,7 +264,7 @@ Result: ## roundDuration -Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`. +Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`. **Syntax** diff --git a/docs/ru/sql-reference/functions/rounding-functions.md b/docs/ru/sql-reference/functions/rounding-functions.md index cc939f69afc..f1dd57505ea 100644 --- a/docs/ru/sql-reference/functions/rounding-functions.md +++ b/docs/ru/sql-reference/functions/rounding-functions.md @@ -10,7 +10,7 @@ sidebar_label: "Функции округления" Возвращает наибольшее круглое число, которое меньше или равно, чем x. Круглым называется число, кратное 1 / 10N или ближайшее к нему число соответствующего типа данных, если 1 / 10N не представимо точно. -N - целочисленная константа, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа. +N - целочисленный аргумент, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа. N может быть отрицательным. Примеры: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 1f20fbff24e..978b6d88d05 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -187,13 +187,13 @@ struct IntegerRoundingComputation }; -#ifdef __SSE4_1__ - -template +template class BaseFloatRoundingComputation; +#ifdef __SSE4_1__ + template <> -class BaseFloatRoundingComputation +class BaseFloatRoundingComputation { public: using ScalarType = Float32; @@ -214,7 +214,7 @@ public: }; template <> -class BaseFloatRoundingComputation +class BaseFloatRoundingComputation { public: using ScalarType = Float64; @@ -234,9 +234,9 @@ public: } }; -#else +#endif -/// Implementation for ARM. Not vectorized. +/// Implementation for ARM/columnar scale argument. Not vectorized. inline float roundWithMode(float x, RoundingMode mode) { @@ -265,7 +265,7 @@ inline double roundWithMode(double x, RoundingMode mode) } template -class BaseFloatRoundingComputation +class BaseFloatRoundingComputation { public: using ScalarType = T; @@ -285,15 +285,13 @@ public: } }; -#endif - /** Implementation of low-level round-off functions for floating-point values. */ -template -class FloatRoundingComputation : public BaseFloatRoundingComputation +template +class FloatRoundingComputation : public BaseFloatRoundingComputation { - using Base = BaseFloatRoundingComputation; + using Base = BaseFloatRoundingComputation; public: static void compute(const T * __restrict in, const typename Base::VectorType & scale, T * __restrict out) @@ -325,15 +323,22 @@ struct FloatRoundingImpl private: static_assert(!is_decimal); - using Op = FloatRoundingComputation; - using Data = std::array; + template + using Op = FloatRoundingComputation; + using Data = std::array::data_count>; using ColumnType = ColumnVector; using Container = typename ColumnType::Container; public: static NO_INLINE void apply(const Container & in, size_t scale, Container & out) { - auto mm_scale = Op::prepare(scale); + auto mm_scale = Op<>::prepare(scale); const size_t data_count = std::tuple_size(); @@ -345,7 +350,7 @@ public: while (p_in < limit) { - Op::compute(p_in, mm_scale, p_out); + Op<>::compute(p_in, mm_scale, p_out); p_in += data_count; p_out += data_count; } @@ -358,10 +363,17 @@ public: size_t tail_size_bytes = (end_in - p_in) * sizeof(*p_in); memcpy(&tmp_src, p_in, tail_size_bytes); - Op::compute(reinterpret_cast(&tmp_src), mm_scale, reinterpret_cast(&tmp_dst)); + Op<>::compute(reinterpret_cast(&tmp_src), mm_scale, reinterpret_cast(&tmp_dst)); memcpy(p_out, &tmp_dst, tail_size_bytes); } } + + static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) + { + using ScalarOp = Op; + auto s = ScalarOp::prepare(scale); + ScalarOp::compute(in, s, out); + } }; template @@ -417,6 +429,11 @@ public: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected 'scale' parameter passed to function"); } } + + static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) + { + Op::compute(in, scale, out); + } }; @@ -452,11 +469,40 @@ public: memcpy(out.data(), in.data(), in.size() * sizeof(T)); } } -}; + static void applyOne(const NativeType* __restrict in, UInt32 in_scale, NativeType* __restrict out, Scale scale_arg) + { + scale_arg = in_scale - scale_arg; + if (scale_arg > 0) + { + auto scale = intExp10OfSize(scale_arg); + Op::compute(in, scale, out); + } + else + { + memcpy(out, in, sizeof(T)); + } + } +}; /** Select the appropriate processing algorithm depending on the scale. */ +inline void validateScale(Int64 scale64) +{ + if (scale64 > std::numeric_limits::max() || scale64 < std::numeric_limits::min()) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large"); +} + +inline Scale getScaleArg(const ColumnConst* scale_col) +{ + const auto& scale_field = scale_col->getField(); + + Int64 scale64 = scale_field.get(); + validateScale(scale64); + + return scale64; +} + template struct Dispatcher { @@ -465,9 +511,10 @@ struct Dispatcher FloatRoundingImpl, IntegerRoundingImpl>; - static ColumnPtr apply(const IColumn * col_general, Scale scale_arg) + template + static ColumnPtr apply(const IColumn * data_col, const IColumn * scale_col = nullptr) { - const auto & col = checkAndGetColumn>(*col_general); + const auto & col = checkAndGetColumn>(*data_col); auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_res = col_res->getData(); @@ -475,20 +522,62 @@ struct Dispatcher if (!vec_res.empty()) { - if (scale_arg == 0) + using ColVecScale = ColumnVector; + if (scale_col == nullptr || isColumnConst(*scale_col)) { - size_t scale = 1; - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst(scale_col)); + if (scale_arg == 0) + { + size_t scale = 1; + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + } + else if (scale_arg > 0) + { + size_t scale = intExp10(scale_arg); + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + } + else + { + size_t scale = intExp10(-scale_arg); + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + } } - else if (scale_arg > 0) + else if (const auto scale_typed = checkAndGetColumn(scale_col)) { - size_t scale = intExp10(scale_arg); - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); - } - else - { - size_t scale = intExp10(-scale_arg); - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + const auto & in = col.getData(); + const auto & scale_data = scale_typed->getData(); + const size_t count = in.size(); + + const T * end_in = in.data() + count; + const T * __restrict p_in = in.data(); + const ScaleType * __restrict p_scale = scale_data.data(); + T * __restrict p_out = vec_res.data(); + + while (p_in < end_in) + { + Int64 scale64 = *p_scale; + validateScale(scale64); + Scale raw_scale = scale64; + + if (raw_scale == 0) + { + size_t scale = 1; + FunctionRoundingImpl::applyOne(p_in, scale, p_out); + } + else if (raw_scale > 0) + { + size_t scale = intExp10(raw_scale); + FunctionRoundingImpl::applyOne(p_in, scale, p_out); + } + else + { + size_t scale = intExp10(-raw_scale); + FunctionRoundingImpl::applyOne(p_in, scale, p_out); + } + ++p_in; + ++p_scale; + ++p_out; + } } } @@ -500,16 +589,46 @@ template { public: - static ColumnPtr apply(const IColumn * col_general, Scale scale_arg) + template + static ColumnPtr apply(const IColumn * data_col, const IColumn* scale_col = nullptr) { - const auto & col = checkAndGetColumn>(*col_general); + const auto & col = checkAndGetColumn>(*data_col); const typename ColumnDecimal::Container & vec_src = col.getData(); auto col_res = ColumnDecimal::create(vec_src.size(), col.getScale()); auto & vec_res = col_res->getData(); if (!vec_res.empty()) - DecimalRoundingImpl::apply(col.getData(), col.getScale(), vec_res, scale_arg); + { + using ColVecScale = ColumnVector; + if (scale_col == nullptr || isColumnConst(*scale_col)) + { + auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst(scale_col)); + DecimalRoundingImpl::apply(col.getData(), col.getScale(), vec_res, scale_arg); + } + else if (const auto scale_typed = checkAndGetColumn(scale_col)) + { + const auto & scale = scale_typed->getData(); + const size_t count = vec_src.size(); + + using NativeType = typename T::NativeType; + const NativeType * __restrict p_in = reinterpret_cast(vec_src.data()); + const ScaleType * __restrict p_scale = scale.data(); + const NativeType * end_in = p_in + count; + NativeType * __restrict p_out = reinterpret_cast(vec_res.data()); + while (p_in < end_in) + { + Int64 scale64 = *p_scale; + validateScale(scale64); + Scale raw_scale = scale64; + + DecimalRoundingImpl::applyOne(p_in, col.getScale(), p_out, raw_scale); + ++p_in; + ++p_scale; + ++p_out; + } + } + } return col_res; } @@ -550,50 +669,41 @@ public: return arguments[0]; } - static Scale getScaleArg(const ColumnsWithTypeAndName & arguments) - { - if (arguments.size() == 2) - { - const IColumn & scale_column = *arguments[1].column; - if (!isColumnConst(scale_column)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must be constant"); - - Field scale_field = assert_cast(scale_column).getField(); - if (scale_field.getType() != Field::Types::UInt64 - && scale_field.getType() != Field::Types::Int64) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type"); - - Int64 scale64 = scale_field.get(); - if (scale64 > std::numeric_limits::max() - || scale64 < std::numeric_limits::min()) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large"); - - return scale64; - } - return 0; - } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnWithTypeAndName & column = arguments[0]; - Scale scale_arg = getScaleArg(arguments); + const ColumnWithTypeAndName & data_column = arguments[0]; ColumnPtr res; - auto call = [&](const auto & types) -> bool + auto callData = [&](const auto & types) -> bool { using Types = std::decay_t; - using DataType = typename Types::LeftType; + using DataType = typename Types::RightType; - if constexpr (IsDataTypeNumber || IsDataTypeDecimal) + if (arguments.size() > 1) { - using FieldType = typename DataType::FieldType; - res = Dispatcher::apply(column.column.get(), scale_arg); + const ColumnWithTypeAndName & scale_column = arguments[1]; + + auto callScale = [&](const auto & scaleTypes) -> bool + { + using ScaleTypes = std::decay_t; + using ScaleType = typename ScaleTypes::RightType; + + if (isColumnConst(*data_column.column) && !isColumnConst(*scale_column.column)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale column must be const for const data column"); + + res = Dispatcher::template apply(data_column.column.get(), scale_column.column.get()); + return true; + }; + + TypeIndex right_index = scale_column.type->getTypeId(); + if (!callOnBasicType(right_index, callScale)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type"); return true; } - return false; + res = Dispatcher::template apply(data_column.column.get()); + return true; }; #if !defined(__SSE4_1__) @@ -605,10 +715,9 @@ public: throw Exception(ErrorCodes::CANNOT_SET_ROUNDING_MODE, "Cannot set floating point rounding mode"); #endif - if (!callOnIndexAndDataType(column.type->getTypeId(), call)) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", column.name, getName()); - } + TypeIndex left_index = data_column.type->getTypeId(); + if (!callOnBasicType(left_index, callData)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", data_column.name, getName()); return res; } diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.reference b/tests/queries/0_stateless/03165_round_scale_as_column.reference new file mode 100644 index 00000000000..9ad25ed466a --- /dev/null +++ b/tests/queries/0_stateless/03165_round_scale_as_column.reference @@ -0,0 +1,2165 @@ +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-1 -1 -1 -1 -1 -1 +-0.9 -1 -1 -1 -0 -0 +-0.8 -1 -1 -1 -0 -0 +-0.7 -1 -1 -1 -0 -0 +-0.6 -1 -1 -1 -0 -0 +-0.5 -0 -0 -1 -0 -0 +-0.4 -0 -0 -1 -0 -0 +-0.3 -0 -0 -1 -0 -0 +-0.2 -0 -0 -1 -0 -0 +-0.1 -0 -0 -1 -0 -0 +0 0 0 0 0 0 +0.1 0 0 0 1 0 +0.2 0 0 0 1 0 +0.3 0 0 0 1 0 +0.4 0 0 0 1 0 +0.5 0 0 0 1 0 +0.6 1 1 0 1 0 +0.7 1 1 0 1 0 +0.8 1 1 0 1 0 +0.9 1 1 0 1 0 +-1 -1 -1 -1 -1 -1 +-0.9 -1 -1 -1 -0 -0 +-0.8 -1 -1 -1 -0 -0 +-0.7 -1 -1 -1 -0 -0 +-0.6 -1 -1 -1 -0 -0 +-0.5 -0 -0 -1 -0 -0 +-0.4 -0 -0 -1 -0 -0 +-0.3 -0 -0 -1 -0 -0 +-0.2 -0 -0 -1 -0 -0 +-0.1 -0 -0 -1 -0 -0 +0 0 0 0 0 0 +0.1 0 0 0 1 0 +0.2 0 0 0 1 0 +0.3 0 0 0 1 0 +0.4 0 0 0 1 0 +0.5 0 0 0 1 0 +0.6 1 1 0 1 0 +0.7 1 1 0 1 0 +0.8 1 1 0 1 0 +0.9 1 1 0 1 0 +-1 -1 -1 -1 -1 -1 +-0.9 -0.9 -0.9 -0.9 -0.9 -0.9 +-0.8 -0.8 -0.8 -0.8 -0.8 -0.8 +-0.7 -0.7 -0.7 -0.7 -0.7 -0.7 +-0.6 -0.6 -0.6 -0.6 -0.6 -0.6 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 +-0.4 -0.4 -0.4 -0.4 -0.4 -0.4 +-0.3 -0.3 -0.3 -0.3 -0.3 -0.3 +-0.2 -0.2 -0.2 -0.2 -0.2 -0.2 +-0.1 -0.1 -0.1 -0.1 -0.1 -0.1 +0 0 0 0 0 0 +0.1 0.1 0.1 0.1 0.1 0.1 +0.2 0.2 0.2 0.2 0.2 0.2 +0.3 0.3 0.3 0.3 0.3 0.3 +0.4 0.4 0.4 0.4 0.4 0.4 +0.5 0.5 0.5 0.5 0.5 0.5 +0.6 0.6 0.6 0.6 0.6 0.6 +0.7 0.7 0.7 0.7 0.7 0.7 +0.8 0.8 0.8 0.8 0.8 0.8 +0.9 0.9 0.9 0.9 0.9 0.9 +-1 -1 -1 -1 -1 -1 +-0.9 -0.9 -0.9 -0.9 -0.9 -0.9 +-0.8 -0.8 -0.8 -0.8 -0.8 -0.8 +-0.7 -0.7 -0.7 -0.7 -0.7 -0.7 +-0.6 -0.6 -0.6 -0.6 -0.6 -0.6 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 +-0.4 -0.4 -0.4 -0.4 -0.4 -0.4 +-0.3 -0.3 -0.3 -0.3 -0.3 -0.3 +-0.2 -0.2 -0.2 -0.2 -0.2 -0.2 +-0.1 -0.1 -0.1 -0.1 -0.1 -0.1 +0 0 0 0 0 0 +0.1 0.1 0.1 0.1 0.1 0.1 +0.2 0.2 0.2 0.2 0.2 0.2 +0.3 0.3 0.3 0.3 0.3 0.3 +0.4 0.4 0.4 0.4 0.4 0.4 +0.5 0.5 0.5 0.5 0.5 0.5 +0.6 0.6 0.6 0.6 0.6 0.6 +0.7 0.7 0.7 0.7 0.7 0.7 +0.8 0.8 0.8 0.8 0.8 0.8 +0.9 0.9 0.9 0.9 0.9 0.9 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 -0 -0 +-8 -10 -10 -10 -0 -0 +-7 -10 -10 -10 -0 -0 +-6 -10 -10 -10 -0 -0 +-5 -0 -0 -10 -0 -0 +-4 -0 -0 -10 -0 -0 +-3 -0 -0 -10 -0 -0 +-2 -0 -0 -10 -0 -0 +-1 -0 -0 -10 -0 -0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 0 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 -0 -0 +-8 -10 -10 -10 -0 -0 +-7 -10 -10 -10 -0 -0 +-6 -10 -10 -10 -0 -0 +-5 -0 -0 -10 -0 -0 +-4 -0 -0 -10 -0 -0 +-3 -0 -0 -10 -0 -0 +-2 -0 -0 -10 -0 -0 +-1 -0 -0 -10 -0 -0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 0 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 -0 -0 -100 -0 -0 +-9 -0 -0 -100 -0 -0 +-8 -0 -0 -100 -0 -0 +-7 -0 -0 -100 -0 -0 +-6 -0 -0 -100 -0 -0 +-5 -0 -0 -100 -0 -0 +-4 -0 -0 -100 -0 -0 +-3 -0 -0 -100 -0 -0 +-2 -0 -0 -100 -0 -0 +-1 -0 -0 -100 -0 -0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 -0 -0 -100 -0 -0 +-9 -0 -0 -100 -0 -0 +-8 -0 -0 -100 -0 -0 +-7 -0 -0 -100 -0 -0 +-6 -0 -0 -100 -0 -0 +-5 -0 -0 -100 -0 -0 +-4 -0 -0 -100 -0 -0 +-3 -0 -0 -100 -0 -0 +-2 -0 -0 -100 -0 -0 +-1 -0 -0 -100 -0 -0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +CHECKPOINT1 +id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 244 4 244 244 244 244 244 +id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 -56 4 -56 -56 -56 -56 -56 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 -12 4 -12 -12 -12 -12 -12 +id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -1 0 -1 -1 -1 -1 -1 +21 -0.9 0 -1 -1 -1 -0 -0 +22 -0.8 0 -1 -1 -1 -0 -0 +23 -0.7 0 -1 -1 -1 -0 -0 +24 -0.6 0 -1 -1 -1 -0 -0 +25 -0.5 0 -0 -0 -1 -0 -0 +26 -0.4 0 -0 -0 -1 -0 -0 +27 -0.3 0 -0 -0 -1 -0 -0 +28 -0.2 0 -0 -0 -1 -0 -0 +29 -0.1 0 -0 -0 -1 -0 -0 +30 0 0 0 0 0 0 0 +31 0.1 0 0 0 0 1 0 +32 0.2 0 0 0 0 1 0 +33 0.3 0 0 0 0 1 0 +34 0.4 0 0 0 0 1 0 +35 0.5 0 0 0 0 1 0 +36 0.6 0 1 1 0 1 0 +37 0.7 0 1 1 0 1 0 +38 0.8 0 1 1 0 1 0 +39 0.9 0 1 1 0 1 0 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 0 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -1 -1 -0 -0 -10 -0 -0 +61 -0.9 -1 -0 -0 -10 -0 -0 +62 -0.8 -1 -0 -0 -10 -0 -0 +63 -0.7 -1 -0 -0 -10 -0 -0 +64 -0.6 -1 -0 -0 -10 -0 -0 +65 -0.5 -1 -0 -0 -10 -0 -0 +66 -0.4 -1 -0 -0 -10 -0 -0 +67 -0.3 -1 -0 -0 -10 -0 -0 +68 -0.2 -1 -0 -0 -10 -0 -0 +69 -0.1 -1 -0 -0 -10 -0 -0 +70 0 -1 0 0 0 0 0 +71 0.1 -1 0 0 0 10 0 +72 0.2 -1 0 0 0 10 0 +73 0.3 -1 0 0 0 10 0 +74 0.4 -1 0 0 0 10 0 +75 0.5 -1 0 0 0 10 0 +76 0.6 -1 0 0 0 10 0 +77 0.7 -1 0 0 0 10 0 +78 0.8 -1 0 0 0 10 0 +79 0.9 -1 0 0 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -1 -2 -0 -0 -100 -0 -0 +101 -0.9 -2 -0 -0 -100 -0 -0 +102 -0.8 -2 -0 -0 -100 -0 -0 +103 -0.7 -2 -0 -0 -100 -0 -0 +104 -0.6 -2 -0 -0 -100 -0 -0 +105 -0.5 -2 -0 -0 -100 -0 -0 +106 -0.4 -2 -0 -0 -100 -0 -0 +107 -0.3 -2 -0 -0 -100 -0 -0 +108 -0.2 -2 -0 -0 -100 -0 -0 +109 -0.1 -2 -0 -0 -100 -0 -0 +110 0 -2 0 0 0 0 0 +111 0.1 -2 0 0 0 100 0 +112 0.2 -2 0 0 0 100 0 +113 0.3 -2 0 0 0 100 0 +114 0.4 -2 0 0 0 100 0 +115 0.5 -2 0 0 0 100 0 +116 0.6 -2 0 0 0 100 0 +117 0.7 -2 0 0 0 100 0 +118 0.8 -2 0 0 0 100 0 +119 0.9 -2 0 0 0 100 0 +200 12345.679 0 12346 12346 12345 12346 12345 +201 12345.679 -1 12350 12350 12340 12350 12340 +202 12345.679 -2 12300 12300 12300 12400 12300 +203 12345.679 -3 12000 12000 12000 13000 12000 +204 12345.679 -4 10000 10000 10000 20000 10000 +205 12345.679 -5 0 0 0 100000 0 +206 12345.679 -6 0 0 0 1000000 0 +207 12345.679 -7 0 0 0 10000000 0 +208 12345.679 -8 0 0 0 100000000 0 +209 12345.679 -9 0 0 0 1000000000 0 +210 12345.679 0 12346 12346 12345 12346 12345 +211 12345.679 -1 12350 12350 12340 12350 12340 +212 12345.679 -2 12300 12300 12300 12400 12300 +213 12345.679 -3 12000 12000 12000 13000 12000 +214 12345.679 -4 10000 10000 10000 20000 10000 +215 12345.679 -5 0 0 0 100000 0 +216 12345.679 -6 0 0 0 1000000 0 +217 12345.679 -7 0 0 0 10000000 0 +218 12345.679 -8 0 0 0 100000000 0 +219 12345.679 -9 0 0 0 1000000000 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -1 0 -1 -1 -1 -1 -1 +21 -0.9 0 -1 -1 -1 -0 -0 +22 -0.8 0 -1 -1 -1 -0 -0 +23 -0.7 0 -1 -1 -1 -0 -0 +24 -0.6 0 -1 -1 -1 -0 -0 +25 -0.5 0 -0 -0 -1 -0 -0 +26 -0.4 0 -0 -0 -1 -0 -0 +27 -0.3 0 -0 -0 -1 -0 -0 +28 -0.2 0 -0 -0 -1 -0 -0 +29 -0.1 0 -0 -0 -1 -0 -0 +30 0 0 0 0 0 0 0 +31 0.1 0 0 0 0 1 0 +32 0.2 0 0 0 0 1 0 +33 0.3 0 0 0 0 1 0 +34 0.4 0 0 0 0 1 0 +35 0.5 0 0 0 0 1 0 +36 0.6 0 1 1 0 1 0 +37 0.7 0 1 1 0 1 0 +38 0.8 0 1 1 0 1 0 +39 0.9 0 1 1 0 1 0 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 0 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -1 -1 -0 -0 -10 -0 -0 +61 -0.9 -1 -0 -0 -10 -0 -0 +62 -0.8 -1 -0 -0 -10 -0 -0 +63 -0.7 -1 -0 -0 -10 -0 -0 +64 -0.6 -1 -0 -0 -10 -0 -0 +65 -0.5 -1 -0 -0 -10 -0 -0 +66 -0.4 -1 -0 -0 -10 -0 -0 +67 -0.3 -1 -0 -0 -10 -0 -0 +68 -0.2 -1 -0 -0 -10 -0 -0 +69 -0.1 -1 -0 -0 -10 -0 -0 +70 0 -1 0 0 0 0 0 +71 0.1 -1 0 0 0 10 0 +72 0.2 -1 0 0 0 10 0 +73 0.3 -1 0 0 0 10 0 +74 0.4 -1 0 0 0 10 0 +75 0.5 -1 0 0 0 10 0 +76 0.6 -1 0 0 0 10 0 +77 0.7 -1 0 0 0 10 0 +78 0.8 -1 0 0 0 10 0 +79 0.9 -1 0 0 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -1 -2 -0 -0 -100 -0 -0 +101 -0.9 -2 -0 -0 -100 -0 -0 +102 -0.8 -2 -0 -0 -100 -0 -0 +103 -0.7 -2 -0 -0 -100 -0 -0 +104 -0.6 -2 -0 -0 -100 -0 -0 +105 -0.5 -2 -0 -0 -100 -0 -0 +106 -0.4 -2 -0 -0 -100 -0 -0 +107 -0.3 -2 -0 -0 -100 -0 -0 +108 -0.2 -2 -0 -0 -100 -0 -0 +109 -0.1 -2 -0 -0 -100 -0 -0 +110 0 -2 0 0 0 0 0 +111 0.1 -2 0 0 0 100 0 +112 0.2 -2 0 0 0 100 0 +113 0.3 -2 0 0 0 100 0 +114 0.4 -2 0 0 0 100 0 +115 0.5 -2 0 0 0 100 0 +116 0.6 -2 0 0 0 100 0 +117 0.7 -2 0 0 0 100 0 +118 0.8 -2 0 0 0 100 0 +119 0.9 -2 0 0 0 100 0 +200 12345.6789 0 12346 12346 12345 12346 12345 +201 12345.6789 -1 12350 12350 12340 12350 12340 +202 12345.6789 -2 12300 12300 12300 12400 12300 +203 12345.6789 -3 12000 12000 12000 13000 12000 +204 12345.6789 -4 10000 10000 10000 20000 10000 +205 12345.6789 -5 0 0 0 100000 0 +206 12345.6789 -6 0 0 0 1000000 0 +207 12345.6789 -7 0 0 0 10000000 0 +208 12345.6789 -8 0 0 0 100000000 0 +209 12345.6789 -9 0 0 0 1000000000 0 +210 12345.6789 0 12346 12346 12345 12346 12345 +211 12345.6789 -1 12350 12350 12340 12350 12340 +212 12345.6789 -2 12300 12300 12300 12400 12300 +213 12345.6789 -3 12000 12000 12000 13000 12000 +214 12345.6789 -4 10000 10000 10000 20000 10000 +215 12345.6789 -5 0 0 0 100000 0 +216 12345.6789 -6 0 0 0 1000000 0 +217 12345.6789 -7 0 0 0 10000000 0 +218 12345.6789 -8 0 0 0 100000000 0 +219 12345.6789 -9 0 0 0 1000000000 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +CHECKPOINT2 +1 42.42 42.42 42.42 42.42 +2 0.0084 0.0084 0.0084 0.0084 +3 6.513 6.513 6.513 6.513 +4 115.6011 115.6011 115.6011 115.6011 +5 1.6029 1.6029 1.6029 1.6029 +6 -0.9999 -0.9999 -0.9999 -0.9999 +7 0.0084 0.0084 0.0084 0.0084 +8 3.7476 3.7476 3.7476 3.7476 +9 5.4066 5.4066 5.4066 5.4066 +10 1.6275 1.6275 1.6275 1.6275 +1 +1 +1 diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.sql b/tests/queries/0_stateless/03165_round_scale_as_column.sql new file mode 100644 index 00000000000..60940b22fc1 --- /dev/null +++ b/tests/queries/0_stateless/03165_round_scale_as_column.sql @@ -0,0 +1,125 @@ +-- Regression test that functions round(), roundBankers(), floor(), ceil() and trunc() work with default 'scale' argument (= the 2nd argument) +SELECT toUInt8(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt8(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; + +SELECT toFloat32((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat64((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; + +-- Test that functions round(), roundBankers(), floor(), ceil() and trunc() accept non-const 'scale' argument +SELECT toFloat32((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; +SELECT toFloat64((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; +SELECT toUInt8(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; + +SELECT toInt8(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toFloat32(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; + +SELECT toUInt8(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt8(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toFloat32(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; + +SELECT toString('CHECKPOINT1'); + +DROP TABLE IF EXISTS tround; + +CREATE TABLE tround ( + id Int32, + scale Int16, + u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, + i8 Int8, i16 Int16, i32 Int32, i64 Int64, + f32 Float32, f64 Float64 +) ENGINE = Memory; + +INSERT INTO tround SELECT number , 0, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+20 , 0, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+40 , -1, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+60 , -1, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+80 , -2, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+100, -2, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; + +INSERT INTO tround SELECT number+200, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; +INSERT INTO tround SELECT number+210, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; + +INSERT INTO tround VALUES (300, 4, 2, 2, 2, 2, 2, 2, 2, 2, 2.0, 2.0); +INSERT INTO tround VALUES (301, 4, 20, 20, 20, 20, 20, 20, 20, 20, 20.0, 20.0); +INSERT INTO tround VALUES (302, 4, 200, 200, 200, 200, 200, 200, 200, 200, 200.0, 200.0); +INSERT INTO tround VALUES (303, 4, 5, 5, 5, 5, 5, 5, 5, 5, 5.0, 5.0); +INSERT INTO tround VALUES (304, 4, 50, 50, 50, 50, 50, 50, 50, 50, 50.0, 50.0); +INSERT INTO tround VALUES (305, 4, 500, 500, 500, 500, 500, 500, 500, 500, 500.0, 500.0); + +SELECT toString('id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, f32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, f64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; + +DROP TABLE IF EXISTS tround; +-- +SELECT toString('CHECKPOINT2'); + +DROP TABLE IF EXISTS tround2; + +CREATE TABLE tround2 ( + id Int32, + scale Int16, + d32 Decimal32(4), d64 Decimal64(4), d128 Decimal128(4), d256 Decimal256(4) +) ENGINE = Memory; + +INSERT INTO tround2 VALUES (1, 6, toDecimal32('42.42', 4), toDecimal64('42.42', 4), toDecimal128('42.42', 4), toDecimal256('42.42', 4)); +INSERT INTO tround2 SELECT 2 , 6, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 3 , 6, sqrt(d32), sqrt(d64), sqrt(d128), sqrt(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 4 , 6, lgamma(d32), lgamma(d64), lgamma(d128), lgamma(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 5 , 6, tgamma(d32)/1e50, tgamma(d64)/1e50, tgamma(d128)/1e50, tgamma(d256)/1e50 FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 6 , 8, sin(d32), sin(d64), sin(d128), sin(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 7 , 8, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 8 , 8, log(d32), log(d64), log(d128), log(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 9 , 8, log2(d32), log2(d64), log2(d128), log2(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 10, 8, log10(d32), log10(d64), log10(d128), log10(d256) FROM tround2 WHERE id = 1; + +SELECT id, round(d32, scale), round(d64, scale), round(d128, scale), round(d256, scale) FROM tround2 ORDER BY id; + +DROP TABLE IF EXISTS tround2; + +SELECT round(1, 1); +SELECT round(materialize(1), materialize(1)); +SELECT round(1, materialize(1)); --{serverError ILLEGAL_COLUMN} +SELECT round(materialize(1), 1); + + + From ff84232033b7f0421f0516fd38a2bb69d025c8aa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 15:52:27 +0000 Subject: [PATCH 266/856] Fixing some tests. --- .../01655_plan_optimizations.reference | 8 -- ...ct_in_order_optimization_explain.reference | 2 +- ...dicate_push_down_filled_join_fix.reference | 8 +- ...filter_push_down_equivalent_sets.reference | 84 +++++++------- ...convert_outer_join_to_inner_join.reference | 104 ++++++++---------- .../03156_analyzer_array_join_distributed.sql | 18 +++ 6 files changed, 111 insertions(+), 113 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 1b9755a74d5..edc6412c932 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -44,12 +44,10 @@ Filter 9 10 1 > one condition of filter should be pushed down after aggregating, other condition is aliased Filter column -ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased Filter column -ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -63,12 +61,10 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2 Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2 Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -82,12 +78,10 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -100,12 +94,10 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column -ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating Filter column: and(notEquals(y, 0), minus(y, 4)) > (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column -ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8)) Aggregating Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8)) 0 1 diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 85e8a802bdc..9b9885478cd 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -87,7 +87,7 @@ Sorting (Stream): a ASC, b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC -Sorting (Stream): __table1.a ASC, b ASC +Sorting (Stream): a ASC, b ASC -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): __table1.a ASC Sorting (Stream): __table1.a ASC diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference index e6c4d5768af..6e8325a2ff3 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference @@ -24,10 +24,10 @@ Positions: 3 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table) Header: id UInt64 value String diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index 00740e6380f..80f4e309505 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -33,10 +33,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -50,10 +50,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -100,10 +100,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -117,10 +117,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -168,12 +168,12 @@ Positions: 4 2 0 1 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 3 - ALIAS id :: 0 -> __table1.id UInt64 : 4 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 4, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1 - FUNCTION equals(__table1.id : 4, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2 + ALIAS id : 0 -> __table1.id UInt64 : 4 + ALIAS value :: 1 -> __table1.value String : 5 + FUNCTION equals(id : 0, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1 + FUNCTION equals(id :: 0, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2 FUNCTION and(equals(__table1.id, 5_UInt8) :: 2, equals(__table1.id, 6_UInt8) :: 1) -> and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) UInt8 : 3 - Positions: 3 4 0 + Positions: 3 4 5 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -188,12 +188,12 @@ Positions: 4 2 0 1 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 3 - ALIAS id :: 0 -> __table2.id UInt64 : 4 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 4, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - FUNCTION equals(__table2.id : 4, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2 + ALIAS id : 0 -> __table2.id UInt64 : 4 + ALIAS value :: 1 -> __table2.value String : 5 + FUNCTION equals(id : 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + FUNCTION equals(id :: 0, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2 FUNCTION and(equals(__table2.id, 6_UInt8) :: 2, equals(__table2.id, 5_UInt8) :: 1) -> and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) UInt8 : 3 - Positions: 3 4 0 + Positions: 3 4 5 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -237,10 +237,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -254,10 +254,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -452,10 +452,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -469,10 +469,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference index 6ca5b6ef572..d35bdeff98b 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference @@ -25,13 +25,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -42,20 +41,18 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table1.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -66,9 +63,8 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table2.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 -- 2 Value_2 2 Value_2 -- @@ -99,13 +95,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -116,20 +111,18 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table1.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -140,9 +133,8 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table2.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 -- 2 Value_2 2 Value_2 -- @@ -173,13 +165,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -190,22 +181,20 @@ Positions: 4 0 2 1 Prewhere filter column: and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 : 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 4 - FUNCTION and(notEquals(__table1.id, 0_UInt8) :: 4, notEquals(__table1.id, 0_UInt8) :: 3) -> and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) UInt8 : 1 - Positions: 2 0 1 + FUNCTION notEquals(id : 0, 0_UInt8 : 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 + FUNCTION and(notEquals(__table1.id, 0_UInt8) :: 3, notEquals(__table1.id, 0_UInt8) :: 2) -> and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) UInt8 : 1 + Positions: 0 1 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -216,10 +205,9 @@ Positions: 4 0 2 1 Prewhere filter column: and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 : 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 4 - FUNCTION and(notEquals(__table2.id, 0_UInt8) :: 4, notEquals(__table2.id, 0_UInt8) :: 3) -> and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) UInt8 : 1 - Positions: 2 0 1 + FUNCTION notEquals(id : 0, 0_UInt8 : 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 + FUNCTION and(notEquals(__table2.id, 0_UInt8) :: 3, notEquals(__table2.id, 0_UInt8) :: 2) -> and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) UInt8 : 1 + Positions: 0 1 -- 2 Value_2 2 Value_2 diff --git a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql index f605a369822..f4ebba0b1b2 100644 --- a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql +++ b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql @@ -8,3 +8,21 @@ SELECT s, arr, a FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) AR SELECT s, arr FROM remote('127.0.0.2', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; SELECT s, arr FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; + +create table hourly( + hour datetime, + `metric.names` Array(String), + `metric.values` Array(Int64) +) Engine=Memory +as select '2020-01-01', ['a', 'b'], [1,2]; + +SELECT + toDate(hour) AS day, + `metric.names`, + sum(`metric.values`) +FROM remote('127.0.0.{1,2}', currentDatabase(), hourly) +ARRAY JOIN metric +GROUP BY + day, + metric.names; +ORDER BY metric.names; From 2db0eddc0f8df5d5600f9ccb10fcc2c65ca0b114 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 16:11:45 +0000 Subject: [PATCH 267/856] Update tests. --- .../01655_plan_optimizations.reference | 8 ++++++++ .../0_stateless/01655_plan_optimizations.sh | 16 ++++++++-------- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index edc6412c932..edf93b4b39f 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -44,10 +44,12 @@ Filter 9 10 1 > one condition of filter should be pushed down after aggregating, other condition is aliased Filter column +ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2 Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased Filter column +ALIAS notEquals(__table1.s, 4_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -61,10 +63,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column +FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted Filter column +FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -78,10 +82,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column +FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column +FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -94,10 +100,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column +ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating Filter column: and(notEquals(y, 0), minus(y, 4)) > (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column +ALIAS notEquals(__table1.s, 8_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8)) Aggregating Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8)) 0 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 864dd69412a..4bd0eb7d908 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -49,14 +49,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -69,14 +69,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4))" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -89,14 +89,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -109,14 +109,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" echo "> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y From 8cb846c28b9fff0571140e31dd8a7dedf5563bc8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 16:12:59 +0000 Subject: [PATCH 268/856] Revert test. --- .../03156_analyzer_array_join_distributed.sql | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql index f4ebba0b1b2..f605a369822 100644 --- a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql +++ b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql @@ -8,21 +8,3 @@ SELECT s, arr, a FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) AR SELECT s, arr FROM remote('127.0.0.2', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; SELECT s, arr FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; - -create table hourly( - hour datetime, - `metric.names` Array(String), - `metric.values` Array(Int64) -) Engine=Memory -as select '2020-01-01', ['a', 'b'], [1,2]; - -SELECT - toDate(hour) AS day, - `metric.names`, - sum(`metric.values`) -FROM remote('127.0.0.{1,2}', currentDatabase(), hourly) -ARRAY JOIN metric -GROUP BY - day, - metric.names; -ORDER BY metric.names; From 1d77cda70b2db1041a89f7bf7537e96795084dae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 17:13:19 +0000 Subject: [PATCH 269/856] Fix distributed array join by nested --- src/Analyzer/ArrayJoinNode.cpp | 8 +++++++- ...6_analyzer_array_join_distributed.reference | 2 ++ .../03156_analyzer_array_join_distributed.sql | 18 ++++++++++++++++++ 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 27d7229d46a..0cfb5d80b2a 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -64,7 +65,12 @@ ASTPtr ArrayJoinNode::toASTImpl(const ConvertToASTOptions & options) const auto * column_node = array_join_expression->as(); if (column_node && column_node->getExpression()) - array_join_expression_ast = column_node->getExpression()->toAST(options); + { + if (const auto * function_node = column_node->getExpression()->as(); function_node && function_node->getFunctionName() == "nested") + array_join_expression_ast = array_join_expression->toAST(options); + else + array_join_expression_ast = column_node->getExpression()->toAST(options); + } else array_join_expression_ast = array_join_expression->toAST(options); diff --git a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.reference b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.reference index b5b2aec9c12..18830a293bd 100644 --- a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.reference +++ b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.reference @@ -10,3 +10,5 @@ Hello 1 Hello 1 Hello 2 Hello 2 +2020-01-01 a 2 +2020-01-01 b 4 diff --git a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql index f605a369822..55f9877b2ac 100644 --- a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql +++ b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql @@ -8,3 +8,21 @@ SELECT s, arr, a FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) AR SELECT s, arr FROM remote('127.0.0.2', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; SELECT s, arr FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; + +create table hourly( + hour datetime, + `metric.names` Array(String), + `metric.values` Array(Int64) +) Engine=Memory +as select '2020-01-01', ['a', 'b'], [1,2]; + +SELECT + toDate(hour) AS day, + `metric.names`, + sum(`metric.values`) +FROM remote('127.0.0.{1,2}', currentDatabase(), hourly) +ARRAY JOIN metric +GROUP BY + day, + metric.names +ORDER BY metric.names; From 3d7beae8fd4140f90917a592f3aae1bfecb90c5a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 4 Jun 2024 19:29:26 +0200 Subject: [PATCH 270/856] Simplify 03023_zeros_generate_random_with_limit_progress_bar --- ...rate_random_with_limit_progress_bar.expect | 49 ------------------- ...e_random_with_limit_progress_bar.reference | 3 ++ ...generate_random_with_limit_progress_bar.sh | 16 ++++++ 3 files changed, 19 insertions(+), 49 deletions(-) delete mode 100755 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect create mode 100755 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect deleted file mode 100755 index de15a199132..00000000000 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/expect -f - -set basedir [file dirname $argv0] -set basename [file tail $argv0] -if {[info exists env(CLICKHOUSE_TMP)]} { - set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP) -} else { - set CLICKHOUSE_TMP "." -} -exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 - -log_user 0 -set timeout 60 -match_max 100000 -set stty_init "rows 25 cols 120" - -expect_after { - -i $any_spawn_id eof { exp_continue } - -i $any_spawn_id timeout { exit 1 } -} - -spawn clickhouse-local -expect ":) " - -# Trivial SELECT with LIMIT from system.zeros shows progress bar. -send "SELECT * FROM system.zeros LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -send "SELECT * FROM system.zeros_mt LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -# As well as from generateRandom -send "SELECT * FROM generateRandom() LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -send "exit\r" -expect eof diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference index e69de29bb2d..6ca5ae94f9a 100644 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference @@ -0,0 +1,3 @@ +Matched +Matched +Matched diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh new file mode 100755 index 00000000000..4bb8fc8880d --- /dev/null +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function run_with_progress_and_match_total_rows() +{ + echo "$1" | \ + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1 | \ + grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found" +} + +run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros LIMIT 100' +run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros_mt LIMIT 100' +run_with_progress_and_match_total_rows 'SELECT * FROM generateRandom() LIMIT 100' From c769b3bda056b50a1822ed57ab0299f46ffad58f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 4 Jun 2024 15:04:14 -0300 Subject: [PATCH 271/856] fix conflicts and adapt --- src/Common/proxyConfigurationToPocoProxyConfig.cpp | 3 +-- tests/integration/helpers/s3_url_proxy_tests_util.py | 9 +++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.cpp b/src/Common/proxyConfigurationToPocoProxyConfig.cpp index c221dd394ca..fbd9a229da3 100644 --- a/src/Common/proxyConfigurationToPocoProxyConfig.cpp +++ b/src/Common/proxyConfigurationToPocoProxyConfig.cpp @@ -1,8 +1,7 @@ #include -#include -#include +#include #include #pragma clang diagnostic push diff --git a/tests/integration/helpers/s3_url_proxy_tests_util.py b/tests/integration/helpers/s3_url_proxy_tests_util.py index 7a57c006689..02603bbf879 100644 --- a/tests/integration/helpers/s3_url_proxy_tests_util.py +++ b/tests/integration/helpers/s3_url_proxy_tests_util.py @@ -124,3 +124,12 @@ def simple_storage_test(cluster, node, proxies, policy): # not checking for POST because it is in a different format check_proxy_logs(cluster, proxies, "http", policy, ["PUT", "GET"]) + +def simple_test_assert_no_proxy(cluster, proxies, protocol, bucket): + minio_endpoint = build_s3_endpoint(protocol, bucket) + node = cluster.instances[bucket] + perform_simple_queries(node, minio_endpoint) + + # No HTTP method should be found in proxy logs if no proxy is active + empty_method_list = [] + check_proxy_logs(cluster, proxies, protocol, bucket, empty_method_list) From 553fcb5e0618858cfa7659522208be49320b0e48 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 4 Jun 2024 20:05:58 +0200 Subject: [PATCH 272/856] Update test.py --- tests/integration/test_replicated_database/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index a5859960cd9..ea569939c1c 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1534,7 +1534,7 @@ def test_all_groups_cluster(started_cluster): "CREATE DATABASE db_cluster ENGINE = Replicated('/clickhouse/databases/all_groups_cluster', 'shard1', 'replica2');" ) - assert "bad_settings_node\ndummy_node\n" == dummy_node.query( + assert "dummy_node\n" == dummy_node.query( "select host_name from system.clusters where name='db_cluster' order by host_name" ) assert "bad_settings_node\n" == bad_settings_node.query( From 53b9ea483fc7b9f888a4478db24a4f108da838b3 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 4 Jun 2024 15:23:52 -0300 Subject: [PATCH 273/856] black fix --- tests/integration/helpers/s3_url_proxy_tests_util.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/s3_url_proxy_tests_util.py b/tests/integration/helpers/s3_url_proxy_tests_util.py index 02603bbf879..9a45855acb8 100644 --- a/tests/integration/helpers/s3_url_proxy_tests_util.py +++ b/tests/integration/helpers/s3_url_proxy_tests_util.py @@ -125,6 +125,7 @@ def simple_storage_test(cluster, node, proxies, policy): # not checking for POST because it is in a different format check_proxy_logs(cluster, proxies, "http", policy, ["PUT", "GET"]) + def simple_test_assert_no_proxy(cluster, proxies, protocol, bucket): minio_endpoint = build_s3_endpoint(protocol, bucket) node = cluster.instances[bucket] From 236a565e91fcd9b7a2224f145799852a71bae593 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 4 Jun 2024 15:44:22 -0300 Subject: [PATCH 274/856] add missing no_proxy_hosts assignment --- src/Common/RemoteProxyConfigurationResolver.cpp | 1 + .../integration/test_s3_table_function_with_http_proxy/test.py | 2 -- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/RemoteProxyConfigurationResolver.cpp b/src/Common/RemoteProxyConfigurationResolver.cpp index b95eb958049..33dc0e957db 100644 --- a/src/Common/RemoteProxyConfigurationResolver.cpp +++ b/src/Common/RemoteProxyConfigurationResolver.cpp @@ -95,6 +95,7 @@ ProxyConfiguration RemoteProxyConfigurationResolver::resolve() cached_config.port = proxy_port; cached_config.tunneling = use_tunneling_for_https_requests_over_http_proxy; cached_config.original_request_protocol = request_protocol; + cached_config.no_proxy_hosts = no_proxy_hosts; cache_timestamp = std::chrono::system_clock::now(); cache_valid = true; diff --git a/tests/integration/test_s3_table_function_with_http_proxy/test.py b/tests/integration/test_s3_table_function_with_http_proxy/test.py index 49cf78ebf2b..2ec73ecbef6 100644 --- a/tests/integration/test_s3_table_function_with_http_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_http_proxy/test.py @@ -24,7 +24,6 @@ def cluster(): main_configs=[ "configs/config.d/proxy_remote_no_proxy.xml", ], - instance_env_variables=True, with_minio=True, ) @@ -41,7 +40,6 @@ def cluster(): main_configs=[ "configs/config.d/proxy_list_no_proxy.xml", ], - instance_env_variables=True, with_minio=True, ) From 2fc57af08a51ac6c6127fd162ef059efefacaa14 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 4 Jun 2024 17:32:53 -0300 Subject: [PATCH 275/856] fix ut --- .../tests/gtest_proxy_remote_configuration_resolver.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/tests/gtest_proxy_remote_configuration_resolver.cpp b/src/Common/tests/gtest_proxy_remote_configuration_resolver.cpp index 7068e0f2061..5489a931f24 100644 --- a/src/Common/tests/gtest_proxy_remote_configuration_resolver.cpp +++ b/src/Common/tests/gtest_proxy_remote_configuration_resolver.cpp @@ -42,6 +42,7 @@ TEST(RemoteProxyConfigurationResolver, HTTPOverHTTP) RemoteProxyConfigurationResolver resolver( remote_server_configuration, ProxyConfiguration::Protocol::HTTP, + "", std::make_shared(proxy_server_mock) ); @@ -68,6 +69,7 @@ TEST(RemoteProxyConfigurationResolver, HTTPSOverHTTPS) RemoteProxyConfigurationResolver resolver( remote_server_configuration, ProxyConfiguration::Protocol::HTTPS, + "", std::make_shared(proxy_server_mock) ); @@ -95,6 +97,7 @@ TEST(RemoteProxyConfigurationResolver, HTTPSOverHTTP) RemoteProxyConfigurationResolver resolver( remote_server_configuration, ProxyConfiguration::Protocol::HTTPS, + "", std::make_shared(proxy_server_mock) ); @@ -122,6 +125,7 @@ TEST(RemoteProxyConfigurationResolver, HTTPSOverHTTPNoTunneling) RemoteProxyConfigurationResolver resolver( remote_server_configuration, ProxyConfiguration::Protocol::HTTPS, + "", std::make_shared(proxy_server_mock), true /* disable_tunneling_for_https_requests_over_http_proxy_ */ ); @@ -153,6 +157,7 @@ TEST(RemoteProxyConfigurationResolver, SimpleCacheTest) RemoteProxyConfigurationResolver resolver( remote_server_configuration, ProxyConfiguration::Protocol::HTTP, + "", fetcher_mock ); From 3ae8176a6d094be53ebd82d2fc7b5e1c75ceccd2 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 4 Jun 2024 20:12:28 -0300 Subject: [PATCH 276/856] read from env only once --- .../EnvironmentProxyConfigurationResolver.cpp | 22 ++++++++++++++----- .../EnvironmentProxyConfigurationResolver.h | 3 +++ 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index 387674feaae..1b7431ee827 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -49,10 +49,8 @@ namespace } } -ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() +ProxyConfiguration EnvironmentProxyConfigurationResolver::buildProxyConfiguration(Protocol protocol, const char * proxy_host, const std::string & no_proxy_hosts_string) { - const auto * proxy_host = getProxyHost(request_protocol); - if (!proxy_host) { return {}; @@ -69,10 +67,22 @@ ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() host, ProxyConfiguration::protocolFromString(scheme), port, - useTunneling(request_protocol, ProxyConfiguration::protocolFromString(scheme), disable_tunneling_for_https_requests_over_http_proxy), - request_protocol, - getNoProxyHostsString() + useTunneling(protocol, ProxyConfiguration::protocolFromString(scheme), disable_tunneling_for_https_requests_over_http_proxy), + protocol, + no_proxy_hosts_string }; } +ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() +{ + static const auto * http_proxy_host = getProxyHost(Protocol::HTTP); + static const auto * https_proxy_host = getProxyHost(Protocol::HTTPS); + static const auto no_proxy_hosts_string = getNoProxyHostsString(); + + static const auto http_proxy_configuration = buildProxyConfiguration(Protocol::HTTP, http_proxy_host, no_proxy_hosts_string); + static const auto https_proxy_configuration = buildProxyConfiguration(Protocol::HTTPS, https_proxy_host, no_proxy_hosts_string); + + return request_protocol == Protocol::HTTP ? http_proxy_configuration : https_proxy_configuration; +} + } diff --git a/src/Common/EnvironmentProxyConfigurationResolver.h b/src/Common/EnvironmentProxyConfigurationResolver.h index 6bc9d8a368c..08f91790162 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.h +++ b/src/Common/EnvironmentProxyConfigurationResolver.h @@ -15,6 +15,9 @@ public: ProxyConfiguration resolve() override; void errorReport(const ProxyConfiguration &) override {} + +private: + ProxyConfiguration buildProxyConfiguration(Protocol protocol, const char * proxy_host, const std::string & no_proxy_hosts); }; } From 09c2151f3b0e2e19a1a1f77e27d3677e95b17fb0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jun 2024 03:08:58 +0200 Subject: [PATCH 277/856] Fix style --- src/Common/UTF8Helpers.cpp | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 006ec33c08b..bfa860af98a 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -103,7 +103,7 @@ template size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept { UTF8Decoder decoder; - int isEscapeSequence = false; + bool is_escape_sequence = false; size_t width = 0; size_t rollback = 0; for (size_t i = 0; i < size; ++i) @@ -116,10 +116,8 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l while (i + 15 < size) { - if (isEscapeSequence) - { + if (is_escape_sequence) break; - } __m128i bytes = _mm_loadu_si128(reinterpret_cast(&data[i])); @@ -145,17 +143,15 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l while (i < size && isPrintableASCII(data[i])) { - auto isParameterByte = isCSIParameterByte(data[i]); - auto isIntermediateByte = isCSIIntermediateByte(data[i]); - auto ignore_width = isEscapeSequence && (isParameterByte || isIntermediateByte); + bool ignore_width = is_escape_sequence && (isCSIParameterByte(data[i]) || isCSIIntermediateByte(data[i])); - if (ignore_width || (data[i] == '[' && isEscapeSequence)) + if (ignore_width || (data[i] == '[' && is_escape_sequence)) { /// don't count the width } - else if (isEscapeSequence && isCSIFinalByte(data[i])) + else if (is_escape_sequence && isCSIFinalByte(data[i])) { - isEscapeSequence = false; + is_escape_sequence = false; } else { From 5b7162df6ee2c3e92c03f02b226b682debee8425 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jun 2024 04:37:59 +0200 Subject: [PATCH 278/856] 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 2a14168075d74062196e259582cdf4007b01da60 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 31 May 2024 16:44:31 +0800 Subject: [PATCH 279/856] improve least/greatest by allow nullable numberic fields compare --- src/Functions/LeastGreatestGeneric.h | 2 +- tests/performance/least_greatest.xml | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/performance/least_greatest.xml diff --git a/src/Functions/LeastGreatestGeneric.h b/src/Functions/LeastGreatestGeneric.h index 9073f14d679..07efb1904eb 100644 --- a/src/Functions/LeastGreatestGeneric.h +++ b/src/Functions/LeastGreatestGeneric.h @@ -111,7 +111,7 @@ public: argument_types.push_back(argument.type); /// More efficient specialization for two numeric arguments. - if (arguments.size() == 2 && isNumber(arguments[0].type) && isNumber(arguments[1].type)) + if (arguments.size() == 2 && isNumber(removeNullable(arguments[0].type)) && isNumber(removeNullable(arguments[1].type))) return std::make_unique(SpecializedFunction::create(context), argument_types, return_type); return std::make_unique( diff --git a/tests/performance/least_greatest.xml b/tests/performance/least_greatest.xml new file mode 100644 index 00000000000..522bcd9004d --- /dev/null +++ b/tests/performance/least_greatest.xml @@ -0,0 +1,10 @@ + + CREATE TABLE test (id Int32, x1 Nullable(Int32), x2 Nullable(Float32)) ENGINE = MergeTree() ORDER BY id + + INSERT INTO test SELECT number, number+1, number + 2 FROM numbers(1000000) + + SELECT COUNT(1) FROM test WHERE least(x1, x2) > 1 + SELECT COUNT(1) FROM test WHERE GREATEST(x1, x2) > 1 + + DROP TABLE IF EXISTS test + \ No newline at end of file From 2c28cf80595592b952dfa8c13e173b92b654089f Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 5 Jun 2024 15:46:59 +0800 Subject: [PATCH 280/856] review fix --- src/Functions/LeastGreatestGeneric.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/LeastGreatestGeneric.h b/src/Functions/LeastGreatestGeneric.h index 07efb1904eb..bbab001b00d 100644 --- a/src/Functions/LeastGreatestGeneric.h +++ b/src/Functions/LeastGreatestGeneric.h @@ -123,7 +123,7 @@ public: if (types.empty()) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} cannot be called without arguments", getName()); - if (types.size() == 2 && isNumber(types[0]) && isNumber(types[1])) + if (types.size() == 2 && isNumber(removeNullable(types[0])) && isNumber(removeNullable(types[1]))) return SpecializedFunction::create(context)->getReturnTypeImpl(types); return getLeastSupertype(types); From 80d5d146cc6569b3268750f7980e765b20b95210 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 5 Jun 2024 08:04:00 +0000 Subject: [PATCH 281/856] Flaky test: try to wait longer --- tests/queries/0_stateless/02530_dictionaries_update_field.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.sh b/tests/queries/0_stateless/02530_dictionaries_update_field.sh index 35881bdf896..b502f3e141e 100755 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.sh +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.sh @@ -60,7 +60,7 @@ for layout in "${layouts[@]}"; do INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); - SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; + SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM $dictionary_name ORDER BY key ASC; -- { echoOff } From a13bf252683670c5db4ce4eb62ab19008e463a52 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 5 Jun 2024 10:26:56 +0200 Subject: [PATCH 282/856] Trigger CI From a0acea74c08b293e093bcd894fa123f437c96159 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 5 Jun 2024 10:36:48 +0200 Subject: [PATCH 283/856] Cleanup --- src/Common/GWPAsan.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 4bda2f7e913..2bdf418e152 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -54,13 +54,8 @@ static bool guarded_alloc_initialized = [] auto & opts = gwp_asan::options::getOptions(); opts.Backtrace = getBackTrace; - opts.MaxSimultaneousAllocations = 256; /// for testing GuardedAlloc.init(opts); - ///std::cerr << "GwpAsan is initialized, the options are { Enabled: " << opts.Enabled - /// << ", MaxSimultaneousAllocations: " << opts.MaxSimultaneousAllocations - /// << ", SampleRate: " << opts.SampleRate << " }\n"; - return true; }(); From 5f3bc4271f6a0fe87a3cd2b9d1e694a88639ef2a Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 5 Jun 2024 10:58:30 +0200 Subject: [PATCH 284/856] rename forgoten isEscapeSequence to is_escape_sequence --- src/Common/UTF8Helpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index bfa860af98a..dd24cb20933 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -184,7 +184,7 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l // special treatment for '\t' and for ESC size_t next_width = width; if (decoder.codepoint == '\x1b') - isEscapeSequence = true; + is_escape_sequence = true; else if (decoder.codepoint == '\t') next_width += 8 - (prefix + width) % 8; else From c108c51f6459af8aa9d8b497d2a578be5fbf8ed1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Jun 2024 10:01:55 +0000 Subject: [PATCH 285/856] Fixing test. --- .../0_stateless/01822_union_and_constans_error.reference | 1 - tests/queries/0_stateless/01822_union_and_constans_error.sql | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01822_union_and_constans_error.reference b/tests/queries/0_stateless/01822_union_and_constans_error.reference index d00491fd7e5..e69de29bb2d 100644 --- a/tests/queries/0_stateless/01822_union_and_constans_error.reference +++ b/tests/queries/0_stateless/01822_union_and_constans_error.reference @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/01822_union_and_constans_error.sql b/tests/queries/0_stateless/01822_union_and_constans_error.sql index 38b7df700cd..9017e8769eb 100644 --- a/tests/queries/0_stateless/01822_union_and_constans_error.sql +++ b/tests/queries/0_stateless/01822_union_and_constans_error.sql @@ -15,6 +15,6 @@ SELECT isNull(t0.c0) OR COUNT('\n?pVa') FROM t0 GROUP BY t0.c0 HAVING isNull(isNull(t0.c0)) -SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0; +SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0 format Null; drop table if exists t0; From 2ba37c53c28b1a90da640033c083e8857c7477f5 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 5 Jun 2024 10:15:06 +0000 Subject: [PATCH 286/856] Fix test --- tests/queries/0_stateless/02530_dictionaries_update_field.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.sh b/tests/queries/0_stateless/02530_dictionaries_update_field.sh index b502f3e141e..9ced78a1196 100755 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.sh +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.sh @@ -60,7 +60,7 @@ for layout in "${layouts[@]}"; do INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); - SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; + SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM $dictionary_name ORDER BY key ASC; -- { echoOff } From 7089d0d84d90e36eeb2b037f3b4eb1b40fa4f56f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Jun 2024 11:33:35 +0000 Subject: [PATCH 287/856] 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 a51577ce8a3ae839e38243247c37ab6a7b72e7e4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 5 Jun 2024 13:42:39 +0200 Subject: [PATCH 288/856] Disable 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 55d8a8f0ec7..fff7b7f7e21 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -146,7 +146,7 @@ namespace DB M(UInt64, global_profiler_real_time_period_ns, 0, "Period for real clock timer of global profiler (in nanoseconds). Set 0 value to turn off the real clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, global_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of global profiler (in nanoseconds). Set 0 value to turn off the CPU clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \ - M(Double, gwp_asan_force_sample_probability, 0.1, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ + M(Double, gwp_asan_force_sample_probability, 0, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp From 1628c2985d588613d6e1356720e96ec904a15651 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 5 Jun 2024 09:42:26 +0000 Subject: [PATCH 289/856] Some fixups --- .../functions/rounding-functions.md | 160 ++++++++++----- src/Functions/FunctionsRound.h | 191 +++++++++--------- .../03165_round_scale_as_column.sql | 91 ++++----- 3 files changed, 245 insertions(+), 197 deletions(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 55a4b3ce9df..6495a43fc85 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -6,49 +6,90 @@ sidebar_label: Rounding # Rounding Functions -## floor(x\[, N\]) +## floor -Returns the largest round number that is less than or equal to `x`. A round number is a multiple of 1/10N, or the nearest number of the appropriate data type if 1 / 10N isn’t exact. -`N` is an integer, optional parameter. By default it is zero, which means to round to an integer. -`N` may be negative. +Returns the largest rounded number less than or equal `x`. +A rounded number is a multiple of 1 / 10 * N, or the nearest number of the appropriate data type if 1 / 10 * N isn’t exact. -Examples: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` +Integer arguments may be rounded with negative `N` argument, with non-negative `N` the function returns `x`, i.e. does nothing. -`x` is any numeric type. The result is a number of the same type. -For integer arguments, it makes sense to round with a negative `N` value (for non-negative `N`, the function does not do anything). -If rounding causes overflow (for example, floor(-128, -1)), an implementation-specific result is returned. +If rounding causes an overflow (for example, `floor(-128, -1)`), the result is undefined. -## ceil(x\[, N\]), ceiling(x\[, N\]) +**Syntax** -Returns the smallest round number that is greater than or equal to `x`. In every other way, it is the same as the `floor` function (see above). +``` sql +floor(x[, N]) +``` -## trunc(x\[, N\]), truncate(x\[, N\]) +**Parameters** -Returns the round number with largest absolute value that has an absolute value less than or equal to `x`‘s. In every other way, it is the same as the ’floor’ function (see above). +- `x` - The value to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` . [(U)Int*](../data-types/int-uint.md). The default is zero, which means rounding to an integer. Can be negative. + +**Returned value** + +A rounded number of the same type as `x`. + +**Examples** + +Query: + +```sql +SELECT floor(123.45, 1) AS rounded +``` + +Result: + +``` +┌─rounded─┐ +│ 123.4 │ +└─────────┘ +``` + +Query: + +```sql +SELECT floor(123.45, -1) +``` + +Result: + +``` +┌─rounded─┐ +│ 120 │ +└─────────┘ +``` + +## ceiling + +Like `floor` but returns the smallest rounded number greater than or equal `x`. + +**Syntax** + +``` sql +ceiling(x[, N]) +``` + +Alias: `ceil` + +## truncate + +Like `floor` but returns the rounded number with largest absolute value that has an absolute value less than or equal to `x`‘s. **Syntax** ```sql -trunc(input, precision) +truncate(x[, N]) ``` -Alias: `truncate`. - -**Parameters** - -- `input`: A numeric type ([Float](../data-types/float.md), [Decimal](../data-types/decimal.md) or [Integer](../data-types/int-uint.md)). -- `precision`: An [Integer](../data-types/int-uint.md) type. - -**Returned value** - -- A data type of `input`. +Alias: `trunc`. **Example** Query: ```sql -SELECT trunc(123.499, 1) as res; +SELECT truncate(123.499, 1) as res; ``` ```response @@ -57,37 +98,40 @@ SELECT trunc(123.499, 1) as res; └───────┘ ``` -## round(x\[, N\]) +## round Rounds a value to a specified number of decimal places. -The function returns the nearest number of the specified order. In case when given number has equal distance to surrounding numbers, the function uses banker’s rounding for float number types and rounds away from zero for the other number types (Decimal). +The function returns the nearest number of the specified order. +If the input value has equal distance to two neighboring numbers, the function uses banker’s rounding for [Float*](../data-types/float.md) inputs and rounds away from zero for the other number types ([Decimal*](../data-types/decimal.md). + +**Syntax** ``` sql -round(expression [, decimal_places]) +round(x[, N]) ``` **Arguments** -- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types). -- `decimal-places` — The number of decimal places to round to. - - If `decimal-places > 0` then the function rounds the value to the right of the decimal point. - - If `decimal-places < 0` then the function rounds the value to the left of the decimal point. - - If `decimal-places = 0` then the function rounds the value to integer. In this case the argument can be omitted. +- `x` — A number to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` — The number of decimal places to round to. Integer. Defaults to `0`. + - If `N > 0`, the function rounds to the right of the decimal point. + - If `N < 0`, the function rounds to the left of the decimal point. + - If `N = 0`, the function rounds to the next integer. **Returned value:** -The rounded number of the same type as the input number. +A rounded number of the same type as `x`. **Examples** -Example of usage with Float: +Example with `Float` inputs: -``` sql +```sql SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; ``` -``` text +``` ┌───x─┬─round(divide(number, 2))─┐ │ 0 │ 0 │ │ 0.5 │ 0 │ @@ -95,13 +139,13 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; └─────┴──────────────────────────┘ ``` -Example of usage with Decimal: +Example with `Decimal` inputs: -``` sql +```sql SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3; ``` -``` text +``` ┌───x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐ │ 0 │ 0 │ │ 0.5 │ 1 │ @@ -109,14 +153,14 @@ SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIM └─────┴──────────────────────────────────────────────────┘ ``` -If you want to keep the trailing zeros, you need to enable `output_format_decimal_trailing_zeros` +To retain trailing zeros, enable setting `output_format_decimal_trailing_zeros`: -``` sql +```sql SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3 settings output_format_decimal_trailing_zeros=1; ``` -``` text +``` ┌──────x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐ │ 0.0000 │ 0.0000 │ │ 0.5000 │ 1.0000 │ @@ -151,9 +195,15 @@ round(3.65, 1) = 3.6 Rounds a number to a specified decimal position. -- If the rounding number is halfway between two numbers, the function uses banker’s rounding. Banker's rounding is a method of rounding fractional numbers. When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. For example: 3.5 rounds up to 4, 2.5 rounds down to 2. It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`. +If the rounding number is halfway between two numbers, the function uses banker’s rounding. +Banker's rounding is a method of rounding fractional numbers +When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. +For example: 3.5 rounds up to 4, 2.5 rounds down to 2. +It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). +The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. +The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`. -- In other cases, the function rounds numbers to the nearest integer. +In other cases, the function rounds numbers to the nearest integer. Using banker’s rounding, you can reduce the effect that rounding numbers has on the results of summing or subtracting these numbers. @@ -166,16 +216,20 @@ For example, sum numbers 1.5, 2.5, 3.5, 4.5 with different rounding: **Syntax** ``` sql -roundBankers(expression [, decimal_places]) +roundBankers(x [, N]) ``` **Arguments** -- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types). -- `decimal-places` — Decimal places. An integer number. - - `decimal-places > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`. - - `decimal-places < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`. - - `decimal-places = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`. + - `N > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`. + - `N < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`. + - `N = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`. + +- `x` — A number to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` — The number of decimal places to round to. Integer. Defaults to `0`. + - If `N > 0`, the function rounds to the right of the decimal point. + - If `N < 0`, the function rounds to the left of the decimal point. + - If `N = 0`, the function rounds to the next integer. **Returned value** @@ -185,13 +239,13 @@ A value rounded by the banker’s rounding method. Query: -``` sql +```sql SELECT number / 2 AS x, roundBankers(x, 0) AS b fROM system.numbers limit 10 ``` Result: -``` text +``` ┌───x─┬─b─┐ │ 0 │ 0 │ │ 0.5 │ 0 │ @@ -208,7 +262,7 @@ Result: Examples of Banker’s rounding: -``` text +``` roundBankers(0.4) = 0 roundBankers(-3.5) = -4 roundBankers(4.5) = 4 diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 978b6d88d05..d6eabdb6e88 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -40,26 +40,22 @@ namespace ErrorCodes } -/** Rounding Functions: - * round(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for floating point numbers. - * roundBankers(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for all numbers. - * floor(x, N) is the largest number <= x (N = 0 by default). - * ceil(x, N) is the smallest number >= x (N = 0 by default). - * trunc(x, N) - is the largest by absolute value number that is not greater than x by absolute value (N = 0 by default). - * - * The value of the parameter N (scale): - * - N > 0: round to the number with N decimal places after the decimal point - * - N < 0: round to an integer with N zero characters - * - N = 0: round to an integer - * - * Type of the result is the type of argument. - * For integer arguments, when passing negative scale, overflow can occur. - * In that case, the behavior is implementation specific. - */ +/// Rounding Functions: +/// - round(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for floating point numbers. +/// - roundBankers(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for all numbers. +/// - floor(x, N) is the largest number <= x (N = 0 by default). +/// - ceil(x, N) is the smallest number >= x (N = 0 by default). +/// - trunc(x, N) - is the largest by absolute value number that is not greater than x by absolute value (N = 0 by default). +/// The value of the parameter N (scale): +/// - N > 0: round to the number with N decimal places after the decimal point +/// - N < 0: round to an integer with N zero characters +/// - N = 0: round to an integer -/** This parameter controls the behavior of the rounding functions. - */ +/// Type of the result is the type of argument. +/// For integer arguments, when passing negative scale, overflow can occur. In that case, the behavior is undefined. + +/// Controls the behavior of the rounding functions. enum class ScaleMode : uint8_t { Positive, // round to a number with N decimal places after the decimal point @@ -75,7 +71,7 @@ enum class RoundingMode : uint8_t Ceil = _MM_FROUND_TO_POS_INF | _MM_FROUND_NO_EXC, Trunc = _MM_FROUND_TO_ZERO | _MM_FROUND_NO_EXC, #else - Round = 8, /// Values are correspond to above just in case. + Round = 8, /// Values correspond to above values, just in case. Floor = 9, Ceil = 10, Trunc = 11, @@ -84,16 +80,21 @@ enum class RoundingMode : uint8_t enum class TieBreakingMode : uint8_t { - Auto, // use banker's rounding for floating point numbers, round up otherwise - Bankers, // use banker's rounding + Auto, /// banker's rounding for floating point numbers, round up otherwise + Bankers, /// banker's rounding +}; + +enum class Vectorize : uint8_t +{ + No, + Yes }; /// For N, no more than the number of digits in the largest type. using Scale = Int16; -/** Rounding functions for integer values. - */ +/// Rounding functions for integer values. template struct IntegerRoundingComputation { @@ -150,7 +151,7 @@ struct IntegerRoundingComputation } } - UNREACHABLE(); + std::unreachable(); } static ALWAYS_INLINE T compute(T x, T scale) @@ -164,10 +165,11 @@ struct IntegerRoundingComputation return computeImpl(x, scale); } - UNREACHABLE(); + std::unreachable(); } - static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) requires std::integral + static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) + requires std::integral { if constexpr (sizeof(T) <= sizeof(scale) && scale_mode == ScaleMode::Negative) { @@ -180,20 +182,23 @@ struct IntegerRoundingComputation *out = compute(*in, static_cast(scale)); } - static ALWAYS_INLINE void compute(const T * __restrict in, T scale, T * __restrict out) requires(!std::integral) + static ALWAYS_INLINE void compute(const T * __restrict in, T scale, T * __restrict out) + requires(!std::integral) { *out = compute(*in, scale); } }; -template -class BaseFloatRoundingComputation; +template +class FloatRoundingComputationBase; #ifdef __SSE4_1__ +/// Vectorized implementation for x86. + template <> -class BaseFloatRoundingComputation +class FloatRoundingComputationBase { public: using ScalarType = Float32; @@ -214,7 +219,7 @@ public: }; template <> -class BaseFloatRoundingComputation +class FloatRoundingComputationBase { public: using ScalarType = Float64; @@ -236,7 +241,7 @@ public: #endif -/// Implementation for ARM/columnar scale argument. Not vectorized. +/// Sequential implementation for ARM. Also used for scalar arguments. inline float roundWithMode(float x, RoundingMode mode) { @@ -248,7 +253,7 @@ inline float roundWithMode(float x, RoundingMode mode) case RoundingMode::Trunc: return truncf(x); } - UNREACHABLE(); + std::unreachable(); } inline double roundWithMode(double x, RoundingMode mode) @@ -261,11 +266,11 @@ inline double roundWithMode(double x, RoundingMode mode) case RoundingMode::Trunc: return trunc(x); } - UNREACHABLE(); + std::unreachable(); } template -class BaseFloatRoundingComputation +class FloatRoundingComputationBase { public: using ScalarType = T; @@ -288,10 +293,10 @@ public: /** Implementation of low-level round-off functions for floating-point values. */ -template -class FloatRoundingComputation : public BaseFloatRoundingComputation +template +class FloatRoundingComputation : public FloatRoundingComputationBase { - using Base = BaseFloatRoundingComputation; + using Base = FloatRoundingComputationBase; public: static void compute(const T * __restrict in, const typename Base::VectorType & scale, T * __restrict out) @@ -323,14 +328,14 @@ struct FloatRoundingImpl private: static_assert(!is_decimal); - template - using Op = FloatRoundingComputation; + using Op = FloatRoundingComputation; using Data = std::array::data_count>; using ColumnType = ColumnVector; using Container = typename ColumnType::Container; @@ -370,7 +375,7 @@ public: static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) { - using ScalarOp = Op; + using ScalarOp = Op; auto s = ScalarOp::prepare(scale); ScalarOp::compute(in, s, out); } @@ -485,8 +490,7 @@ public: } }; -/** Select the appropriate processing algorithm depending on the scale. - */ +/// Select the appropriate processing algorithm depending on the scale. inline void validateScale(Int64 scale64) { if (scale64 > std::numeric_limits::max() || scale64 < std::numeric_limits::min()) @@ -495,7 +499,7 @@ inline void validateScale(Int64 scale64) inline Scale getScaleArg(const ColumnConst* scale_col) { - const auto& scale_field = scale_col->getField(); + const auto & scale_field = scale_col->getField(); Int64 scale64 = scale_field.get(); validateScale(scale64); @@ -503,6 +507,7 @@ inline Scale getScaleArg(const ColumnConst* scale_col) return scale64; } +/// Generic dispatcher template struct Dispatcher { @@ -512,44 +517,44 @@ struct Dispatcher IntegerRoundingImpl>; template - static ColumnPtr apply(const IColumn * data_col, const IColumn * scale_col = nullptr) + static ColumnPtr apply(const IColumn * value_col, const IColumn * scale_col = nullptr) { - const auto & col = checkAndGetColumn>(*data_col); + const auto & value_col_typed = checkAndGetColumn>(*value_col); auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col.getData().size()); + vec_res.resize(value_col_typed.getData().size()); if (!vec_res.empty()) { - using ColVecScale = ColumnVector; if (scale_col == nullptr || isColumnConst(*scale_col)) { - auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst(scale_col)); + auto scale_arg = (scale_col == nullptr) ? 0 : getScaleArg(checkAndGetColumnConst>(scale_col)); if (scale_arg == 0) { size_t scale = 1; - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + FunctionRoundingImpl::apply(value_col_typed.getData(), scale, vec_res); } else if (scale_arg > 0) { size_t scale = intExp10(scale_arg); - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + FunctionRoundingImpl::apply(value_col_typed.getData(), scale, vec_res); } else { size_t scale = intExp10(-scale_arg); - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + FunctionRoundingImpl::apply(value_col_typed.getData(), scale, vec_res); } } - else if (const auto scale_typed = checkAndGetColumn(scale_col)) + /// Non-const scale argument: + else if (const auto * scale_col_typed = checkAndGetColumn>(scale_col)) { - const auto & in = col.getData(); - const auto & scale_data = scale_typed->getData(); - const size_t count = in.size(); + const auto & value_data = value_col_typed.getData(); + const auto & scale_data = scale_col_typed->getData(); + const size_t rows = value_data.size(); - const T * end_in = in.data() + count; - const T * __restrict p_in = in.data(); + const T * end_in = value_data.data() + rows; + const T * __restrict p_in = value_data.data(); const ScaleType * __restrict p_scale = scale_data.data(); T * __restrict p_out = vec_res.data(); @@ -585,36 +590,37 @@ struct Dispatcher } }; +/// Dispatcher for Decimal inputs template struct Dispatcher { public: template - static ColumnPtr apply(const IColumn * data_col, const IColumn* scale_col = nullptr) + static ColumnPtr apply(const IColumn * value_col, const IColumn* scale_col = nullptr) { - const auto & col = checkAndGetColumn>(*data_col); - const typename ColumnDecimal::Container & vec_src = col.getData(); + const auto & value_col_typed = checkAndGetColumn>(*value_col); + const typename ColumnDecimal::Container & vec_src = value_col_typed.getData(); - auto col_res = ColumnDecimal::create(vec_src.size(), col.getScale()); + auto col_res = ColumnDecimal::create(vec_src.size(), value_col_typed.getScale()); auto & vec_res = col_res->getData(); if (!vec_res.empty()) { - using ColVecScale = ColumnVector; if (scale_col == nullptr || isColumnConst(*scale_col)) { - auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst(scale_col)); - DecimalRoundingImpl::apply(col.getData(), col.getScale(), vec_res, scale_arg); + auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst>(scale_col)); + DecimalRoundingImpl::apply(value_col_typed.getData(), value_col_typed.getScale(), vec_res, scale_arg); } - else if (const auto scale_typed = checkAndGetColumn(scale_col)) + /// Non-cosnt scale argument + else if (const auto * scale_col_typed = checkAndGetColumn>(scale_col)) { - const auto & scale = scale_typed->getData(); - const size_t count = vec_src.size(); + const auto & scale = scale_col_typed->getData(); + const size_t rows = vec_src.size(); using NativeType = typename T::NativeType; const NativeType * __restrict p_in = reinterpret_cast(vec_src.data()); const ScaleType * __restrict p_scale = scale.data(); - const NativeType * end_in = p_in + count; + const NativeType * end_in = p_in + rows; NativeType * __restrict p_out = reinterpret_cast(vec_res.data()); while (p_in < end_in) { @@ -622,7 +628,7 @@ public: validateScale(scale64); Scale raw_scale = scale64; - DecimalRoundingImpl::applyOne(p_in, col.getScale(), p_out, raw_scale); + DecimalRoundingImpl::applyOne(p_in, value_col_typed.getScale(), p_out, raw_scale); ++p_in; ++p_scale; ++p_out; @@ -634,9 +640,8 @@ public: } }; -/** A template for functions that round the value of an input parameter of type - * (U)Int8/16/32/64, Float32/64 or Decimal32/64/128, and accept an additional optional parameter (default is 0). - */ +/// Functions that round the value of an input parameter of type (U)Int8/16/32/64, Float32/64 or Decimal32/64/128. +/// Accept an additional optional parameter (0 by default). template class FunctionRounding : public IFunction { @@ -644,16 +649,12 @@ public: static constexpr auto name = Name::name; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } - + String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } - /// Get result types by argument types. If the function does not apply to these arguments, throw an exception. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if ((arguments.empty()) || (arguments.size() > 2)) @@ -669,14 +670,12 @@ public: return arguments[0]; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnWithTypeAndName & data_column = arguments[0]; + const ColumnWithTypeAndName & value_arg = arguments[0]; ColumnPtr res; - auto callData = [&](const auto & types) -> bool + auto call_data = [&](const auto & types) -> bool { using Types = std::decay_t; using DataType = typename Types::RightType; @@ -685,24 +684,24 @@ public: { const ColumnWithTypeAndName & scale_column = arguments[1]; - auto callScale = [&](const auto & scaleTypes) -> bool + auto call_scale = [&](const auto & scaleTypes) -> bool { using ScaleTypes = std::decay_t; using ScaleType = typename ScaleTypes::RightType; - if (isColumnConst(*data_column.column) && !isColumnConst(*scale_column.column)) + if (isColumnConst(*value_arg.column) && !isColumnConst(*scale_column.column)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale column must be const for const data column"); - res = Dispatcher::template apply(data_column.column.get(), scale_column.column.get()); + res = Dispatcher::template apply(value_arg.column.get(), scale_column.column.get()); return true; }; TypeIndex right_index = scale_column.type->getTypeId(); - if (!callOnBasicType(right_index, callScale)) + if (!callOnBasicType(right_index, call_scale)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type"); return true; } - res = Dispatcher::template apply(data_column.column.get()); + res = Dispatcher::template apply(value_arg.column.get()); return true; }; @@ -715,9 +714,9 @@ public: throw Exception(ErrorCodes::CANNOT_SET_ROUNDING_MODE, "Cannot set floating point rounding mode"); #endif - TypeIndex left_index = data_column.type->getTypeId(); - if (!callOnBasicType(left_index, callData)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", data_column.name, getName()); + TypeIndex left_index = value_arg.type->getTypeId(); + if (!callOnBasicType(left_index, call_data)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", value_arg.name, getName()); return res; } @@ -734,9 +733,8 @@ public: }; -/** Rounds down to a number within explicitly specified array. - * If the value is less than the minimal bound - returns the minimal bound. - */ +/// Rounds down to a number within explicitly specified array. +/// If the value is less than the minimal bound - returns the minimal bound. class FunctionRoundDown : public IFunction { public: @@ -744,7 +742,6 @@ public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } - bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.sql b/tests/queries/0_stateless/03165_round_scale_as_column.sql index 60940b22fc1..2304d2730a9 100644 --- a/tests/queries/0_stateless/03165_round_scale_as_column.sql +++ b/tests/queries/0_stateless/03165_round_scale_as_column.sql @@ -1,4 +1,4 @@ --- Regression test that functions round(), roundBankers(), floor(), ceil() and trunc() work with default 'scale' argument (= the 2nd argument) +-- Tests functions round(), roundBankers(), floor(), ceil() and trunc() with default 'scale' argument SELECT toUInt8(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toUInt16(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toUInt32(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; @@ -7,13 +7,13 @@ SELECT toInt8(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), t SELECT toInt16(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toInt32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toInt64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; + SELECT toFloat32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toFloat64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; - SELECT toFloat32((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toFloat64((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; --- Test that functions round(), roundBankers(), floor(), ceil() and trunc() accept non-const 'scale' argument +-- Functions round(), roundBankers(), floor(), ceil() and trunc() accept non-const 'scale' arguments SELECT toFloat32((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; SELECT toFloat64((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; SELECT toUInt8(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; @@ -41,9 +41,9 @@ SELECT toFloat64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, m SELECT toString('CHECKPOINT1'); -DROP TABLE IF EXISTS tround; +DROP TABLE IF EXISTS tab; -CREATE TABLE tround ( +CREATE TABLE tab ( id Int32, scale Int16, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, @@ -51,75 +51,72 @@ CREATE TABLE tround ( f32 Float32, f64 Float64 ) ENGINE = Memory; -INSERT INTO tround SELECT number , 0, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+20 , 0, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+40 , -1, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+60 , -1, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+80 , -2, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+100, -2, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number , 0, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+20 , 0, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+40 , -1, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+60 , -1, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+80 , -2, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+100, -2, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+200, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; -INSERT INTO tround SELECT number+210, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; +INSERT INTO tab SELECT number+200, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; +INSERT INTO tab SELECT number+210, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; -INSERT INTO tround VALUES (300, 4, 2, 2, 2, 2, 2, 2, 2, 2, 2.0, 2.0); -INSERT INTO tround VALUES (301, 4, 20, 20, 20, 20, 20, 20, 20, 20, 20.0, 20.0); -INSERT INTO tround VALUES (302, 4, 200, 200, 200, 200, 200, 200, 200, 200, 200.0, 200.0); -INSERT INTO tround VALUES (303, 4, 5, 5, 5, 5, 5, 5, 5, 5, 5.0, 5.0); -INSERT INTO tround VALUES (304, 4, 50, 50, 50, 50, 50, 50, 50, 50, 50.0, 50.0); -INSERT INTO tround VALUES (305, 4, 500, 500, 500, 500, 500, 500, 500, 500, 500.0, 500.0); +INSERT INTO tab VALUES (300, 4, 2, 2, 2, 2, 2, 2, 2, 2, 2.0, 2.0); +INSERT INTO tab VALUES (301, 4, 20, 20, 20, 20, 20, 20, 20, 20, 20.0, 20.0); +INSERT INTO tab VALUES (302, 4, 200, 200, 200, 200, 200, 200, 200, 200, 200.0, 200.0); +INSERT INTO tab VALUES (303, 4, 5, 5, 5, 5, 5, 5, 5, 5, 5.0, 5.0); +INSERT INTO tab VALUES (304, 4, 50, 50, 50, 50, 50, 50, 50, 50, 50.0, 50.0); +INSERT INTO tab VALUES (305, 4, 500, 500, 500, 500, 500, 500, 500, 500, 500.0, 500.0); SELECT toString('id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, u8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, u8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, u16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, u16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, u32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, u32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, u64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, u64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, i8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, i8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, i16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, i16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, i32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, i32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, i64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, i64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, f32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, f32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, f64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, f64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; -DROP TABLE IF EXISTS tround; +DROP TABLE tab; -- SELECT toString('CHECKPOINT2'); -DROP TABLE IF EXISTS tround2; +DROP TABLE IF EXISTS tab; -CREATE TABLE tround2 ( +CREATE TABLE tab ( id Int32, scale Int16, d32 Decimal32(4), d64 Decimal64(4), d128 Decimal128(4), d256 Decimal256(4) ) ENGINE = Memory; -INSERT INTO tround2 VALUES (1, 6, toDecimal32('42.42', 4), toDecimal64('42.42', 4), toDecimal128('42.42', 4), toDecimal256('42.42', 4)); -INSERT INTO tround2 SELECT 2 , 6, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 3 , 6, sqrt(d32), sqrt(d64), sqrt(d128), sqrt(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 4 , 6, lgamma(d32), lgamma(d64), lgamma(d128), lgamma(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 5 , 6, tgamma(d32)/1e50, tgamma(d64)/1e50, tgamma(d128)/1e50, tgamma(d256)/1e50 FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 6 , 8, sin(d32), sin(d64), sin(d128), sin(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 7 , 8, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 8 , 8, log(d32), log(d64), log(d128), log(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 9 , 8, log2(d32), log2(d64), log2(d128), log2(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 10, 8, log10(d32), log10(d64), log10(d128), log10(d256) FROM tround2 WHERE id = 1; +INSERT INTO tab VALUES (1, 6, toDecimal32('42.42', 4), toDecimal64('42.42', 4), toDecimal128('42.42', 4), toDecimal256('42.42', 4)); +INSERT INTO tab SELECT 2 , 6, cos(d32), cos(d64), cos(d128), cos(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 3 , 6, sqrt(d32), sqrt(d64), sqrt(d128), sqrt(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 4 , 6, lgamma(d32), lgamma(d64), lgamma(d128), lgamma(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 5 , 6, tgamma(d32)/1e50, tgamma(d64)/1e50, tgamma(d128)/1e50, tgamma(d256)/1e50 FROM tab WHERE id = 1; +INSERT INTO tab SELECT 6 , 8, sin(d32), sin(d64), sin(d128), sin(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 7 , 8, cos(d32), cos(d64), cos(d128), cos(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 8 , 8, log(d32), log(d64), log(d128), log(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 9 , 8, log2(d32), log2(d64), log2(d128), log2(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 10, 8, log10(d32), log10(d64), log10(d128), log10(d256) FROM tab WHERE id = 1; -SELECT id, round(d32, scale), round(d64, scale), round(d128, scale), round(d256, scale) FROM tround2 ORDER BY id; +SELECT id, round(d32, scale), round(d64, scale), round(d128, scale), round(d256, scale) FROM tab ORDER BY id; -DROP TABLE IF EXISTS tround2; +DROP TABLE tab; SELECT round(1, 1); SELECT round(materialize(1), materialize(1)); SELECT round(1, materialize(1)); --{serverError ILLEGAL_COLUMN} SELECT round(materialize(1), 1); - - - From 69d23f5e67a13b07b6b29e8c54c9f6e29f86fb9c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 29 Dec 2023 15:02:11 +0100 Subject: [PATCH 290/856] Fix all problems in tests that had been found by flake8 Signed-off-by: Azat Khuzhin --- tests/integration/helpers/hdfs_api.py | 3 +-- tests/integration/test_backup_restore_new/test.py | 2 +- .../test_disallow_concurrency.py | 4 ++-- .../test_convert_ordinary.py | 2 +- .../test_backward_compatibility/test_functions.py | 2 +- .../integration/test_disk_over_web_server/test.py | 4 ++-- tests/integration/test_jbod_balancer/test.py | 2 +- tests/integration/test_jdbc_bridge/test.py | 8 ++++---- .../test_keeper_snapshot_small_distance/test.py | 2 +- tests/integration/test_keeper_snapshots/test.py | 1 - .../test_keeper_three_nodes_start/test.py | 1 - .../test_merge_tree_azure_blob_storage/test.py | 5 +---- .../test.py | 6 ++---- tests/integration/test_scheduler/test.py | 1 + tests/integration/test_storage_hudi/test.py | 2 +- tests/integration/test_storage_iceberg/test.py | 2 +- tests/integration/test_storage_rabbitmq/test.py | 14 ++++++-------- tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_ttl_replicated/test.py | 2 +- 19 files changed, 28 insertions(+), 37 deletions(-) diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 5739496cb50..4e4468fef77 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -110,10 +110,9 @@ class HDFSApi(object): logging.debug( "Stdout:\n{}\n".format(res.stdout.decode("utf-8")) ) - logging.debug("Env:\n{}\n".format(env)) raise Exception( "Command {} return non-zero code {}: {}".format( - args, res.returncode, res.stderr.decode("utf-8") + cmd, res.returncode, res.stderr.decode("utf-8") ) ) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index ef9e536976b..68b8d29f42e 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1474,7 +1474,7 @@ def test_backup_all(exclude_system_log_tables): restore_settings = [] if not exclude_system_log_tables: restore_settings.append("allow_non_empty_tables=true") - restore_command = f"RESTORE ALL FROM {backup_name} {'SETTINGS '+ ', '.join(restore_settings) if restore_settings else ''}" + restore_command = f"RESTORE ALL FROM {backup_name} {'SETTINGS ' + ', '.join(restore_settings) if restore_settings else ''}" session_id = new_session_id() instance.http_query( diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index c9f20333654..cd0f2032559 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -161,13 +161,13 @@ def wait_for_fail_restore(node, restore_id): elif status == "RESTORING": assert_eq_with_retry( node, - f"SELECT status FROM system.backups WHERE id = '{backup_id}'", + f"SELECT status FROM system.backups WHERE id = '{restore_id}'", "RESTORE_FAILED", sleep_time=2, retry_count=50, ) error = node.query( - f"SELECT error FROM system.backups WHERE id == '{backup_id}'" + f"SELECT error FROM system.backups WHERE id == '{restore_id}'" ).rstrip("\n") assert re.search( "Cannot restore the table default.tbl because it already contains some data", diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index b8db4e005a4..f5d0c066600 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -187,7 +187,7 @@ def check_convert_all_dbs_to_atomic(): # 6 tables, MVs contain 2 rows (inner tables does not match regexp) assert "8\t{}\n".format(8 * len("atomic")) == node.query( - "SELECT count(), sum(n) FROM atomic.merge".format(db) + "SELECT count(), sum(n) FROM atomic.merge" ) node.query("DETACH TABLE ordinary.detached PERMANENTLY") diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index 1cf5c3deb81..758dda655da 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -89,7 +89,7 @@ def test_aggregate_states(start_cluster): logging.info("Skipping %s", aggregate_function) skipped += 1 continue - logging.exception("Failed %s", function) + logging.exception("Failed %s", aggregate_function) failed += 1 continue diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 9f43ab73fa3..f4ea7d54571 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -116,7 +116,7 @@ def test_usage(cluster, node_name): (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( - i, uuids[i], i, i + i, uuids[i] ) ) @@ -338,7 +338,7 @@ def test_page_cache(cluster): (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( - i, uuids[i], i, i + i, uuids[i] ) ) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index 69ab83283ff..8635f5e612a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -90,7 +90,7 @@ def wait_until_fully_merged(node, table): except: return - raise Exception(f"There are still merges on-going after {retry} assignments") + raise Exception(f"There are still merges on-going after {i} assignments") def test_jbod_balanced_merge(start_cluster): diff --git a/tests/integration/test_jdbc_bridge/test.py b/tests/integration/test_jdbc_bridge/test.py index c4a0a525df3..1efd868e4a7 100644 --- a/tests/integration/test_jdbc_bridge/test.py +++ b/tests/integration/test_jdbc_bridge/test.py @@ -91,7 +91,7 @@ def test_jdbc_insert(started_cluster): """ CREATE TABLE test.test_insert ENGINE = Memory AS SELECT * FROM test.ClickHouseTable; - SELECT * + SELECT * FROM jdbc('{0}?mutation', 'INSERT INTO test.test_insert VALUES({1}, ''{1}'', ''{1}'')'); """.format( datasource, records @@ -115,7 +115,7 @@ def test_jdbc_update(started_cluster): """ CREATE TABLE test.test_update ENGINE = Memory AS SELECT * FROM test.ClickHouseTable; - SELECT * + SELECT * FROM jdbc( '{}?mutation', 'SET mutations_sync = 1; ALTER TABLE test.test_update UPDATE Str=''{}'' WHERE Num = {} - 1;' @@ -145,7 +145,7 @@ def test_jdbc_delete(started_cluster): """ CREATE TABLE test.test_delete ENGINE = Memory AS SELECT * FROM test.ClickHouseTable; - SELECT * + SELECT * FROM jdbc( '{}?mutation', 'SET mutations_sync = 1; ALTER TABLE test.test_delete DELETE WHERE Num < {} - 1;' @@ -158,7 +158,7 @@ def test_jdbc_delete(started_cluster): expected = records - 1 actual = instance.query( "SELECT Str FROM jdbc('{}', 'SELECT * FROM test.test_delete')".format( - datasource, records + datasource ) ) assert int(actual) == expected, "expecting {} but got {}".format(expected, actual) diff --git a/tests/integration/test_keeper_snapshot_small_distance/test.py b/tests/integration/test_keeper_snapshot_small_distance/test.py index be8bf1bd245..612c5b3c65d 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/test.py +++ b/tests/integration/test_keeper_snapshot_small_distance/test.py @@ -1,5 +1,5 @@ #!/usr/bin/env python3 -##!/usr/bin/env python3 + import pytest from helpers.cluster import ClickHouseCluster import helpers.keeper_utils as keeper_utils diff --git a/tests/integration/test_keeper_snapshots/test.py b/tests/integration/test_keeper_snapshots/test.py index 6dfb2078559..951970dba23 100644 --- a/tests/integration/test_keeper_snapshots/test.py +++ b/tests/integration/test_keeper_snapshots/test.py @@ -1,6 +1,5 @@ #!/usr/bin/env python3 -#!/usr/bin/env python3 import pytest from helpers.cluster import ClickHouseCluster import helpers.keeper_utils as keeper_utils diff --git a/tests/integration/test_keeper_three_nodes_start/test.py b/tests/integration/test_keeper_three_nodes_start/test.py index bc93a6089cb..6576d386fcb 100644 --- a/tests/integration/test_keeper_three_nodes_start/test.py +++ b/tests/integration/test_keeper_three_nodes_start/test.py @@ -1,6 +1,5 @@ #!/usr/bin/env python3 -#!/usr/bin/env python3 import pytest from helpers.cluster import ClickHouseCluster import random diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 7f77627e793..45ae88f427e 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -537,10 +537,7 @@ def test_freeze_unfreeze(cluster): def test_apply_new_settings(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) - config_path = os.path.join( - SCRIPT_DIR, - "./_gen/disk_storage_conf.xml".format(cluster.instances_dir_name), - ) + config_path = os.path.join(SCRIPT_DIR, "./_gen/disk_storage_conf.xml") azure_query( node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}" diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index f04425d83d4..0e87cb0e690 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -179,9 +179,7 @@ def test_different_data_types(started_cluster): for i in range(10): col = random.choice(["a", "b", "c"]) cursor.execute("UPDATE test_data_types SET {} = {};".format(col, i)) - cursor.execute( - """UPDATE test_data_types SET i = '2020-12-12';""".format(col, i) - ) + cursor.execute("UPDATE test_data_types SET i = '2020-12-12';") check_tables_are_synchronized(instance, "test_data_types", "id") @@ -452,7 +450,7 @@ def test_many_concurrent_queries(started_cluster): # also change primary key value print("try update primary key {}".format(thread_id)) cursor.execute( - "UPDATE {table}_{} SET key=key%100000+100000*{} WHERE key%{}=0".format( + "UPDATE {} SET key=key%100000+100000*{} WHERE key%{}=0".format( table_name, i + 1, i + 1 ) ) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index e6def99c076..8e37bd8d403 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -6,6 +6,7 @@ import time import threading import pytest +from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 6fe7a193129..0c3fbfb3cda 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -4,7 +4,7 @@ import os import json import helpers.client -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index d9dee0541b0..7762d17b96f 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -1,5 +1,5 @@ import helpers.client -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV import pyspark diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 23a95d5dd71..3240039ee81 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -702,7 +702,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): assert ( int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + ), "ClickHouse lost some messages: {}".format(result1) assert int(result2) == 10 @@ -1516,7 +1516,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): assert ( int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + ), "ClickHouse lost some messages: {}".format(result1) assert int(result2) == 4 * num_tables @@ -1966,7 +1966,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): assert ( int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + ), "ClickHouse lost some messages: {}".format(result1) # 4 tables, 2 consumers for each table => 8 consumer tags assert int(result2) == 8 @@ -2427,9 +2427,7 @@ def test_rabbitmq_drop_table_properly(rabbitmq_cluster): time.sleep(30) try: - exists = channel.queue_declare( - callback, queue="rabbit_queue_drop", passive=True - ) + exists = channel.queue_declare(queue="rabbit_queue_drop", passive=True) except Exception as e: exists = False @@ -3364,7 +3362,7 @@ def test_rabbitmq_flush_by_block_size(rabbitmq_cluster): routing_key="", body=json.dumps({"key": 0, "value": 0}), ) - except e: + except Exception as e: logging.debug(f"Got error: {str(e)}") produce_thread = threading.Thread(target=produce) @@ -3442,7 +3440,7 @@ def test_rabbitmq_flush_by_time(rabbitmq_cluster): ) logging.debug("Produced a message") time.sleep(0.8) - except e: + except Exception as e: logging.debug(f"Got error: {str(e)}") produce_thread = threading.Thread(target=produce) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 94432b89ab6..3b79ea7916d 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1850,7 +1850,7 @@ class TestCancelBackgroundMoving: config = inspect.cleandoc( f""" - { 256 * 1024 } + {256 * 1024} """ ) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index f944adbea41..538322473ee 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -325,7 +325,7 @@ def optimize_with_retry(node, table_name, retry=20): settings={"optimize_throw_if_noop": "1"}, ) break - except e: + except: time.sleep(0.5) From a474816fc744088ae0c300971de5043a5c054c72 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 29 Dec 2023 16:18:03 +0100 Subject: [PATCH 291/856] Add missing botocore import into clickhouse_backupview.py Signed-off-by: Azat Khuzhin --- utils/backupview/clickhouse_backupview.py | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/backupview/clickhouse_backupview.py b/utils/backupview/clickhouse_backupview.py index 4ba1f391d02..d1331e2ab49 100755 --- a/utils/backupview/clickhouse_backupview.py +++ b/utils/backupview/clickhouse_backupview.py @@ -8,6 +8,7 @@ import shutil import zipfile # For reading backups from zip archives import boto3 # For reading backups from S3 +import botocore ## Examples: From b2535d7f508c189c9fcbf871c3b60ac722afdaf7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 5 Jun 2024 09:50:39 +0200 Subject: [PATCH 292/856] Fix invalid escape sequence warnings Signed-off-by: Azat Khuzhin --- tests/integration/helpers/uclient.py | 4 ++-- tests/integration/test_prometheus_endpoint/test.py | 2 +- .../0_stateless/01056_window_view_proc_hop_watch.py | 6 +++--- .../01059_window_view_event_hop_watch_strict_asc.py | 8 ++++---- .../01062_window_view_event_hop_watch_asc.py | 6 +++--- .../01065_window_view_event_hop_watch_bounded.py | 4 ++-- .../0_stateless/01069_window_view_proc_tumble_watch.py | 8 ++++---- .../0_stateless/01070_window_view_watch_events.py | 6 +++--- .../0_stateless/01078_window_view_alter_query_watch.py | 10 +++++----- .../0_stateless/01082_window_view_watch_limit.py | 4 ++-- tests/queries/0_stateless/01921_test_progress_bar.py | 4 ++-- tests/queries/0_stateless/02473_infile_progress.py | 4 ++-- tests/queries/0_stateless/helpers/client.py | 4 ++-- tests/queries/0_stateless/helpers/shell.py | 2 +- 14 files changed, 36 insertions(+), 36 deletions(-) diff --git a/tests/integration/helpers/uclient.py b/tests/integration/helpers/uclient.py index 45c8b8f64e2..195eb52ffeb 100644 --- a/tests/integration/helpers/uclient.py +++ b/tests/integration/helpers/uclient.py @@ -8,7 +8,7 @@ sys.path.insert(0, os.path.join(CURDIR)) from . import uexpect -prompt = ":\) " +prompt = ":\\) " end_of_block = r".*\r\n.*\r\n" @@ -21,7 +21,7 @@ class client(object): self.client.eol("\r") self.client.logger(log, prefix=name) self.client.timeout(20) - self.client.expect("[#\$] ", timeout=2) + self.client.expect("[#\\$] ", timeout=2) self.client.send(command) def __enter__(self): diff --git a/tests/integration/test_prometheus_endpoint/test.py b/tests/integration/test_prometheus_endpoint/test.py index f140ebdfbe7..c1f04497b55 100644 --- a/tests/integration/test_prometheus_endpoint/test.py +++ b/tests/integration/test_prometheus_endpoint/test.py @@ -28,7 +28,7 @@ def parse_response_line(line): if line.startswith("#"): return {} - match = re.match("^([a-zA-Z_:][a-zA-Z0-9_:]+)(\{.*\})? -?(\d)", line) + match = re.match(r"^([a-zA-Z_:][a-zA-Z0-9_:]+)(\{.*\})? -?(\d)", line) assert match, line name, _, val = match.groups() return {name: int(val)} diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index 2db14fcdddf..e65650816ab 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -49,16 +49,16 @@ with client(name="client1>", log=log) as client1, client( client1.send("WATCH 01056_window_view_proc_hop_watch.wv") client1.expect("Query id" + end_of_block) - client1.expect("Progress: 0.00 rows.*\)") + client1.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO 01056_window_view_proc_hop_watch.mt VALUES (1, now('US/Samoa') + 3)" ) client1.expect("1" + end_of_block) - client1.expect("Progress: 1.00 rows.*\)") + client1.expect("Progress: 1.00 rows.*\\)") # send Ctrl-C client1.send("\x03", eol="") - match = client1.expect("(%s)|([#\$] )" % prompt) + match = client1.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) diff --git a/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py index 2323ee5c838..3dbb176b0dc 100755 --- a/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py +++ b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py @@ -47,7 +47,7 @@ with client(name="client1>", log=log) as client1, client( client1.send("WATCH db_01059_event_hop_watch_strict_asc.wv") client1.expect("Query id" + end_of_block) - client1.expect("Progress: 0.00 rows.*\)") + client1.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO db_01059_event_hop_watch_strict_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));" ) @@ -57,7 +57,7 @@ with client(name="client1>", log=log) as client1, client( ) client2.expect("Ok.") client1.expect("1*1990-01-01 12:00:02" + end_of_block) - client1.expect("Progress: 1.00 rows.*\)") + client1.expect("Progress: 1.00 rows.*\\)") client2.send( "INSERT INTO db_01059_event_hop_watch_strict_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));" @@ -65,11 +65,11 @@ with client(name="client1>", log=log) as client1, client( client2.expect("Ok.") client1.expect("1*1990-01-01 12:00:06" + end_of_block) client1.expect("1*1990-01-01 12:00:08" + end_of_block) - client1.expect("Progress: 3.00 rows.*\)") + client1.expect("Progress: 3.00 rows.*\\)") # send Ctrl-C client1.send("\x03", eol="") - match = client1.expect("(%s)|([#\$] )" % prompt) + match = client1.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) diff --git a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py index db9e8cef6c5..d6cc3ee1a88 100755 --- a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py +++ b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py @@ -49,7 +49,7 @@ with client(name="client1>", log=log) as client1, client( client1.send("WATCH 01062_window_view_event_hop_watch_asc.wv") client1.expect("Query id" + end_of_block) - client1.expect("Progress: 0.00 rows.*\)") + client1.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));" ) @@ -69,11 +69,11 @@ with client(name="client1>", log=log) as client1, client( client2.expect(prompt) client1.expect("1" + end_of_block) client1.expect("2" + end_of_block) - client1.expect("Progress: 3.00 rows.*\)") + client1.expect("Progress: 3.00 rows.*\\)") # send Ctrl-C client1.send("\x03", eol="") - match = client1.expect("(%s)|([#\$] )" % prompt) + match = client1.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) diff --git a/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py index b8d5ff02d37..e5f9ab59f60 100755 --- a/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py +++ b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py @@ -50,7 +50,7 @@ with client(name="client1>", log=log) as client1, client( client1.send("WATCH 01065_window_view_event_hop_watch_bounded.wv") client1.expect("Query id" + end_of_block) - client1.expect("Progress: 0.00 rows.*\)") + client1.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO 01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:00');" ) @@ -72,7 +72,7 @@ with client(name="client1>", log=log) as client1, client( # send Ctrl-C client1.send("\x03", eol="") - match = client1.expect("(%s)|([#\$] )" % prompt) + match = client1.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) diff --git a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py index 21c2e831afc..8c3a46992dc 100755 --- a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py +++ b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py @@ -49,23 +49,23 @@ with client(name="client1>", log=log) as client1, client( client1.send("WATCH 01069_window_view_proc_tumble_watch.wv") client1.expect("Query id" + end_of_block) - client1.expect("Progress: 0.00 rows.*\)") + client1.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO 01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 3)" ) client2.expect("Ok.") client1.expect("1" + end_of_block) - client1.expect("Progress: 1.00 rows.*\)") + client1.expect("Progress: 1.00 rows.*\\)") client2.send( "INSERT INTO 01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 3)" ) client2.expect("Ok.") client1.expect("1" + end_of_block) - client1.expect("Progress: 2.00 rows.*\)") + client1.expect("Progress: 2.00 rows.*\\)") # send Ctrl-C client1.send("\x03", eol="") - match = client1.expect("(%s)|([#\$] )" % prompt) + match = client1.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) diff --git a/tests/queries/0_stateless/01070_window_view_watch_events.py b/tests/queries/0_stateless/01070_window_view_watch_events.py index 1cf7678a014..172a82a29da 100755 --- a/tests/queries/0_stateless/01070_window_view_watch_events.py +++ b/tests/queries/0_stateless/01070_window_view_watch_events.py @@ -49,7 +49,7 @@ with client(name="client1>", log=log) as client1, client( client1.send("WATCH 01070_window_view_watch_events.wv EVENTS") client1.expect("Query id" + end_of_block) - client1.expect("Progress: 0.00 rows.*\)") + client1.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO 01070_window_view_watch_events.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));" ) @@ -59,11 +59,11 @@ with client(name="client1>", log=log) as client1, client( ) client2.expect("Ok.") client1.expect("1990-01-01 12:00:05" + end_of_block) - client1.expect("Progress: 1.00 rows.*\)") + client1.expect("Progress: 1.00 rows.*\\)") # send Ctrl-C client1.send("\x03", eol="") - match = client1.expect("(%s)|([#\$] )" % prompt) + match = client1.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) diff --git a/tests/queries/0_stateless/01078_window_view_alter_query_watch.py b/tests/queries/0_stateless/01078_window_view_alter_query_watch.py index 3f3dfe0cda8..05aeb1b4ccb 100755 --- a/tests/queries/0_stateless/01078_window_view_alter_query_watch.py +++ b/tests/queries/0_stateless/01078_window_view_alter_query_watch.py @@ -55,7 +55,7 @@ with client(name="client1>", log=log) as client1, client( client1.send("WATCH 01078_window_view_alter_query_watch.wv") client1.expect("Query id" + end_of_block) - client1.expect("Progress: 0.00 rows.*\)") + client1.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO 01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));" ) @@ -65,7 +65,7 @@ with client(name="client1>", log=log) as client1, client( ) client2.expect("Ok.") client1.expect("1" + end_of_block) - client1.expect("Progress: 1.00 rows.*\)") + client1.expect("Progress: 1.00 rows.*\\)") client2.send( "ALTER TABLE 01078_window_view_alter_query_watch.wv MODIFY QUERY SELECT count(a) * 2 AS count, hopEnd(wid) AS w_end FROM 01078_window_view_alter_query_watch.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid" ) @@ -75,7 +75,7 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client3.send("WATCH 01078_window_view_alter_query_watch.wv") client3.expect("Query id" + end_of_block) - client3.expect("Progress: 0.00 rows.*\)") + client3.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO 01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));" ) @@ -85,11 +85,11 @@ with client(name="client1>", log=log) as client1, client( ) client2.expect("Ok.") client3.expect("2" + end_of_block) - client3.expect("Progress: 1.00 rows.*\)") + client3.expect("Progress: 1.00 rows.*\\)") # send Ctrl-C client3.send("\x03", eol="") - match = client3.expect("(%s)|([#\$] )" % prompt) + match = client3.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client3.send(client3.command) client3.expect(prompt) diff --git a/tests/queries/0_stateless/01082_window_view_watch_limit.py b/tests/queries/0_stateless/01082_window_view_watch_limit.py index 9938ebcab98..5dcdfdb5020 100755 --- a/tests/queries/0_stateless/01082_window_view_watch_limit.py +++ b/tests/queries/0_stateless/01082_window_view_watch_limit.py @@ -49,7 +49,7 @@ with client(name="client1>", log=log) as client1, client( client1.send("WATCH 01082_window_view_watch_limit.wv LIMIT 1") client1.expect("Query id" + end_of_block) - client1.expect("Progress: 0.00 rows.*\)") + client1.expect("Progress: 0.00 rows.*\\)") client2.send( "INSERT INTO 01082_window_view_watch_limit.mt VALUES (1, '1990/01/01 12:00:00');" ) @@ -59,7 +59,7 @@ with client(name="client1>", log=log) as client1, client( ) client2.expect("Ok.") client1.expect("1" + end_of_block) - client1.expect("Progress: 1.00 rows.*\)") + client1.expect("Progress: 1.00 rows.*\\)") client1.expect("1 row" + end_of_block) client1.expect(prompt) diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index 54c7ae59894..6406534a647 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -15,6 +15,6 @@ log = None with client(name="client1>", log=log) as client1: client1.expect(prompt) client1.send("SELECT number FROM numbers(1000) FORMAT Null") - client1.expect("Progress: 1\.00 thousand rows, 8\.00 KB .*" + end_of_block) - client1.expect("0 rows in set. Elapsed: [\\w]{1}\.[\\w]{3} sec.") + client1.expect("Progress: 1\\.00 thousand rows, 8\\.00 KB .*" + end_of_block) + client1.expect("0 rows in set. Elapsed: [\\w]{1}\\.[\\w]{3} sec.") client1.expect("Peak memory usage: .*B" + end_of_block) diff --git a/tests/queries/0_stateless/02473_infile_progress.py b/tests/queries/0_stateless/02473_infile_progress.py index 9941736107f..4165eeb6d31 100755 --- a/tests/queries/0_stateless/02473_infile_progress.py +++ b/tests/queries/0_stateless/02473_infile_progress.py @@ -32,12 +32,12 @@ with client( ) client1.expect(prompt) client1.send(f"INSERT INTO test.infile_progress FROM INFILE '{filename}'") - client1.expect("Progress: 5.00 rows, 10.00 B.*\)") + client1.expect("Progress: 5.00 rows, 10.00 B.*\\)") client1.expect(prompt) # send Ctrl-C client1.send("\x03", eol="") - match = client1.expect("(%s)|([#\$] )" % prompt) + match = client1.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 5c8589dfca1..ac0896f2e93 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -8,7 +8,7 @@ sys.path.insert(0, os.path.join(CURDIR)) import uexpect -prompt = ":\) " +prompt = ":\\) " end_of_block = r".*\r\n.*\r\n" @@ -21,7 +21,7 @@ class client(object): self.client.eol("\r") self.client.logger(log, prefix=name) self.client.timeout(120) - self.client.expect("[#\$] ", timeout=60) + self.client.expect("[#\\$] ", timeout=60) self.client.send(command) def __enter__(self): diff --git a/tests/queries/0_stateless/helpers/shell.py b/tests/queries/0_stateless/helpers/shell.py index befb3dcd543..c3fff61ffc9 100644 --- a/tests/queries/0_stateless/helpers/shell.py +++ b/tests/queries/0_stateless/helpers/shell.py @@ -10,7 +10,7 @@ import uexpect class shell(object): - def __init__(self, command=None, name="", log=None, prompt="[#\$] "): + def __init__(self, command=None, name="", log=None, prompt="[#\\$] "): if command is None: command = ["/bin/bash", "--noediting"] self.prompt = prompt From 11905682a9facddcde8296309e97dedee5479afb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 29 Dec 2023 14:51:24 +0100 Subject: [PATCH 293/856] Check python code with flake8 Recently assert-on-tuple had been introduced in tests [1], let's prevent this. [1]: https://github.com/ClickHouse/ClickHouse/pull/56367#discussion_r1437098533 v2: pin flake8 to 4.0.1 (instead of originally 6.1) due to other dependencies, hope that it will find such errors Signed-off-by: Azat Khuzhin --- docker/test/style/Dockerfile | 1 + docker/test/style/run.sh | 2 + docs/en/development/continuous-integration.md | 3 + utils/check-style/check-flake8 | 55 +++++++++++++++++++ .../check-style/process_style_check_result.py | 1 + 5 files changed, 62 insertions(+) create mode 100755 utils/check-style/check-flake8 diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index cb29185f068..91768c8328d 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -30,6 +30,7 @@ RUN pip3 install \ mypy==1.8.0 \ pylint==3.1.0 \ python-magic==0.4.24 \ + flake8==4.0.1 \ requests \ thefuzz \ types-requests \ diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index cc6cb292b66..64803191532 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -9,6 +9,8 @@ echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt echo "Check python formatting with black" | ts ./check-black -n |& tee /test_output/black_output.txt +echo "Check python with flake8" | ts +./check-flake8 |& tee /test_output/flake8_output.txt echo "Check python type hinting with mypy" | ts ./check-mypy -n |& tee /test_output/mypy_output.txt echo "Check typos" | ts diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index c348eb5ca07..c283cfbf4c2 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -91,6 +91,9 @@ cd ./utils/check-style # Check python type hinting with mypy ./check-mypy +# Check python with flake8 +./check-flake8 + # Check code with codespell ./check-typos diff --git a/utils/check-style/check-flake8 b/utils/check-style/check-flake8 new file mode 100755 index 00000000000..58dd8a99d40 --- /dev/null +++ b/utils/check-style/check-flake8 @@ -0,0 +1,55 @@ +#!/usr/bin/env bash + +function join_by() { local IFS="$1"; shift; echo "$*"; } + +set -e + +# We check only our code, that's why we skip contrib +GIT_ROOT=$(git rev-parse --show-cdup) +GIT_ROOT=${GIT_ROOT:-./} + +# Find all *.py, *.python files and executable files without extension +# that are determined as python scripts by 'file' util +# in the repo except the contrib directory. +find_cmd=( + find "$GIT_ROOT" -type f -not -path "${GIT_ROOT}contrib/*" + \( + \( + -name '*.py' -or -name "*.python" -or + \( + -executable -not -name "*.*" -exec sh -c 'file {} | grep -q "Python script"' \; + \) + \) + # We skip modules generated by the protocol buffer compiler from *.proto files. + -and -not -name '*_pb2.py' -and -not -name '*_pb2_grpc.py' + \) -print0 +) + +ignores=( + E101 # Indentation contains mixed spaces and tabs + E203 # Whitespace before ':' + E226 # missing whitespace around arithmetic operator + E266 # Too many leading '#' for block comment + E401 # Multiple imports on one line + E402 # Module level import not at top of file + E501 # line too long + E711 # Comparison to None should be 'cond is None:' + E712 # Comparison to true should be 'if cond is true:' or 'if cond:' + E713 # Test for membership should be 'not in' + E714 # Test for object identity should be 'is not' + E722 # Do not use bare except, specify exception instead + E731 # Do not assign a lambda expression, use a def + E741 # Do not use variables named 'I', 'O', or 'l' + F401 # Module imported but unused + F403 # 'from module import *' used; unable to detect undefined names + F405 # Name may be undefined, or defined from star imports: module + F522 # .format(...) unused named arguments + F541 # f-string without any placeholders + F811 # redefinition of unused name from line N + F841 # local variable name is assigned to but never used + W191 # Indentation contains tabs + W291 # Trailing whitespace + W293 # Blank line contains whitespace + W503 # Line break occurred before a binary operator +) +"${find_cmd[@]}" | xargs -0 flake8 --ignore "$(join_by , "${ignores[@]}")" diff --git a/utils/check-style/process_style_check_result.py b/utils/check-style/process_style_check_result.py index e603084732d..2c349114a59 100755 --- a/utils/check-style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -18,6 +18,7 @@ def process_result(result_folder): "style", "pylint", "black", + "flake8", "mypy", "typos", "whitespaces", From 70b5457d54fdbbf5a4a8c3bb169d63060548f222 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Jun 2024 09:52:42 -0300 Subject: [PATCH 294/856] build poco no proxy only once --- src/Common/EnvironmentProxyConfigurationResolver.cpp | 3 ++- src/Common/proxyConfigurationToPocoProxyConfig.cpp | 6 +++--- src/Common/proxyConfigurationToPocoProxyConfig.h | 2 ++ 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index 1b7431ee827..32ad321fdee 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -1,6 +1,7 @@ #include "EnvironmentProxyConfigurationResolver.h" #include +#include #include namespace DB @@ -77,7 +78,7 @@ ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() { static const auto * http_proxy_host = getProxyHost(Protocol::HTTP); static const auto * https_proxy_host = getProxyHost(Protocol::HTTPS); - static const auto no_proxy_hosts_string = getNoProxyHostsString(); + static const auto no_proxy_hosts_string = buildPocoNonProxyHosts(getNoProxyHostsString()); static const auto http_proxy_configuration = buildProxyConfiguration(Protocol::HTTP, http_proxy_host, no_proxy_hosts_string); static const auto https_proxy_configuration = buildProxyConfiguration(Protocol::HTTPS, https_proxy_host, no_proxy_hosts_string); diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.cpp b/src/Common/proxyConfigurationToPocoProxyConfig.cpp index fbd9a229da3..c1a439d2f36 100644 --- a/src/Common/proxyConfigurationToPocoProxyConfig.cpp +++ b/src/Common/proxyConfigurationToPocoProxyConfig.cpp @@ -36,6 +36,8 @@ std::string buildPocoRegexpEntryWithoutLeadingDot(const std::string & host) return RE2::QuoteMeta(view_without_leading_dot); } +} + /* * Even though there is not an RFC that defines NO_PROXY, it is usually a comma-separated list of domains. * Different tools implement their own versions of `NO_PROXY` support. Some support CIDR blocks, some support wildcard etc. @@ -93,8 +95,6 @@ std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts_string) return result; } -} - Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration) { Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; @@ -104,7 +104,7 @@ Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(co poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol); poco_proxy_config.tunnel = proxy_configuration.tunneling; poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol); - poco_proxy_config.nonProxyHosts = buildPocoNonProxyHosts(proxy_configuration.no_proxy_hosts); + poco_proxy_config.nonProxyHosts = proxy_configuration.no_proxy_hosts; return poco_proxy_config; } diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.h b/src/Common/proxyConfigurationToPocoProxyConfig.h index d093b0f3521..c118bd059f9 100644 --- a/src/Common/proxyConfigurationToPocoProxyConfig.h +++ b/src/Common/proxyConfigurationToPocoProxyConfig.h @@ -8,4 +8,6 @@ namespace DB Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration); +std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts_string); + } From 877717840bd889d0952531b2159321737d1d6200 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Jun 2024 10:18:04 -0300 Subject: [PATCH 295/856] fix and simplify proxy provider ut --- ..._proxy_configuration_resolver_provider.cpp | 158 ++++++------------ 1 file changed, 48 insertions(+), 110 deletions(-) diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index d248835699a..368ce12cd7b 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -1,6 +1,9 @@ #include #include +#include +#include +#include #include #include @@ -38,18 +41,11 @@ TEST_F(ProxyConfigurationResolverProviderTests, EnvironmentResolverShouldBeUsedI EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server, no_proxy_hosts); const auto & config = getContext().context->getConfigRef(); - auto http_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, config)->resolve(); - auto https_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, config)->resolve(); + auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, config); + auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, config); - ASSERT_EQ(http_configuration.host, http_env_proxy_server.getHost()); - ASSERT_EQ(http_configuration.port, http_env_proxy_server.getPort()); - ASSERT_EQ(http_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_env_proxy_server.getScheme())); - ASSERT_EQ(http_configuration.no_proxy_hosts, no_proxy_hosts); - - ASSERT_EQ(https_configuration.host, https_env_proxy_server.getHost()); - ASSERT_EQ(https_configuration.port, https_env_proxy_server.getPort()); - ASSERT_EQ(https_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_env_proxy_server.getScheme())); - ASSERT_EQ(https_configuration.no_proxy_hosts, no_proxy_hosts); + ASSERT_TRUE(std::dynamic_pointer_cast(http_resolver)); + ASSERT_TRUE(std::dynamic_pointer_cast(https_resolver)); } TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPOnly) @@ -62,19 +58,11 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPOnly) config->setString("proxy.http.uri", http_list_proxy_server.toString()); context->setConfig(config); - auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve(); + auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config); + auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config); - ASSERT_EQ(http_proxy_configuration.host, http_list_proxy_server.getHost()); - ASSERT_EQ(http_proxy_configuration.port, http_list_proxy_server.getPort()); - ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_list_proxy_server.getScheme())); - ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, no_proxy_hosts); - - auto https_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve(); - - // No https configuration since it's not set - ASSERT_EQ(https_proxy_configuration.host, ""); - ASSERT_EQ(https_proxy_configuration.port, 0); - ASSERT_EQ(https_proxy_configuration.no_proxy_hosts, ""); + ASSERT_TRUE(std::dynamic_pointer_cast(http_resolver)); + ASSERT_TRUE(std::dynamic_pointer_cast(https_resolver)); } TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPSOnly) @@ -87,19 +75,11 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPSOnly) config->setString("proxy.https.uri", https_list_proxy_server.toString()); context->setConfig(config); - auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve(); + auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config); + auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config); - ASSERT_EQ(http_proxy_configuration.host, ""); - ASSERT_EQ(http_proxy_configuration.port, 0); - - auto https_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve(); - - ASSERT_EQ(https_proxy_configuration.host, https_list_proxy_server.getHost()); - - // still HTTP because the proxy host is not HTTPS - ASSERT_EQ(https_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_list_proxy_server.getScheme())); - ASSERT_EQ(https_proxy_configuration.port, https_list_proxy_server.getPort()); - ASSERT_EQ(https_proxy_configuration.no_proxy_hosts, no_proxy_hosts); + ASSERT_TRUE(std::dynamic_pointer_cast(http_resolver)); + ASSERT_TRUE(std::dynamic_pointer_cast(https_resolver)); } TEST_F(ProxyConfigurationResolverProviderTests, ListBoth) @@ -117,35 +97,38 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListBoth) context->setConfig(config); - auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve(); + auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config); + auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config); - ASSERT_EQ(http_proxy_configuration.host, http_list_proxy_server.getHost()); - ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_list_proxy_server.getScheme())); - ASSERT_EQ(http_proxy_configuration.port, http_list_proxy_server.getPort()); - ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, no_proxy_hosts); - - auto https_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve(); - - ASSERT_EQ(https_proxy_configuration.host, https_list_proxy_server.getHost()); - - // still HTTP because the proxy host is not HTTPS - ASSERT_EQ(https_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_list_proxy_server.getScheme())); - ASSERT_EQ(https_proxy_configuration.port, https_list_proxy_server.getPort()); - ASSERT_EQ(https_proxy_configuration.no_proxy_hosts, no_proxy_hosts); + ASSERT_TRUE(std::dynamic_pointer_cast(http_resolver)); + ASSERT_TRUE(std::dynamic_pointer_cast(https_resolver)); } -TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTP) +TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTPS) { - /* - * Since there is no way to call `ProxyConfigurationResolver::resolve` on remote resolver, - * it is hard to verify the remote resolver was actually picked. One hackish way to assert - * the remote resolver was OR was not picked based on the configuration, is to use the - * environment resolver. Since the environment resolver is always returned as a fallback, - * we can assert the remote resolver was not picked if `ProxyConfigurationResolver::resolve` - * succeeds and returns an environment proxy configuration. - * */ - EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server); + ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); + config->setString("proxy", ""); + config->setString("proxy.http", ""); + config->setString("proxy.http.resolver", ""); + config->setString("proxy.http.resolver.endpoint", "http://resolver:8080/hostname"); + + // even tho proxy protocol / scheme is https, it should not be picked (prior to this PR, it would be picked) + config->setString("proxy.http.resolver.proxy_scheme", "https"); + config->setString("proxy.http.resolver.proxy_port", "80"); + config->setString("proxy.http.resolver.proxy_cache_time", "10"); + + context->setConfig(config); + + auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config); + auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config); + + ASSERT_TRUE(std::dynamic_pointer_cast(http_resolver)); + ASSERT_TRUE(std::dynamic_pointer_cast(https_resolver)); +} + +TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverHTTPSOnly) +{ ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); config->setString("proxy", ""); @@ -161,66 +144,21 @@ TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolC context->setConfig(config); - auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve(); + auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config); + auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config); - /* - * Asserts env proxy is used and not the remote resolver. If the remote resolver is picked, it is an error because - * there is no `http` specification for remote resolver - * */ - ASSERT_EQ(http_proxy_configuration.host, http_env_proxy_server.getHost()); - ASSERT_EQ(http_proxy_configuration.port, http_env_proxy_server.getPort()); - ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_env_proxy_server.getScheme())); - ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, ""); + ASSERT_TRUE(std::dynamic_pointer_cast(http_resolver)); + ASSERT_TRUE(std::dynamic_pointer_cast(https_resolver)); } -TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTPS) -{ - /* - * Since there is no way to call `ProxyConfigurationResolver::resolve` on remote resolver, - * it is hard to verify the remote resolver was actually picked. One hackish way to assert - * the remote resolver was OR was not picked based on the configuration, is to use the - * environment resolver. Since the environment resolver is always returned as a fallback, - * we can assert the remote resolver was not picked if `ProxyConfigurationResolver::resolve` - * succeeds and returns an environment proxy configuration. - * */ - EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server); - - ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); - - config->setString("proxy", ""); - config->setString("proxy.http", ""); - config->setString("proxy.http.resolver", ""); - config->setString("proxy.http.resolver.endpoint", "http://resolver:8080/hostname"); - - // even tho proxy protocol / scheme is https, it should not be picked (prior to this PR, it would be picked) - config->setString("proxy.http.resolver.proxy_scheme", "https"); - config->setString("proxy.http.resolver.proxy_port", "80"); - config->setString("proxy.http.resolver.proxy_cache_time", "10"); - - context->setConfig(config); - - auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve(); - - /* - * Asserts env proxy is used and not the remote resolver. If the remote resolver is picked, it is an error because - * there is no `http` specification for remote resolver - * */ - ASSERT_EQ(http_proxy_configuration.host, https_env_proxy_server.getHost()); - ASSERT_EQ(http_proxy_configuration.port, https_env_proxy_server.getPort()); - ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_env_proxy_server.getScheme())); - ASSERT_EQ(http_proxy_configuration.no_proxy_hosts, ""); -} - -// remote resolver is tricky to be tested in unit tests - template void test_tunneling(DB::ContextMutablePtr context) { - EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server); - ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); config->setString("proxy", ""); + config->setString("proxy.https", ""); + config->setString("proxy.https.uri", http_list_proxy_server.toString()); if constexpr (STRING) { From 0525eeb3d85761a6395b5c126f41180c8c117c61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Jun 2024 13:41:25 +0000 Subject: [PATCH 296/856] Fixing test. --- src/DataTypes/DataTypeNullable.cpp | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index df9689919cc..0ecb5370a7d 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -177,15 +177,7 @@ DataTypePtr removeNullableOrLowCardinalityNullable(const DataTypePtr & type) bool canContainNull(const IDataType & type) { - if (type.isNullable() || type.isLowCardinalityNullable() || isDynamic(type)) - return true; - - if (const auto * variant = typeid_cast(&type)) - for (const auto & elem : variant->getVariants()) - if (canContainNull(*elem)) - return true; - - return false; + return type.isNullable() || type.isLowCardinalityNullable() || isDynamic(type) || isVariant(type); } } From 7ce67265c70075009617c1b43700d351bf43a9ad Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Wed, 5 Jun 2024 14:06:31 +0000 Subject: [PATCH 297/856] PR post-review fixes --- src/Functions/FunctionsRound.h | 51 ++++++++++++---------------------- 1 file changed, 18 insertions(+), 33 deletions(-) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index b31d1bcee8a..a05619b54bb 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -373,11 +373,11 @@ public: } } - static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) + static void applyOne(T in, size_t scale, T& out) { using ScalarOp = Op; auto s = ScalarOp::prepare(scale); - ScalarOp::compute(in, s, out); + ScalarOp::compute(&in, s, &out); } }; @@ -435,9 +435,9 @@ public: } } - static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) + static void applyOne(T in, size_t scale, T& out) { - Op::compute(in, scale, out); + Op::compute(&in, scale, &out); } }; @@ -475,17 +475,17 @@ public: } } - static void applyOne(const NativeType* __restrict in, UInt32 in_scale, NativeType* __restrict out, Scale scale_arg) + static void applyOne(NativeType in, UInt32 in_scale, NativeType& out, Scale scale_arg) { scale_arg = in_scale - scale_arg; if (scale_arg > 0) { auto scale = intExp10OfSize(scale_arg); - Op::compute(in, scale, out); + Op::compute(&in, scale, &out); } else { - memcpy(out, in, sizeof(T)); + memcpy(&out, &in, sizeof(T)); } } }; @@ -553,35 +553,27 @@ struct Dispatcher const auto & scale_data = scale_col_typed->getData(); const size_t rows = value_data.size(); - const T * end_in = value_data.data() + rows; - const T * __restrict p_in = value_data.data(); - const ScaleType * __restrict p_scale = scale_data.data(); - T * __restrict p_out = vec_res.data(); - - while (p_in < end_in) + for (size_t i = 0; i < rows; ++i) { - Int64 scale64 = *p_scale; + Int64 scale64 = scale_data[i]; validateScale(scale64); Scale raw_scale = scale64; if (raw_scale == 0) { size_t scale = 1; - FunctionRoundingImpl::applyOne(p_in, scale, p_out); + FunctionRoundingImpl::applyOne(value_data[i], scale, vec_res[i]); } else if (raw_scale > 0) { size_t scale = intExp10(raw_scale); - FunctionRoundingImpl::applyOne(p_in, scale, p_out); + FunctionRoundingImpl::applyOne(value_data[i], scale, vec_res[i]); } else { size_t scale = intExp10(-raw_scale); - FunctionRoundingImpl::applyOne(p_in, scale, p_out); + FunctionRoundingImpl::applyOne(value_data[i], scale, vec_res[i]); } - ++p_in; - ++p_scale; - ++p_out; } } } @@ -596,7 +588,7 @@ struct Dispatcher { public: template - static ColumnPtr apply(const IColumn * value_col, const IColumn* scale_col = nullptr) + static ColumnPtr apply(const IColumn * value_col, const IColumn * scale_col = nullptr) { const auto & value_col_typed = checkAndGetColumn>(*value_col); const typename ColumnDecimal::Container & vec_src = value_col_typed.getData(); @@ -611,27 +603,20 @@ public: auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst>(scale_col)); DecimalRoundingImpl::apply(value_col_typed.getData(), value_col_typed.getScale(), vec_res, scale_arg); } - /// Non-cosnt scale argument + /// Non-const scale argument else if (const auto * scale_col_typed = checkAndGetColumn>(scale_col)) { const auto & scale = scale_col_typed->getData(); const size_t rows = vec_src.size(); - using NativeType = typename T::NativeType; - const NativeType * __restrict p_in = reinterpret_cast(vec_src.data()); - const ScaleType * __restrict p_scale = scale.data(); - const NativeType * end_in = p_in + rows; - NativeType * __restrict p_out = reinterpret_cast(vec_res.data()); - while (p_in < end_in) + for (size_t i = 0; i < rows; ++i) { - Int64 scale64 = *p_scale; + Int64 scale64 = scale[i]; validateScale(scale64); Scale raw_scale = scale64; - DecimalRoundingImpl::applyOne(p_in, value_col_typed.getScale(), p_out, raw_scale); - ++p_in; - ++p_scale; - ++p_out; + DecimalRoundingImpl::applyOne(value_col_typed.getElement(i), value_col_typed.getScale(), + reinterpret_cast::NativeT&>(col_res->getElement(i)), raw_scale); } } } From 7590845bf4d2dc8510479a6c8bdb4e10c0058110 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 5 Jun 2024 14:23:24 +0000 Subject: [PATCH 298/856] slightly better --- src/Storages/MergeTree/MergeTask.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index acbcecb8b7e..25e0d825778 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -7,7 +7,6 @@ #include #include -#include "Core/NamesAndTypes.h" #include #include #include @@ -153,7 +152,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu auto it = global_ctx->skip_indexes_by_column.find(column.name); if (it != global_ctx->skip_indexes_by_column.end()) { - for (auto && index : it->second) + for (auto & index : it->second) global_ctx->merging_skip_indexes.push_back(std::move(index)); global_ctx->skip_indexes_by_column.erase(it); @@ -997,8 +996,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); - auto merging_column_names = global_ctx->merging_columns.getNames(); - for (const auto & part : global_ctx->future_part->parts) { Pipe pipe = createMergeTreeSequentialSource( @@ -1006,7 +1003,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() *global_ctx->data, global_ctx->storage_snapshot, part, - merging_column_names, + global_ctx->merging_columns.getNames(), /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, /*apply_deleted_mask=*/ true, From 687b38e52d978c36eb7cfd13990b7f700a5d23df Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 5 Jun 2024 14:46:28 +0000 Subject: [PATCH 299/856] fix test with ordinary database --- .../queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql | 2 +- .../queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql index ba770656532..4f2fb8dea0c 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql @@ -32,7 +32,7 @@ SELECT groups[2] AS merged, groups[3] AS gathered FROM system.text_log -WHERE query_id = uuid || '::all_1_2_1' AND notEmpty(groups) +WHERE (query_id = uuid || '::all_1_2_1') OR (query_id = currentDatabase() || '.t_ind_merge_1::all_1_2_1') AND notEmpty(groups) ORDER BY event_time_microseconds; DROP TABLE t_ind_merge_1; diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql index e29653a2e77..0710b1069a8 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql @@ -35,7 +35,7 @@ SELECT groups[2] AS merged, groups[3] AS gathered FROM system.text_log -WHERE query_id = uuid || '::all_1_2_1' AND notEmpty(groups) +WHERE (query_id = uuid || '::all_1_2_1') OR (query_id = currentDatabase() || '.t_ind_merge_2::all_1_2_1') AND notEmpty(groups) ORDER BY event_time_microseconds; DROP TABLE t_ind_merge_2; From ee5100171e383c2c82cdaacdedd9e78146981bb5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Jun 2024 15:50:27 +0000 Subject: [PATCH 300/856] Avoid duplicating names for new inputs of ActionsDAG::split result. --- src/Interpreters/ActionsDAG.cpp | 34 +++++++++++++++++-- ...mv_prewhere_duplicating_name_bug.reference | 2 ++ ...03166_mv_prewhere_duplicating_name_bug.sql | 7 ++++ 3 files changed, 41 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference create mode 100644 tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index cfccc835d29..6915f82d067 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1718,6 +1718,13 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split /// List of nodes from current actions which are not inputs, but will be in second part. NodeRawConstPtrs new_inputs; + /// Avoid new inputs to have the same name as existing inputs. + /// It's allowed for DAG but may break Block invariant 'columns with identical name must have identical structure'. + std::unordered_set duplicate_inputs; + size_t duplicate_counter = 0; + for (const auto * input : inputs) + duplicate_inputs.insert(input->result_name); + struct Frame { const Node * node = nullptr; @@ -1830,7 +1837,8 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split input_node.result_name = child->result_name; child_data.to_second = &second_nodes.emplace_back(std::move(input_node)); - new_inputs.push_back(child); + if (child->type != ActionType::INPUT) + new_inputs.push_back(child); } } @@ -1886,7 +1894,29 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split for (const auto * input : new_inputs) { - const auto & cur = data[input]; + auto & cur = data[input]; + + bool is_name_updated = false; + while (!duplicate_inputs.insert(cur.to_first->result_name).second) + { + is_name_updated = true; + cur.to_first->result_name = fmt::format("{}_{}", input->result_name, duplicate_counter); + ++duplicate_counter; + } + + if (is_name_updated) + { + Node input_node; + input_node.type = ActionType::INPUT; + input_node.result_type = cur.to_first->result_type; + input_node.result_name = cur.to_first->result_name; + + auto * new_input = &second_nodes.emplace_back(std::move(input_node)); + cur.to_second->type = ActionType::ALIAS; + cur.to_second->children = {new_input}; + cur.to_second = new_input; + } + second_inputs.push_back(cur.to_second); first_outputs.push_back(cur.to_first); } diff --git a/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference new file mode 100644 index 00000000000..b50fdcee209 --- /dev/null +++ b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference @@ -0,0 +1,2 @@ +a b +a b diff --git a/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql new file mode 100644 index 00000000000..e32d23920dd --- /dev/null +++ b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql @@ -0,0 +1,7 @@ +create table src (x Int64) engine = Log; +create table dst (s String, lc LowCardinality(String)) engine MergeTree order by s; +create materialized view mv to dst (s String, lc String) as select 'a' as s, toLowCardinality('b') as lc from src; +insert into src values (1); + +select s, lc from mv where not ignore(lc) settings allow_experimental_analyzer=0; +select s, lc from mv where not ignore(lc) settings allow_experimental_analyzer=1; From 90332b3c8916cbdc22cb1291a943598193f66e24 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Jun 2024 13:46:45 -0300 Subject: [PATCH 301/856] fix tunneling --- .../EnvironmentProxyConfigurationResolver.cpp | 56 ++++++++------ .../EnvironmentProxyConfigurationResolver.h | 3 - src/Common/ProxyConfiguration.h | 6 ++ src/Common/ProxyConfigurationResolver.h | 7 -- src/Common/ProxyListConfigurationResolver.cpp | 7 +- .../RemoteProxyConfigurationResolver.cpp | 2 +- .../gtest_proxy_environment_configuration.cpp | 75 ++++--------------- 7 files changed, 61 insertions(+), 95 deletions(-) diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index 32ad321fdee..bf062b8d9ba 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -48,30 +48,33 @@ namespace return no_proxy; } -} -ProxyConfiguration EnvironmentProxyConfigurationResolver::buildProxyConfiguration(Protocol protocol, const char * proxy_host, const std::string & no_proxy_hosts_string) -{ - if (!proxy_host) + ProxyConfiguration buildProxyConfiguration( + ProxyConfiguration::Protocol request_protocol, + const Poco::URI & uri, + const std::string & no_proxy_hosts_string, + bool disable_tunneling_for_https_requests_over_http_proxy) { - return {}; + const auto & host = uri.getHost(); + const auto & scheme = uri.getScheme(); + const auto port = uri.getPort(); + + const bool use_tunneling_for_https_requests_over_http_proxy = ProxyConfiguration::useTunneling( + request_protocol, + ProxyConfiguration::protocolFromString(scheme), + disable_tunneling_for_https_requests_over_http_proxy); + + LOG_TRACE(getLogger("EnvironmentProxyConfigurationResolver"), "Use proxy from environment: {}://{}:{}", scheme, host, port); + + return ProxyConfiguration { + host, + ProxyConfiguration::protocolFromString(scheme), + port, + use_tunneling_for_https_requests_over_http_proxy, + request_protocol, + no_proxy_hosts_string + }; } - - auto uri = Poco::URI(proxy_host); - auto host = uri.getHost(); - auto scheme = uri.getScheme(); - auto port = uri.getPort(); - - LOG_TRACE(getLogger("EnvironmentProxyConfigurationResolver"), "Use proxy from environment: {}://{}:{}", scheme, host, port); - - return ProxyConfiguration { - host, - ProxyConfiguration::protocolFromString(scheme), - port, - useTunneling(protocol, ProxyConfiguration::protocolFromString(scheme), disable_tunneling_for_https_requests_over_http_proxy), - protocol, - no_proxy_hosts_string - }; } ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() @@ -80,10 +83,15 @@ ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() static const auto * https_proxy_host = getProxyHost(Protocol::HTTPS); static const auto no_proxy_hosts_string = buildPocoNonProxyHosts(getNoProxyHostsString()); - static const auto http_proxy_configuration = buildProxyConfiguration(Protocol::HTTP, http_proxy_host, no_proxy_hosts_string); - static const auto https_proxy_configuration = buildProxyConfiguration(Protocol::HTTPS, https_proxy_host, no_proxy_hosts_string); + static const Poco::URI http_proxy_uri(http_proxy_host ? http_proxy_host : ""); + static const Poco::URI https_proxy_uri(https_proxy_host ? https_proxy_host : ""); + + return buildProxyConfiguration( + request_protocol, + request_protocol == Protocol::HTTP ? http_proxy_uri : https_proxy_uri, + no_proxy_hosts_string, + disable_tunneling_for_https_requests_over_http_proxy); - return request_protocol == Protocol::HTTP ? http_proxy_configuration : https_proxy_configuration; } } diff --git a/src/Common/EnvironmentProxyConfigurationResolver.h b/src/Common/EnvironmentProxyConfigurationResolver.h index 08f91790162..6bc9d8a368c 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.h +++ b/src/Common/EnvironmentProxyConfigurationResolver.h @@ -15,9 +15,6 @@ public: ProxyConfiguration resolve() override; void errorReport(const ProxyConfiguration &) override {} - -private: - ProxyConfiguration buildProxyConfiguration(Protocol protocol, const char * proxy_host, const std::string & no_proxy_hosts); }; } diff --git a/src/Common/ProxyConfiguration.h b/src/Common/ProxyConfiguration.h index d2d195b2d2e..3398afce37c 100644 --- a/src/Common/ProxyConfiguration.h +++ b/src/Common/ProxyConfiguration.h @@ -44,6 +44,12 @@ struct ProxyConfiguration } } + static bool useTunneling(Protocol request_protocol, Protocol proxy_protocol, bool disable_tunneling_for_https_requests_over_http_proxy) + { + bool is_https_request_over_http_proxy = request_protocol == Protocol::HTTPS && proxy_protocol == Protocol::HTTP; + return is_https_request_over_http_proxy && !disable_tunneling_for_https_requests_over_http_proxy; + } + std::string host = std::string{}; Protocol protocol = Protocol::HTTP; uint16_t port = 0; diff --git a/src/Common/ProxyConfigurationResolver.h b/src/Common/ProxyConfigurationResolver.h index b82936502bb..1e9f4ad77f7 100644 --- a/src/Common/ProxyConfigurationResolver.h +++ b/src/Common/ProxyConfigurationResolver.h @@ -19,13 +19,6 @@ struct ProxyConfigurationResolver virtual void errorReport(const ProxyConfiguration & config) = 0; protected: - - static bool useTunneling(Protocol request_protocol, Protocol proxy_protocol, bool disable_tunneling_for_https_requests_over_http_proxy) - { - bool is_https_request_over_http_proxy = request_protocol == Protocol::HTTPS && proxy_protocol == Protocol::HTTP; - return is_https_request_over_http_proxy && !disable_tunneling_for_https_requests_over_http_proxy; - } - Protocol request_protocol; bool disable_tunneling_for_https_requests_over_http_proxy = false; }; diff --git a/src/Common/ProxyListConfigurationResolver.cpp b/src/Common/ProxyListConfigurationResolver.cpp index e0452e98544..a23c3a61951 100644 --- a/src/Common/ProxyListConfigurationResolver.cpp +++ b/src/Common/ProxyListConfigurationResolver.cpp @@ -29,11 +29,16 @@ ProxyConfiguration ProxyListConfigurationResolver::resolve() auto & proxy = proxies[index]; + bool use_tunneling_for_https_requests_over_http_proxy = ProxyConfiguration::useTunneling( + request_protocol, + ProxyConfiguration::protocolFromString(proxy.getScheme()), + disable_tunneling_for_https_requests_over_http_proxy); + return ProxyConfiguration { proxy.getHost(), ProxyConfiguration::protocolFromString(proxy.getScheme()), proxy.getPort(), - useTunneling(request_protocol, ProxyConfiguration::protocolFromString(proxy.getScheme()), disable_tunneling_for_https_requests_over_http_proxy), + use_tunneling_for_https_requests_over_http_proxy, request_protocol, no_proxy_hosts }; diff --git a/src/Common/RemoteProxyConfigurationResolver.cpp b/src/Common/RemoteProxyConfigurationResolver.cpp index 33dc0e957db..8fd9d381ece 100644 --- a/src/Common/RemoteProxyConfigurationResolver.cpp +++ b/src/Common/RemoteProxyConfigurationResolver.cpp @@ -85,7 +85,7 @@ ProxyConfiguration RemoteProxyConfigurationResolver::resolve() auto proxy_protocol = ProxyConfiguration::protocolFromString(proxy_protocol_string); - bool use_tunneling_for_https_requests_over_http_proxy = useTunneling( + bool use_tunneling_for_https_requests_over_http_proxy = ProxyConfiguration::useTunneling( request_protocol, proxy_protocol, disable_tunneling_for_https_requests_over_http_proxy); diff --git a/src/Common/tests/gtest_proxy_environment_configuration.cpp b/src/Common/tests/gtest_proxy_environment_configuration.cpp index 81388fd877f..095e44dfb86 100644 --- a/src/Common/tests/gtest_proxy_environment_configuration.cpp +++ b/src/Common/tests/gtest_proxy_environment_configuration.cpp @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -13,73 +14,29 @@ namespace auto https_proxy_server = Poco::URI("https://proxy_server:3128"); } -TEST(EnvironmentProxyConfigurationResolver, TestHTTP) +TEST(EnvironmentProxyConfigurationResolver, TestHTTPandHTTPS) { std::string no_proxy_string = "localhost,,127.0.0.1,some_other_domain,,,, last_domain,"; - EnvironmentProxySetter setter(http_proxy_server, {}, no_proxy_string); + std::string poco_no_proxy_regex = buildPocoNonProxyHosts(no_proxy_string); + EnvironmentProxySetter setter(http_proxy_server, https_proxy_server, no_proxy_string); - EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTP); + EnvironmentProxyConfigurationResolver http_resolver(ProxyConfiguration::Protocol::HTTP); - auto configuration = resolver.resolve(); + auto http_configuration = http_resolver.resolve(); - ASSERT_EQ(configuration.host, http_proxy_server.getHost()); - ASSERT_EQ(configuration.port, http_proxy_server.getPort()); - ASSERT_EQ(configuration.protocol, ProxyConfiguration::protocolFromString(http_proxy_server.getScheme())); - ASSERT_EQ(configuration.no_proxy_hosts, no_proxy_string); -} + ASSERT_EQ(http_configuration.host, http_proxy_server.getHost()); + ASSERT_EQ(http_configuration.port, http_proxy_server.getPort()); + ASSERT_EQ(http_configuration.protocol, ProxyConfiguration::protocolFromString(http_proxy_server.getScheme())); + ASSERT_EQ(http_configuration.no_proxy_hosts, poco_no_proxy_regex); -TEST(EnvironmentProxyConfigurationResolver, TestHTTPNoEnv) -{ - EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTP); + EnvironmentProxyConfigurationResolver https_resolver(ProxyConfiguration::Protocol::HTTPS); - auto configuration = resolver.resolve(); + auto https_configuration = https_resolver.resolve(); - ASSERT_EQ(configuration.host, ""); - ASSERT_EQ(configuration.protocol, ProxyConfiguration::Protocol::HTTP); - ASSERT_EQ(configuration.port, 0u); - ASSERT_TRUE(configuration.no_proxy_hosts.empty()); -} - -TEST(EnvironmentProxyConfigurationResolver, TestHTTPs) -{ - EnvironmentProxySetter setter({}, https_proxy_server); - - EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTPS); - - auto configuration = resolver.resolve(); - - ASSERT_EQ(configuration.host, https_proxy_server.getHost()); - ASSERT_EQ(configuration.port, https_proxy_server.getPort()); - ASSERT_EQ(configuration.protocol, ProxyConfiguration::protocolFromString(https_proxy_server.getScheme())); -} - -TEST(EnvironmentProxyConfigurationResolver, TestHTTPsNoEnv) -{ - EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTPS); - - auto configuration = resolver.resolve(); - - ASSERT_EQ(configuration.host, ""); - ASSERT_EQ(configuration.protocol, ProxyConfiguration::Protocol::HTTP); - ASSERT_EQ(configuration.port, 0u); -} - -TEST(EnvironmentProxyConfigurationResolver, TestHTTPsOverHTTPTunnelingDisabled) -{ - // use http proxy for https, this would use connect protocol by default - EnvironmentProxySetter setter({}, http_proxy_server); - - bool disable_tunneling_for_https_requests_over_http_proxy = true; - - EnvironmentProxyConfigurationResolver resolver( - ProxyConfiguration::Protocol::HTTPS, disable_tunneling_for_https_requests_over_http_proxy); - - auto configuration = resolver.resolve(); - - ASSERT_EQ(configuration.host, http_proxy_server.getHost()); - ASSERT_EQ(configuration.port, http_proxy_server.getPort()); - ASSERT_EQ(configuration.protocol, ProxyConfiguration::protocolFromString(http_proxy_server.getScheme())); - ASSERT_EQ(configuration.tunneling, false); + ASSERT_EQ(https_configuration.host, https_proxy_server.getHost()); + ASSERT_EQ(https_configuration.port, https_proxy_server.getPort()); + ASSERT_EQ(https_configuration.protocol, ProxyConfiguration::protocolFromString(https_proxy_server.getScheme())); + ASSERT_EQ(https_configuration.no_proxy_hosts, poco_no_proxy_regex); } } From d99f8b4acc3991cbcb8c269fc27773146d47c6cf Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 5 Jun 2024 20:35:22 +0330 Subject: [PATCH 302/856] Added Named collections for accessing Kafka --- docs/en/operations/named-collections.md | 56 +++++++++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index c9d94dd95ee..5ba9029152e 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -443,3 +443,59 @@ SELECT dictGet('dict', 'b', 1); │ a │ └─────────────────────────┘ ``` + +## Named collections for accessing Kafka + +The description of parameters see [Kafka](../engines/table-engines/integrations/kafka.md). + +### DDL example + +```sql +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka SETTINGS kafka_broker_list = 'localhost:9092', + kafka_topic_list = 'kafka_topic', + kafka_group_name = 'consumer_group', + kafka_format = 'JSONEachRow', + kafka_max_block_size = 1048576; +``` +### XML example + +```xml + + + + localhost:9092 + kafka_topic + consumer_group + JSONEachRow + 1048576 + + + +``` + +### kafka Table, named collection examples + +Both of the following examples use the same named collection `my_kafka_cluster`: + + +```sql +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka(my_kafka_cluster) + + +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka(my_kafka_cluster) +SETTINGS kafka_num_consumers = 4, + kafka_thread_per_consumer = 1; + +``` From 8bbf46ce2fe7f3d4e6c65b3268e44bb29642eb0e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Jun 2024 17:15:48 +0000 Subject: [PATCH 303/856] Trying to fix a test. --- src/Interpreters/ActionsDAG.cpp | 42 ++++++++++--------- src/Interpreters/ActionsDAG.h | 2 +- .../Optimizations/optimizePrewhere.cpp | 2 +- 3 files changed, 25 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 6915f82d067..840aff9ebfb 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1704,7 +1704,7 @@ void ActionsDAG::mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs } } -ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split_nodes, bool create_split_nodes_mapping) const +ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split_nodes, bool create_split_nodes_mapping, bool avoid_duplicate_inputs) const { /// Split DAG into two parts. /// (first_nodes, first_outputs) is a part which will have split_list in result. @@ -1722,8 +1722,9 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split /// It's allowed for DAG but may break Block invariant 'columns with identical name must have identical structure'. std::unordered_set duplicate_inputs; size_t duplicate_counter = 0; - for (const auto * input : inputs) - duplicate_inputs.insert(input->result_name); + if (avoid_duplicate_inputs) + for (const auto * input : inputs) + duplicate_inputs.insert(input->result_name); struct Frame { @@ -1896,25 +1897,28 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split { auto & cur = data[input]; - bool is_name_updated = false; - while (!duplicate_inputs.insert(cur.to_first->result_name).second) + if (avoid_duplicate_inputs) { - is_name_updated = true; - cur.to_first->result_name = fmt::format("{}_{}", input->result_name, duplicate_counter); - ++duplicate_counter; - } + bool is_name_updated = false; + while (!duplicate_inputs.insert(cur.to_first->result_name).second) + { + is_name_updated = true; + cur.to_first->result_name = fmt::format("{}_{}", input->result_name, duplicate_counter); + ++duplicate_counter; + } - if (is_name_updated) - { - Node input_node; - input_node.type = ActionType::INPUT; - input_node.result_type = cur.to_first->result_type; - input_node.result_name = cur.to_first->result_name; + if (is_name_updated) + { + Node input_node; + input_node.type = ActionType::INPUT; + input_node.result_type = cur.to_first->result_type; + input_node.result_name = cur.to_first->result_name; - auto * new_input = &second_nodes.emplace_back(std::move(input_node)); - cur.to_second->type = ActionType::ALIAS; - cur.to_second->children = {new_input}; - cur.to_second = new_input; + auto * new_input = &second_nodes.emplace_back(std::move(input_node)); + cur.to_second->type = ActionType::ALIAS; + cur.to_second->children = {new_input}; + cur.to_second = new_input; + } } second_inputs.push_back(cur.to_second); diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 8c0e3f0e576..7c6753ac9de 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -343,7 +343,7 @@ public: /// initial DAG : (a, b, c, d, e) -> (w, x, y, z) | 1 a 2 b 3 c 4 d 5 e 6 -> 1 2 3 4 5 6 w x y z /// split (first) : (a, c, d) -> (i, j, k, w, y) | 1 a 2 b 3 c 4 d 5 e 6 -> 1 2 b 3 4 5 e 6 i j k w y /// split (second) : (i, j, k, y, b, e) -> (x, y, z) | 1 2 b 3 4 5 e 6 i j k w y -> 1 2 3 4 5 6 w x y z - SplitResult split(std::unordered_set split_nodes, bool create_split_nodes_mapping = false) const; + SplitResult split(std::unordered_set split_nodes, bool create_split_nodes_mapping = false, bool avoid_duplicate_inputs = false) const; /// Splits actions into two parts. Returned first half may be swapped with ARRAY JOIN. SplitResult splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index fbd9b451ddc..74da7b8190f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -118,7 +118,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) outputs.resize(size); } - auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true); + auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true, true); /// This is the leak of abstraction. /// Splited actions may have inputs which are needed only for PREWHERE. From 966b08f192928cb799c12e18a62b6fa6a8dea2ad Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jun 2024 19:26:20 +0200 Subject: [PATCH 304/856] Fix segfault for a very tricky case --- src/Storages/TTLDescription.cpp | 12 +++-- .../__init__.py | 1 + .../configs/enable_parallel_replicas.xml | 11 ++++ .../configs/node1_macro.xml | 6 +++ .../test.py | 51 +++++++++++++++++++ 5 files changed, 78 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py create mode 100644 tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml create mode 100644 tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml create mode 100644 tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 6e7ea32ee59..f831465277d 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -166,8 +166,14 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType { ExpressionAndSets result; auto ttl_string = queryToString(ast); - auto syntax_analyzer_result = TreeRewriter(context).analyze(ast, columns); - ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context); + auto context_copy = Context::createCopy(context); + /// FIXME All code here will work with old analyzer, however for TTL + /// with subqueries it's possible that new analyzer will be enabled in ::read method + /// of underlying storage when all other parts of infra are not ready for it + /// (built with old analyzer). + context_copy->setSetting("allow_experimental_analyzer", Field{0}); + auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns); + ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy); auto dag = analyzer.getActionsDAG(false); const auto * col = &dag->findInOutputs(ast->getColumnName()); @@ -177,7 +183,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType dag->getOutputs() = {col}; dag->removeUnusedActions(); - result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context)); + result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context_copy)); result.sets = analyzer.getPreparedSets(); return result; diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml new file mode 100644 index 00000000000..c654074740a --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml @@ -0,0 +1,11 @@ + + + + 1 + 1 + default + 100 + 0 + + + diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml new file mode 100644 index 00000000000..86e392f09ca --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml @@ -0,0 +1,6 @@ + + + node1 + default + + diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py new file mode 100644 index 00000000000..1c6d15d9c7b --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py @@ -0,0 +1,51 @@ +#!/usr/bin/env python3 + +import logging +import random +import string +import time + +import pytest +from multiprocessing.dummy import Pool +from helpers.cluster import ClickHouseCluster +import minio + + +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.add_instance( + "node1", + main_configs=["configs/node1_macro.xml"], + user_configs=[ + "configs/enable_parallel_replicas.xml", + ], + with_minio=True, + with_zookeeper=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_replicated_db_and_ttl(started_cluster): + node1 = cluster.instances["node1"] + node1.query("DROP DATABASE default") + node1.query("CREATE DATABASE default ENGINE Replicated('/replicated')") + + node1.query( + "CREATE TABLE 02908_main (a UInt32) ENGINE = ReplicatedMergeTree ORDER BY a" + ) + node1.query( + "CREATE TABLE 02908_dependent (a UInt32, ts DateTime) ENGINE = ReplicatedMergeTree ORDER BY a TTL ts + 1 WHERE a IN (SELECT a FROM 02908_main)" + ) + + node1.query("INSERT INTO 02908_main VALUES (1)") + node1.query("INSERT INTO 02908_dependent VALUES (1, now())") + + node1.query("SELECT * FROM 02908_dependent") From 46f14046e4e410f67ef7b436562a03c5bb605a98 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 5 Jun 2024 21:34:53 +0330 Subject: [PATCH 305/856] Added example on how to create named collection for kafka using ddl --- docs/en/operations/named-collections.md | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 5ba9029152e..9b9b682481d 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -451,15 +451,13 @@ The description of parameters see [Kafka](../engines/table-engines/integrations/ ### DDL example ```sql -CREATE TABLE queue( - timestamp UInt64, - level String, - message String -)ENGINE = Kafka SETTINGS kafka_broker_list = 'localhost:9092', - kafka_topic_list = 'kafka_topic', - kafka_group_name = 'consumer_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1048576; +CREATE NAMED COLLECTION my_kafka_cluster AS +kafka_broker_list = 'localhost:9092', +kafka_topic_list = 'kafka_topic', +kafka_group_name = 'consumer_group', +kafka_format = 'JSONEachRow', +kafka_max_block_size = '1048576'; + ``` ### XML example From 927720a02033239cb71568008c0a90f5e5482fc0 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:13 +0330 Subject: [PATCH 306/856] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 9b9b682481d..12d68b266c6 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -488,11 +488,13 @@ CREATE TABLE queue( )ENGINE = Kafka(my_kafka_cluster) -CREATE TABLE queue( +CREATE TABLE queue +( timestamp UInt64, level String, message String -)ENGINE = Kafka(my_kafka_cluster) +) +ENGINE = Kafka(my_kafka_cluster) SETTINGS kafka_num_consumers = 4, kafka_thread_per_consumer = 1; From 0316fa701fcaf2a81360fa0c7f16e4e83942d22d Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:23 +0330 Subject: [PATCH 307/856] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 12d68b266c6..4a3d303c114 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -475,7 +475,7 @@ kafka_max_block_size = '1048576'; ``` -### kafka Table, named collection examples +### Example of using named collections with a Kafka table Both of the following examples use the same named collection `my_kafka_cluster`: From 082efe2668bac79c13642ef4f9ea103e34fc0b62 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:28 +0330 Subject: [PATCH 308/856] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 4a3d303c114..a563ed626fe 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -481,11 +481,13 @@ Both of the following examples use the same named collection `my_kafka_cluster`: ```sql -CREATE TABLE queue( +CREATE TABLE queue +( timestamp UInt64, level String, message String -)ENGINE = Kafka(my_kafka_cluster) +) +ENGINE = Kafka(my_kafka_cluster) CREATE TABLE queue From 8aea0c3fecf852cbed7c2dd55f98777fcb71e97d Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:19 +0330 Subject: [PATCH 309/856] Omitted extra spaces Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index a563ed626fe..0438fcae3b2 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -465,11 +465,11 @@ kafka_max_block_size = '1048576'; - localhost:9092 - kafka_topic - consumer_group - JSONEachRow - 1048576 + localhost:9092 + kafka_topic + consumer_group + JSONEachRow + 1048576 From 844a94383b7473d2174075257c3a99fa4a54c978 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:36 +0330 Subject: [PATCH 310/856] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 0438fcae3b2..734b8dbe24a 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -489,7 +489,6 @@ CREATE TABLE queue ) ENGINE = Kafka(my_kafka_cluster) - CREATE TABLE queue ( timestamp UInt64, From 007a8ca32edba3b83f32ddffd95b4c2a13f78069 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:45 +0330 Subject: [PATCH 311/856] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 734b8dbe24a..1d261d935af 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -498,5 +498,4 @@ CREATE TABLE queue ENGINE = Kafka(my_kafka_cluster) SETTINGS kafka_num_consumers = 4, kafka_thread_per_consumer = 1; - ``` From 946a5913b10821496658a9083a82866855a82123 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Jun 2024 15:46:11 -0300 Subject: [PATCH 312/856] return empty if uri is empty --- src/Common/EnvironmentProxyConfigurationResolver.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index bf062b8d9ba..fff2d354e3a 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -55,6 +55,11 @@ namespace const std::string & no_proxy_hosts_string, bool disable_tunneling_for_https_requests_over_http_proxy) { + if (uri.empty()) + { + return {}; + } + const auto & host = uri.getHost(); const auto & scheme = uri.getScheme(); const auto port = uri.getPort(); From 42097398b38cc73fe3a99cdb515c138f665f1a5d Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 28 May 2024 16:33:24 +0200 Subject: [PATCH 313/856] Do not remove trailing space in *.reference test files (by modern IDEs) by adding it to .editorconfig --- .editorconfig | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.editorconfig b/.editorconfig index 8ac06debb5a..8ecaf9b0267 100644 --- a/.editorconfig +++ b/.editorconfig @@ -19,3 +19,7 @@ charset = utf-8 indent_style = space indent_size = 4 trim_trailing_whitespace = true + +# Some SQL results have trailing whitespace which is removed by IDEs +[tests/queries/**.reference] +trim_trailing_whitespace = false From 54bc5d12effe75708774d7a70c7b94296cf3f771 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 28 May 2024 16:40:43 +0200 Subject: [PATCH 314/856] Add randomization of new settings in clickhouse-test prefer_external_sort_block_bytes cross_join_min_rows_to_compress cross_join_min_bytes_to_compress max_parsing_threads min_external_table_block_size_bytes --- tests/clickhouse-test | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index af203563d58..36870d59c3a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -808,10 +808,10 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "max_bytes_before_external_sort": threshold_generator( - 0.3, 0.5, 1, 10 * 1024 * 1024 * 1024 + 0.3, 0.5, 0, 10 * 1024 * 1024 * 1024 ), "max_bytes_before_external_group_by": threshold_generator( - 0.3, 0.5, 1, 10 * 1024 * 1024 * 1024 + 0.3, 0.5, 0, 10 * 1024 * 1024 * 1024 ), "max_bytes_before_remerge_sort": lambda: random.randint(1, 3000000000), "min_compress_block_size": lambda: random.randint(1, 1048576 * 3), @@ -850,6 +850,11 @@ class SettingsRandomizer: "merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability": lambda: round( random.random(), 2 ), + "prefer_external_sort_block_bytes": lambda: random.choice([0, 1, 100000000]), + "cross_join_min_rows_to_compress": lambda: random.choice([0, 1, 100000000]), + "cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]), + "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), + "max_parsing_threads": lambda: random.choice([0, 1, 10]), } @staticmethod From d59ccbdb5040f8623600c3c5d2c113adb5c64ba4 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 28 May 2024 18:33:08 +0200 Subject: [PATCH 315/856] stateless tests: Add test for unequal JOINs --- ...006_join_on_inequal_expression_2.reference | 290 ++++++++++++++++++ .../03006_join_on_inequal_expression_2.sql.j2 | 82 +++++ ...006_join_on_inequal_expression_3.reference | 90 ++++++ .../03006_join_on_inequal_expression_3.sql.j2 | 44 +++ ...006_join_on_inequal_expression_4.reference | 123 ++++++++ .../03006_join_on_inequal_expression_4.sql.j2 | 47 +++ 6 files changed, 676 insertions(+) create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference new file mode 100644 index 00000000000..bab1fbd050f --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference @@ -0,0 +1,290 @@ +-- { echoOn } +-- inequality operation +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- BETWEEN +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- Stupid condition +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- Window functions with stupid condition +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 15 3 +2 15 beta 2 10 beta 15 3 +3 20 gamma 0 0 15 3 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 15 3 +2 15 beta 2 10 beta 15 3 +3 20 gamma 0 0 15 3 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 12.5 3 +2 15 beta 2 10 beta 12.5 3 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 12.5 3 +2 15 beta 2 10 beta 12.5 3 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 8.333333333333334 7 +1 10 alpha 1 5 ALPHA 8.333333333333334 7 +2 15 beta 2 10 beta 8.333333333333334 7 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 8.333333333333334 7 +1 10 alpha 1 5 ALPHA 8.333333333333334 7 +2 15 beta 2 10 beta 8.333333333333334 7 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 11.25 7 +1 10 alpha 1 5 ALPHA 11.25 7 +2 15 beta 2 10 beta 11.25 7 +3 20 gamma 0 0 11.25 7 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 11.25 7 +1 10 alpha 1 5 ALPHA 11.25 7 +2 15 beta 2 10 beta 11.25 7 +3 20 gamma 0 0 11.25 7 +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 new file mode 100644 index 00000000000..f15fced161c --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 @@ -0,0 +1,82 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, 'gamma'); +INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta'); + + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_use_nulls=0; +-- { echoOn } +-- inequality operation +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- BETWEEN +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- Stupid condition +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- Window functions with stupid condition +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 {{ join_type }} JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference new file mode 100644 index 00000000000..1ec6f911897 --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference @@ -0,0 +1,90 @@ +-- { echoOn } + +-- Support for query lower +SET join_algorithm='hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- Subquery JOIN +SET join_algorithm='hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) LEFT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) LEFT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) INNER JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) INNER JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) RIGHT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) RIGHT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) FULL JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +0 0 2 10 beta 0 0 +0 0 4 25 delta 0 0 +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) FULL JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +0 0 2 10 beta 0 0 +0 0 4 25 delta 0 0 +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 new file mode 100644 index 00000000000..a97153ce3aa --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 @@ -0,0 +1,44 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, 'gamma'); +INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta'); + + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_use_nulls=0; +-- { echoOn } + +-- Support for query lower +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT * FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + + +-- Subquery JOIN +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT * FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) {{ join_type }} JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference new file mode 100644 index 00000000000..f2a5901077a --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference @@ -0,0 +1,123 @@ +-- { echoOn } + +-- These queries work +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 new file mode 100644 index 00000000000..59df0b2abc2 --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 @@ -0,0 +1,47 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32 +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32 +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a) VALUES (1, 10), (2, 15), (3, 20); +INSERT INTO t2 (key, a) VALUES (1, 5), (2, 10), (4, 25); + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_algorithm='hash'; +-- { echoOn } + +-- These queries work +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- LOGICAL_ERROR Not-ready Set is passed as the second argument for function 'in' +-- https://github.com/ClickHouse/ClickHouse/issues/64513 +-- SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; From 103a448e710fc2fa406574b9d29e93e4a0665733 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 29 May 2024 12:11:00 +0200 Subject: [PATCH 316/856] stateless tests: complete parallel_parsing test --- .../0_stateless/01701_parallel_parsing_infinite_segmentation.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh b/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh index 9284348dd62..d2dcd501428 100755 --- a/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh +++ b/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh @@ -7,6 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "create table insert_big_json(a String, b String) engine=MergeTree() order by tuple()"; -python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||: +python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 --max_parsing_threads=2 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||: ${CLICKHOUSE_CLIENT} -q "drop table insert_big_json" From e6bf468e563f45cf228ed98c3ce872de205aebff Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 29 May 2024 12:11:15 +0200 Subject: [PATCH 317/856] stateless tests: add missing order by --- tests/queries/0_stateless/00576_nested_and_prewhere.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00576_nested_and_prewhere.sql b/tests/queries/0_stateless/00576_nested_and_prewhere.sql index 5916e679f1e..f5d9f0d6240 100644 --- a/tests/queries/0_stateless/00576_nested_and_prewhere.sql +++ b/tests/queries/0_stateless/00576_nested_and_prewhere.sql @@ -4,10 +4,10 @@ CREATE TABLE nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeT INSERT INTO nested SELECT number, number % 2, range(number % 10) FROM system.numbers LIMIT 100000; ALTER TABLE nested ADD COLUMN n.b Array(UInt64); -SELECT DISTINCT n.b FROM nested PREWHERE filter; +SELECT DISTINCT n.b FROM nested PREWHERE filter ORDER BY ALL; ALTER TABLE nested ADD COLUMN n.c Array(UInt64) DEFAULT arrayMap(x -> x * 2, n.a); -SELECT DISTINCT n.c FROM nested PREWHERE filter; -SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter; +SELECT DISTINCT n.c FROM nested PREWHERE filter ORDER BY ALL; +SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter ORDER BY ALL; DROP TABLE nested; From fd930971301edfc6f5f199744354ab4f5005beb7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 29 May 2024 04:10:38 +0000 Subject: [PATCH 318/856] Fix writing ORC statistics for unsigned types --- contrib/orc | 2 +- .../Impl/NativeORCBlockInputFormat.cpp | 7 +++- .../Formats/Impl/ORCBlockOutputFormat.cpp | 12 +++--- .../0_stateless/02892_orc_filter_pushdown.sql | 2 +- .../03164_orc_signedness.reference | 41 +++++++++++++++++++ .../0_stateless/03164_orc_signedness.sql | 40 ++++++++++++++++++ 6 files changed, 96 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03164_orc_signedness.reference create mode 100644 tests/queries/0_stateless/03164_orc_signedness.sql diff --git a/contrib/orc b/contrib/orc index e24f2c2a3ca..947cebaf943 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit e24f2c2a3ca0769c96704ab20ad6f512a83ea2ad +Subproject commit 947cebaf9432d708253ac08dc3012daa6b4ede6f diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 0b55f633c6a..dcd5a531b05 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -269,7 +269,12 @@ convertFieldToORCLiteral(const orc::Type & orc_type, const Field & field, DataTy case orc::SHORT: case orc::INT: case orc::LONG: { - /// May throw exception + /// May throw exception. + /// + /// In particular, it'll throw if we request the column as unsigned, like this: + /// SELECT * FROM file('t.orc', ORC, 'x UInt8') WHERE x > 10 + /// We have to reject this, otherwise it would miss values > 127 (because + /// they're treated as negative by ORC). auto val = field.get(); return orc::Literal(val); } diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 1e36c100667..6f543a05fba 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -315,18 +315,20 @@ void ORCBlockOutputFormat::writeColumn( if (null_bytemap) orc_column.hasNulls = true; + /// ORC doesn't have unsigned types, so cast everything to signed and sign-extend to Int64 to + /// make the ORC library calculate min and max correctly. switch (type->getTypeId()) { case TypeIndex::Enum8: [[fallthrough]]; case TypeIndex::Int8: { /// Note: Explicit cast to avoid clang-tidy error: 'signed char' to 'long' conversion; consider casting to 'unsigned char' first. - writeNumbers(orc_column, column, null_bytemap, [](const Int8 & value){ return static_cast(value); }); + writeNumbers(orc_column, column, null_bytemap, [](const Int8 & value){ return Int64(Int8(value)); }); break; } case TypeIndex::UInt8: { - writeNumbers(orc_column, column, null_bytemap, [](const UInt8 & value){ return value; }); + writeNumbers(orc_column, column, null_bytemap, [](const UInt8 & value){ return Int64(Int8(value)); }); break; } case TypeIndex::Enum16: [[fallthrough]]; @@ -338,7 +340,7 @@ void ORCBlockOutputFormat::writeColumn( case TypeIndex::Date: [[fallthrough]]; case TypeIndex::UInt16: { - writeNumbers(orc_column, column, null_bytemap, [](const UInt16 & value){ return value; }); + writeNumbers(orc_column, column, null_bytemap, [](const UInt16 & value){ return Int64(Int16(value)); }); break; } case TypeIndex::Date32: [[fallthrough]]; @@ -349,12 +351,12 @@ void ORCBlockOutputFormat::writeColumn( } case TypeIndex::UInt32: { - writeNumbers(orc_column, column, null_bytemap, [](const UInt32 & value){ return value; }); + writeNumbers(orc_column, column, null_bytemap, [](const UInt32 & value){ return Int64(Int32(value)); }); break; } case TypeIndex::IPv4: { - writeNumbers(orc_column, column, null_bytemap, [](const IPv4 & value){ return value.toUnderType(); }); + writeNumbers(orc_column, column, null_bytemap, [](const IPv4 & value){ return Int64(Int32(value.toUnderType())); }); break; } case TypeIndex::Int64: diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql index f9aa7696ac6..f1d1ba12570 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-cpu-aarch64 +-- Tags: no-fasttest, no-parallel set output_format_orc_string_as_string = 1; set output_format_orc_row_index_stride = 100; diff --git a/tests/queries/0_stateless/03164_orc_signedness.reference b/tests/queries/0_stateless/03164_orc_signedness.reference new file mode 100644 index 00000000000..3ee822a94c1 --- /dev/null +++ b/tests/queries/0_stateless/03164_orc_signedness.reference @@ -0,0 +1,41 @@ +-- { echoOn } +select x from file('i8.orc') where indexHint(x = -128); +-128 +select x from file('i8.orc') where indexHint(x = 128); +select x from file('u8.orc') where indexHint(x = -128); +-128 +select x from file('u8.orc') where indexHint(x = 128); +select x from file('i16.orc') where indexHint(x = -32768); +-32768 +select x from file('i16.orc') where indexHint(x = 32768); +select x from file('u16.orc') where indexHint(x = -32768); +-32768 +select x from file('u16.orc') where indexHint(x = 32768); +select x from file('i32.orc') where indexHint(x = -2147483648); +-2147483648 +select x from file('i32.orc') where indexHint(x = 2147483648); +select x from file('u32.orc') where indexHint(x = -2147483648); +-2147483648 +select x from file('u32.orc') where indexHint(x = 2147483648); +select x from file('i64.orc') where indexHint(x = -9223372036854775808); +-9223372036854775808 +select x from file('i64.orc') where indexHint(x = 9223372036854775808); +-9223372036854775808 +select x from file('u64.orc') where indexHint(x = -9223372036854775808); +-9223372036854775808 +select x from file('u64.orc') where indexHint(x = 9223372036854775808); +-9223372036854775808 +select x from file('u8.orc', ORC, 'x UInt8') where indexHint(x > 10); +128 +select x from file('u8.orc', ORC, 'x UInt64') where indexHint(x > 10); +18446744073709551488 +select x from file('u16.orc', ORC, 'x UInt16') where indexHint(x > 10); +32768 +select x from file('u16.orc', ORC, 'x UInt64') where indexHint(x > 10); +18446744073709518848 +select x from file('u32.orc', ORC, 'x UInt32') where indexHint(x > 10); +2147483648 +select x from file('u32.orc', ORC, 'x UInt64') where indexHint(x > 10); +18446744071562067968 +select x from file('u64.orc', ORC, 'x UInt64') where indexHint(x > 10); +9223372036854775808 diff --git a/tests/queries/0_stateless/03164_orc_signedness.sql b/tests/queries/0_stateless/03164_orc_signedness.sql new file mode 100644 index 00000000000..ced99c7dca7 --- /dev/null +++ b/tests/queries/0_stateless/03164_orc_signedness.sql @@ -0,0 +1,40 @@ +set input_format_orc_filter_push_down = 1; +set engine_file_truncate_on_insert = 1; + +insert into function file('i8.orc') select materialize(-128)::Int8 as x; +insert into function file('u8.orc') select materialize(128)::UInt8 as x; +insert into function file('i16.orc') select materialize(-32768)::Int16 as x; +insert into function file('u16.orc') select materialize(32768)::UInt16 as x; +insert into function file('i32.orc') select materialize(-2147483648)::Int32 as x; +insert into function file('u32.orc') select materialize(2147483648)::UInt32 as x; +insert into function file('i64.orc') select materialize(-9223372036854775808)::Int64 as x; +insert into function file('u64.orc') select materialize(9223372036854775808)::UInt64 as x; + +-- { echoOn } +select x from file('i8.orc') where indexHint(x = -128); +select x from file('i8.orc') where indexHint(x = 128); +select x from file('u8.orc') where indexHint(x = -128); +select x from file('u8.orc') where indexHint(x = 128); + +select x from file('i16.orc') where indexHint(x = -32768); +select x from file('i16.orc') where indexHint(x = 32768); +select x from file('u16.orc') where indexHint(x = -32768); +select x from file('u16.orc') where indexHint(x = 32768); + +select x from file('i32.orc') where indexHint(x = -2147483648); +select x from file('i32.orc') where indexHint(x = 2147483648); +select x from file('u32.orc') where indexHint(x = -2147483648); +select x from file('u32.orc') where indexHint(x = 2147483648); + +select x from file('i64.orc') where indexHint(x = -9223372036854775808); +select x from file('i64.orc') where indexHint(x = 9223372036854775808); +select x from file('u64.orc') where indexHint(x = -9223372036854775808); +select x from file('u64.orc') where indexHint(x = 9223372036854775808); + +select x from file('u8.orc', ORC, 'x UInt8') where indexHint(x > 10); +select x from file('u8.orc', ORC, 'x UInt64') where indexHint(x > 10); +select x from file('u16.orc', ORC, 'x UInt16') where indexHint(x > 10); +select x from file('u16.orc', ORC, 'x UInt64') where indexHint(x > 10); +select x from file('u32.orc', ORC, 'x UInt32') where indexHint(x > 10); +select x from file('u32.orc', ORC, 'x UInt64') where indexHint(x > 10); +select x from file('u64.orc', ORC, 'x UInt64') where indexHint(x > 10); From b300af350349b5bbefaa4036eed3d7c5d5a102d8 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 29 May 2024 05:00:03 +0000 Subject: [PATCH 319/856] no-fasttest --- tests/queries/0_stateless/03164_orc_signedness.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03164_orc_signedness.sql b/tests/queries/0_stateless/03164_orc_signedness.sql index ced99c7dca7..ae2d0428ca5 100644 --- a/tests/queries/0_stateless/03164_orc_signedness.sql +++ b/tests/queries/0_stateless/03164_orc_signedness.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest, no-parallel + set input_format_orc_filter_push_down = 1; set engine_file_truncate_on_insert = 1; From 40a3708c8f139c28f72e10f916c45a21ad235e28 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 31 May 2024 19:55:13 +0000 Subject: [PATCH 320/856] Fix test --- .../02892_orc_filter_pushdown.reference | 46 +++++++++---------- .../0_stateless/02892_orc_filter_pushdown.sql | 34 ++++++-------- 2 files changed, 35 insertions(+), 45 deletions(-) diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference index 9059b403a34..e6c2e9b2b57 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference @@ -1,8 +1,4 @@ number Nullable(Int64) -u8 Nullable(Int8) -u16 Nullable(Int16) -u32 Nullable(Int32) -u64 Nullable(Int64) i8 Nullable(Int8) i16 Nullable(Int16) i32 Nullable(Int32) @@ -22,34 +18,34 @@ d64 Nullable(Decimal(18, 10)) d128 Nullable(Decimal(38, 20)) -- Go over all types individually -- { echoOn } -select count(), sum(number) from file('02892.orc') where indexHint(u8 in (10, 15, 250)); -800 4229600 -select count(1), min(u8), max(u8) from file('02892.orc') where u8 in (10, 15, 250); -66 10 15 +select count(), sum(number) from file('02892.orc') where indexHint(i8 in (10, 15, -6)); +1100 5744450 +select count(1), min(i8), max(i8) from file('02892.orc') where i8 in (10, 15, -6); +99 -6 15 select count(), sum(number) from file('02892.orc') where indexHint(i8 between -3 and 2); 1000 4999500 select count(1), min(i8), max(i8) from file('02892.orc') where i8 between -3 and 2; 208 -3 2 -select count(), sum(number) from file('02892.orc') where indexHint(u16 between 4000 and 61000 or u16 == 42); -1800 6479100 -select count(1), min(u16), max(u16) from file('02892.orc') where u16 between 4000 and 61000 or u16 == 42; +select count(), sum(number) from file('02892.orc') where indexHint(i16 between 4000 and 61000 or i16 == 42); +1200 1099400 +select count(1), min(i16), max(i16) from file('02892.orc') where i16 between 4000 and 61000 or i16 == 42; 1002 42 5000 select count(), sum(number) from file('02892.orc') where indexHint(i16 between -150 and 250); 500 2474750 select count(1), min(i16), max(i16) from file('02892.orc') where i16 between -150 and 250; 401 -150 250 -select count(), sum(number) from file('02892.orc') where indexHint(u32 in (42, 4294966296)); -200 999900 -select count(1), min(u32), max(u32) from file('02892.orc') where u32 in (42, 4294966296); -1 42 42 +select count(), sum(number) from file('02892.orc') where indexHint(i32 in (42, -1000)); +200 1099900 +select count(1), min(i32), max(i32) from file('02892.orc') where i32 in (42, -1000); +2 -1000 42 select count(), sum(number) from file('02892.orc') where indexHint(i32 between -150 and 250); 500 2474750 select count(1), min(i32), max(i32) from file('02892.orc') where i32 between -150 and 250; 401 -150 250 -select count(), sum(number) from file('02892.orc') where indexHint(u64 in (42, 18446744073709550616)); -100 494950 -select count(1), min(u64), max(u64) from file('02892.orc') where u64 in (42, 18446744073709550616); -1 42 42 +select count(), sum(number) from file('02892.orc') where indexHint(i64 in (42, -1000)); +200 1099900 +select count(1), min(i64), max(i64) from file('02892.orc') where i64 in (42, -1000); +2 -1000 42 select count(), sum(number) from file('02892.orc') where indexHint(i64 between -150 and 250); 500 2474750 select count(1), min(i64), max(i64) from file('02892.orc') where i64 between -150 and 250; @@ -111,21 +107,21 @@ select count(), sum(number) from file('02892.orc') where indexHint(0); 0 \N select count(), min(number), max(number) from file('02892.orc') where indexHint(0); 0 \N \N -select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or u64 == 2000); +select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or i64 == 2000); 300 1204850 -select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or u64 == 2000); +select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or i64 == 2000); 12 2000 999 select count(), sum(number) from file('02892.orc') where indexHint(s like 'z%'); 0 \N select count(), min(s), max(s) from file('02892.orc') where (s like 'z%'); 0 \N \N -select count(), sum(number) from file('02892.orc') where indexHint(u8 == 10 or 1 == 1); +select count(), sum(number) from file('02892.orc') where indexHint(i8 == 10 or 1 == 1); 10000 49995000 -select count(), min(u8), max(u8) from file('02892.orc') where (u8 == 10 or 1 == 1); +select count(), min(i8), max(i8) from file('02892.orc') where (i8 == 10 or 1 == 1); 10000 -128 127 -select count(), sum(number) from file('02892.orc') where indexHint(u8 < 0); +select count(), sum(number) from file('02892.orc') where indexHint(i8 < 0); 5300 26042350 -select count(), min(u8), max(u8) from file('02892.orc') where (u8 < 0); +select count(), min(i8), max(i8) from file('02892.orc') where (i8 < 0); 5001 -128 -1 -- { echoOn } select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null is NULL); diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql index f1d1ba12570..e3736de6a17 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql @@ -16,15 +16,9 @@ SET session_timezone = 'UTC'; -- Try all the types. insert into function file('02892.orc') - -- Use negative numbers to test sign extension for signed types and lack of sign extension for - -- unsigned types. with 5000 - number as n select number, - intDiv(n, 11)::UInt8 as u8, - n::UInt16 u16, - n::UInt32 as u32, - n::UInt64 as u64, intDiv(n, 11)::Int8 as i8, n::Int16 i16, n::Int32 as i32, @@ -50,26 +44,26 @@ desc file('02892.orc'); -- Go over all types individually -- { echoOn } -select count(), sum(number) from file('02892.orc') where indexHint(u8 in (10, 15, 250)); -select count(1), min(u8), max(u8) from file('02892.orc') where u8 in (10, 15, 250); +select count(), sum(number) from file('02892.orc') where indexHint(i8 in (10, 15, -6)); +select count(1), min(i8), max(i8) from file('02892.orc') where i8 in (10, 15, -6); select count(), sum(number) from file('02892.orc') where indexHint(i8 between -3 and 2); select count(1), min(i8), max(i8) from file('02892.orc') where i8 between -3 and 2; -select count(), sum(number) from file('02892.orc') where indexHint(u16 between 4000 and 61000 or u16 == 42); -select count(1), min(u16), max(u16) from file('02892.orc') where u16 between 4000 and 61000 or u16 == 42; +select count(), sum(number) from file('02892.orc') where indexHint(i16 between 4000 and 61000 or i16 == 42); +select count(1), min(i16), max(i16) from file('02892.orc') where i16 between 4000 and 61000 or i16 == 42; select count(), sum(number) from file('02892.orc') where indexHint(i16 between -150 and 250); select count(1), min(i16), max(i16) from file('02892.orc') where i16 between -150 and 250; -select count(), sum(number) from file('02892.orc') where indexHint(u32 in (42, 4294966296)); -select count(1), min(u32), max(u32) from file('02892.orc') where u32 in (42, 4294966296); +select count(), sum(number) from file('02892.orc') where indexHint(i32 in (42, -1000)); +select count(1), min(i32), max(i32) from file('02892.orc') where i32 in (42, -1000); select count(), sum(number) from file('02892.orc') where indexHint(i32 between -150 and 250); select count(1), min(i32), max(i32) from file('02892.orc') where i32 between -150 and 250; -select count(), sum(number) from file('02892.orc') where indexHint(u64 in (42, 18446744073709550616)); -select count(1), min(u64), max(u64) from file('02892.orc') where u64 in (42, 18446744073709550616); +select count(), sum(number) from file('02892.orc') where indexHint(i64 in (42, -1000)); +select count(1), min(i64), max(i64) from file('02892.orc') where i64 in (42, -1000); select count(), sum(number) from file('02892.orc') where indexHint(i64 between -150 and 250); select count(1), min(i64), max(i64) from file('02892.orc') where i64 between -150 and 250; @@ -117,17 +111,17 @@ select count(1), min(d128), max(128) from file('02892.orc') where (d128 between select count(), sum(number) from file('02892.orc') where indexHint(0); select count(), min(number), max(number) from file('02892.orc') where indexHint(0); -select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or u64 == 2000); -select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or u64 == 2000); +select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or i64 == 2000); +select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or i64 == 2000); select count(), sum(number) from file('02892.orc') where indexHint(s like 'z%'); select count(), min(s), max(s) from file('02892.orc') where (s like 'z%'); -select count(), sum(number) from file('02892.orc') where indexHint(u8 == 10 or 1 == 1); -select count(), min(u8), max(u8) from file('02892.orc') where (u8 == 10 or 1 == 1); +select count(), sum(number) from file('02892.orc') where indexHint(i8 == 10 or 1 == 1); +select count(), min(i8), max(i8) from file('02892.orc') where (i8 == 10 or 1 == 1); -select count(), sum(number) from file('02892.orc') where indexHint(u8 < 0); -select count(), min(u8), max(u8) from file('02892.orc') where (u8 < 0); +select count(), sum(number) from file('02892.orc') where indexHint(i8 < 0); +select count(), min(i8), max(i8) from file('02892.orc') where (i8 < 0); -- { echoOff } -- Nullable and LowCardinality. From f264c8ed590a9a790fb3b4abb85f6949b10fdc2d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 10:00:32 +0200 Subject: [PATCH 321/856] Set default for MaxSimultaneousAllocations --- src/Common/GWPAsan.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 2bdf418e152..4c6c8c7e9cc 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -53,6 +53,9 @@ static bool guarded_alloc_initialized = [] gwp_asan::options::initOptions(env_options_raw, printString); auto & opts = gwp_asan::options::getOptions(); + if (!env_options_raw || !std::string_view{env_options_raw}.contains("MaxSimultaneousAllocations")) + opts.MaxSimultaneousAllocations = 1024; + opts.Backtrace = getBackTrace; GuardedAlloc.init(opts); From f9ac18d74a80fe35e24baa8f896be7e891280888 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Jun 2024 08:16:11 +0000 Subject: [PATCH 322/856] better description for history of a setting changes --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 8b157517263..b47b3a02466 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -95,7 +95,7 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"min_untracked_memory", 4_MiB, 4_KiB, "A new setting."}, + {"min_untracked_memory", 4_MiB, 4_KiB, "A new setting to enable more accurate memory tracking."}, }}, {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, From 45fda3fd3990b8047290af1b226d857cb47608ed Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Jun 2024 08:20:52 +0000 Subject: [PATCH 323/856] use Mi suffix to make things obvious --- tests/integration/test_failed_async_inserts/test.py | 2 +- tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_failed_async_inserts/test.py b/tests/integration/test_failed_async_inserts/test.py index 2bb56b250ea..e7e504e565f 100644 --- a/tests/integration/test_failed_async_inserts/test.py +++ b/tests/integration/test_failed_async_inserts/test.py @@ -45,7 +45,7 @@ def test_failed_async_inserts(started_cluster): ignore_error=True, ) - select_query = "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = 4194304" + select_query = "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = '4Mi'" assert node.query(select_query) == "4\n" diff --git a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql index 92ef928bc2f..de84846c1d7 100644 --- a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql +++ b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql @@ -8,7 +8,7 @@ -- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400 SET use_uncompressed_cache = 0; -SET min_untracked_memory = 4194304; -- 4MiB +SET min_untracked_memory = '4Mi'; -- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements SELECT 'UInt32'; From b9edf204d9bf3b37072f3f2c6051fcc7fd286cfa Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Jun 2024 08:51:22 +0000 Subject: [PATCH 324/856] better --- tests/integration/test_settings_constraints_distributed/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index a1f44af1069..51541721a29 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -137,7 +137,7 @@ def test_select_clamps_settings(): assert ( distributed.query( - query, settings={"max_memory_usage": 1, "min_untracked_memory": 4194304} + query, settings={"max_memory_usage": 1, "min_untracked_memory": 4 * 1024 * 1024} ) == "node1\tmax_memory_usage\t11111111\n" "node1\treadonly\t0\n" From 74897790aa146ff814817912c600734c70990895 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 6 Jun 2024 09:00:35 +0000 Subject: [PATCH 325/856] Automatic style fix --- .../integration/test_settings_constraints_distributed/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index 51541721a29..d29b66b43bb 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -137,7 +137,8 @@ def test_select_clamps_settings(): assert ( distributed.query( - query, settings={"max_memory_usage": 1, "min_untracked_memory": 4 * 1024 * 1024} + query, + settings={"max_memory_usage": 1, "min_untracked_memory": 4 * 1024 * 1024}, ) == "node1\tmax_memory_usage\t11111111\n" "node1\treadonly\t0\n" From 9ce824d41c2323f6556dd6f4015d9ec99272083c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 6 Jun 2024 11:04:38 +0200 Subject: [PATCH 326/856] Addressed review comments --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 27 +++++++++---------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 16bd70cf2e5..a2d21cf49c2 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -132,23 +132,21 @@ void WriteBufferFromAzureBlobStorage::preFinalize() /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (detached_part_data.size() == 1 && block_ids.empty()) + if (block_ids.empty() && detached_part_data.size() == 1 && detached_part_data.front().data_size <= max_single_part_upload_size) { - if (detached_part_data.front().data_size <= max_single_part_upload_size) - { - auto part_data = std::move(detached_part_data.front()); - auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); - execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); - LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); + auto part_data = std::move(detached_part_data.front()); + auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); + execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); + LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); - detached_part_data.pop_front(); - block_ids.clear(); - return; - } + detached_part_data.pop_front(); + return; + } + else + { + writeMultipartUpload(); } - - writeMultipartUpload(); } void WriteBufferFromAzureBlobStorage::finalizeImpl() @@ -295,4 +293,3 @@ void WriteBufferFromAzureBlobStorage::writeMultipartUpload() } #endif - From 1d3cf170534b1ae8382397d8d2672868e237f33f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:11:08 +0200 Subject: [PATCH 327/856] Fix global trace collector --- programs/server/Server.cpp | 99 +++++++++++++++-------------- src/Common/QueryProfiler.cpp | 6 +- src/Interpreters/Context.cpp | 17 ++++- src/Interpreters/Context.h | 2 + src/Interpreters/TraceCollector.cpp | 30 +++++++-- src/Interpreters/TraceCollector.h | 11 +++- 6 files changed, 105 insertions(+), 60 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..f47cd53f9e9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -773,7 +773,51 @@ try LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); #endif - bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); + bool has_trace_collector = false; + /// Disable it if we collect test coverage information, because it will work extremely slow. +#if !WITH_COVERAGE + /// Profilers cannot work reliably with any other libunwind or without PHDR cache. + if (hasPHDRCache() && config().has("trace_log")) + { + has_trace_collector = true; + + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + { + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + } + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + { + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + } + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + { + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + } + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + { + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + } +#endif + + /// Describe multiple reasons when query profiler cannot work. + +#if WITH_COVERAGE + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); +#endif + +#if defined(SANITIZER) + LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" + " when two different stack unwinding methods will interfere with each other."); +#endif + + if (!hasPHDRCache()) + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" + " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will @@ -782,8 +826,12 @@ try server_settings.max_thread_pool_size, server_settings.max_thread_pool_free_size, server_settings.thread_pool_queue_size, - will_have_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, - will_have_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + has_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, + has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + + if (has_trace_collector) + global_context->createTraceCollector(); + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; @@ -1950,52 +1998,9 @@ try LOG_DEBUG(log, "Loaded metadata."); - /// Init trace collector only after trace_log system table was created - /// Disable it if we collect test coverage information, because it will work extremely slow. -#if !WITH_COVERAGE - /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache()) - { + if (has_trace_collector) global_context->initializeTraceCollector(); - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } -#endif - - /// Describe multiple reasons when query profiler cannot work. - -#if WITH_COVERAGE - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); -#endif - -#if defined(SANITIZER) - LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" - " when two different stack unwinding methods will interfere with each other."); -#endif - - if (!hasPHDRCache()) - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" - " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); - #if defined(OS_LINUX) auto tasks_stats_provider = TasksStatsCounters::findBestAvailableProvider(); if (tasks_stats_provider == TasksStatsCounters::MetricsProvider::None) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index c3affbdd968..746010b5462 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -228,9 +228,9 @@ void Timer::cleanup() #endif template -QueryProfilerBase::QueryProfilerBase([[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) - : log(getLogger("QueryProfiler")) - , pause_signal(pause_signal_) +QueryProfilerBase::QueryProfilerBase( + [[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) + : log(getLogger("QueryProfiler")), pause_signal(pause_signal_) { #if defined(SANITIZER) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 06b3adb328d..b1fb6a68618 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -740,12 +740,18 @@ struct ContextSharedPart : boost::noncopyable void initializeTraceCollector(std::shared_ptr trace_log) { - if (!trace_log) - return; + if (!trace_collector.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector needs to be first created before initialization"); + + trace_collector->initialize(trace_log); + } + + void createTraceCollector() + { if (hasTraceCollector()) return; - trace_collector.emplace(std::move(trace_log)); + trace_collector.emplace(); } void addWarningMessage(const String & message) TSA_REQUIRES(mutex) @@ -3891,6 +3897,11 @@ void Context::initializeSystemLogs() }); } +void Context::createTraceCollector() +{ + shared->createTraceCollector(); +} + void Context::initializeTraceCollector() { shared->initializeTraceCollector(getTraceLog()); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87a7baa0469..68f37377926 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1077,6 +1077,8 @@ public: void initializeSystemLogs(); /// Call after initialization before using trace collector. + void createTraceCollector(); + void initializeTraceCollector(); /// Call after unexpected crash happen. diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 8e9c397b7a1..77f70d754c8 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -1,5 +1,4 @@ -#include "TraceCollector.h" - +#include #include #include #include @@ -14,8 +13,12 @@ namespace DB { -TraceCollector::TraceCollector(std::shared_ptr trace_log_) - : trace_log(std::move(trace_log_)) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +TraceCollector::TraceCollector() { TraceSender::pipe.open(); @@ -28,6 +31,23 @@ TraceCollector::TraceCollector(std::shared_ptr trace_log_) thread = ThreadFromGlobalPool(&TraceCollector::run, this); } +void TraceCollector::initialize(std::shared_ptr trace_log_) +{ + if (is_trace_log_initialized) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector is already initialized"); + + trace_log_ptr = trace_log_; + is_trace_log_initialized.store(true, std::memory_order_release); +} + +std::shared_ptr TraceCollector::getTraceLog() +{ + if (!is_trace_log_initialized.load(std::memory_order_acquire)) + return nullptr; + + return trace_log_ptr; +} + void TraceCollector::tryClosePipe() { try @@ -120,7 +140,7 @@ void TraceCollector::run() ProfileEvents::Count increment; readPODBinary(increment, in); - if (trace_log) + if (auto trace_log = getTraceLog()) { // time and time_in_microseconds are both being constructed from the same timespec so that the // times will be equal up to the precision of a second. diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index 382e7511ac6..db1ce286b0e 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -1,4 +1,5 @@ #pragma once +#include #include class StackTrace; @@ -16,11 +17,17 @@ class TraceLog; class TraceCollector { public: - explicit TraceCollector(std::shared_ptr trace_log_); + explicit TraceCollector(); ~TraceCollector(); + void initialize(std::shared_ptr trace_log_); + private: - std::shared_ptr trace_log; + std::shared_ptr getTraceLog(); + + std::atomic is_trace_log_initialized = false; + std::shared_ptr trace_log_ptr; + ThreadFromGlobalPool thread; void tryClosePipe(); From fcee260b259b7750f18389e26debc90de22017a5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:22:48 +0200 Subject: [PATCH 328/856] Update src/Interpreters/TraceCollector.h Co-authored-by: alesapin --- src/Interpreters/TraceCollector.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index db1ce286b0e..c2894394dd0 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -17,7 +17,7 @@ class TraceLog; class TraceCollector { public: - explicit TraceCollector(); + TraceCollector(); ~TraceCollector(); void initialize(std::shared_ptr trace_log_); From bfb1c4c79396bb234b63a0a963fc643af292956d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:29:42 +0200 Subject: [PATCH 329/856] better --- programs/server/Server.cpp | 41 +++++++++++++++----------------------- 1 file changed, 16 insertions(+), 25 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f47cd53f9e9..84ff0e94cef 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -777,31 +777,7 @@ try /// Disable it if we collect test coverage information, because it will work extremely slow. #if !WITH_COVERAGE /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache() && config().has("trace_log")) - { - has_trace_collector = true; - - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } + has_trace_collector = hasPHDRCache() && config().has("trace_log"); #endif /// Describe multiple reasons when query profiler cannot work. @@ -830,8 +806,23 @@ try has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); if (has_trace_collector) + { global_context->createTraceCollector(); + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; From d4294dae0f75f7a8990f0521e9d1e225b5cdfd25 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 6 Jun 2024 11:38:58 +0200 Subject: [PATCH 330/856] Document flameGraph aggregate function --- .../reference/flame_graph.md | 95 +++++++++++++++++++ .../aggregate-functions/reference/index.md | 1 + 2 files changed, 96 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/flame_graph.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md b/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md new file mode 100644 index 00000000000..4aa1cd0c8a8 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md @@ -0,0 +1,95 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/flamegraph +sidebar_position: 110 +--- + +# flameGraph + +Aggregate function which builds a [flamegraph](https://www.brendangregg.com/flamegraphs.html) using the list of stacktraces. Outputs an array of strings which can be used by [flamegraph.pl util](https://github.com/brendangregg/FlameGraph) to render an SVG of the flamegraph. + +## Syntax + +```sql +flameGraph(traces, [size], [ptr]) +``` + +## Parameters + +- `traces` — a stacktrace. [Array](../../data-types/array.md)([UInt64](../../data-types/int-uint.md)). +- `size` — an allocation size for memory profiling. (optional - default `1`). [UInt64](../../data-types/int-uint.md). +- `ptr` — an allocation address. (optional - default `0`). [UInt64](../../data-types/int-uint.md). + +:::note +In the case where `ptr != 0`, a flameGraph will map allocations (size > 0) and deallocations (size < 0) with the same size and ptr. +Only allocations which were not freed are shown. Non mapped deallocations are ignored. +::: + +## Returned value + +- An array of strings for use with [flamegraph.pl util](https://github.com/brendangregg/FlameGraph). [Array](../../data-types/array.md)([String](../../data-types/string.md)). + +## Examples + +### Building a flamegraph based on a CPU query profiler + +```sql +SET query_profiler_cpu_time_period_ns=10000000; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +``` + +```text +clickhouse client --allow_introspection_functions=1 -q "select arrayJoin(flameGraph(arrayReverse(trace))) from system.trace_log where trace_type = 'CPU' and query_id = 'xxx'" | ~/dev/FlameGraph/flamegraph.pl > flame_cpu.svg +``` + +### Building a flamegraph based on a memory query profiler, showing all allocations + +```sql +SET memory_profiler_sample_probability=1, max_untracked_memory=1; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +``` + +```text +clickhouse client --allow_introspection_functions=1 -q "select arrayJoin(flameGraph(trace, size)) from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx'" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem.svg +``` + +### Building a flamegraph based on a memory query profiler, showing allocations which were not deallocated in query context + +```sql +SET memory_profiler_sample_probability=1, max_untracked_memory=1, use_uncompressed_cache=1, merge_tree_max_rows_to_use_cache=100000000000, merge_tree_max_bytes_to_use_cache=1000000000000; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +``` + +```text +clickhouse client --allow_introspection_functions=1 -q "SELECT arrayJoin(flameGraph(trace, size, ptr)) FROM system.trace_log WHERE trace_type = 'MemorySample' AND query_id = 'xxx'" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_untracked.svg +``` + +### Build a flamegraph based on memory query profiler, showing active allocations at the fixed point of time + +```sql +SET memory_profiler_sample_probability=1, max_untracked_memory=1; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +``` + +- 1 - Memory usage per second + +```sql +SELECT event_time, m, formatReadableSize(max(s) as m) FROM (SELECT event_time, sum(size) OVER (ORDER BY event_time) AS s FROM system.trace_log WHERE query_id = 'xxx' AND trace_type = 'MemorySample') GROUP BY event_time ORDER BY event_time; +``` + +- 2 - Find a time point with maximal memory usage + +```sql +SELECT argMax(event_time, s), max(s) FROM (SELECT event_time, sum(size) OVER (ORDER BY event_time) AS s FROM system.trace_log WHERE query_id = 'xxx' AND trace_type = 'MemorySample'); +``` + +- 3 - Fix active allocations at fixed point of time + +```text +clickhouse client --allow_introspection_functions=1 -q "SELECT arrayJoin(flameGraph(trace, size, ptr)) FROM (SELECT * FROM system.trace_log WHERE trace_type = 'MemorySample' AND query_id = 'xxx' AND event_time <= 'yyy' ORDER BY event_time)" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_pos.svg +``` + +- 4 - Find deallocations at fixed point of time + +```text +clickhouse client --allow_introspection_functions=1 -q "SELECT arrayJoin(flameGraph(trace, -size, ptr)) FROM (SELECT * FROM system.trace_log WHERE trace_type = 'MemorySample' AND query_id = 'xxx' AND event_time > 'yyy' ORDER BY event_time desc)" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_neg.svg +``` \ No newline at end of file diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index a56b1c97681..e3725b6a430 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -58,6 +58,7 @@ ClickHouse-specific aggregate functions: - [topKWeighted](../reference/topkweighted.md) - [deltaSum](../reference/deltasum.md) - [deltaSumTimestamp](../reference/deltasumtimestamp.md) +- [flameGraph](../reference/flame_graph.md) - [groupArray](../reference/grouparray.md) - [groupArrayLast](../reference/grouparraylast.md) - [groupUniqArray](../reference/groupuniqarray.md) From 49ec0961cfb3ca0a1d4a880f76b4b9fed3007b36 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Jun 2024 09:44:05 +0000 Subject: [PATCH 331/856] Fix test --- .../02530_dictionaries_update_field.reference | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 88c910e0313..4d5a7447a49 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -10,7 +10,7 @@ SELECT key, value FROM dict_flat ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated @@ -27,7 +27,7 @@ SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -44,7 +44,7 @@ SELECT key, value FROM dict_hashed ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -61,7 +61,7 @@ SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -78,7 +78,7 @@ SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -95,7 +95,7 @@ SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated From d0a2abe17be7381c261a6d631037bd3a57d4fdb1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 4 Jun 2024 20:07:47 +0000 Subject: [PATCH 332/856] time_virtual_col: initial --- .../ObjectStorage/StorageObjectStorageSource.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 13 +++++++++++-- src/Storages/VirtualColumnUtils.h | 2 +- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index b31d0f8a92e..8d5c03ae11e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -199,7 +199,7 @@ Chunk StorageObjectStorageSource::generate() chunk, read_from_format_info.requested_virtual_columns, getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), - object_info.metadata->size_bytes, &filename); + object_info.metadata->size_bytes, &filename, object_info.metadata->last_modified); return chunk; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index cec55cefda2..5362cffece5 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -111,7 +112,7 @@ void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) NameSet getVirtualNamesForFileLikeStorage() { - return {"_path", "_file", "_size"}; + return {"_path", "_file", "_size", "_time"}; } VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns) @@ -129,6 +130,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_path", std::make_shared(std::make_shared())); add_virtual("_file", std::make_shared(std::make_shared())); add_virtual("_size", makeNullable(std::make_shared())); + add_virtual("_time", makeNullable(std::make_shared())); return desc; } @@ -188,7 +190,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const } void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename) + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename, std::optional last_modified) { for (const auto & virtual_column : requested_virtual_columns) { @@ -216,6 +218,13 @@ void addRequestedPathFileAndSizeVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } + else if (virtual_column.name == "_time") + { + if (last_modified) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), last_modified->utcTime())->convertToFullColumnIfConst()); + else + chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); + } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 62f2e4855b5..65826e1f1e7 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -69,7 +69,7 @@ void filterByPathOrFile(std::vector & sources, const std::vector & pa } void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr); + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr, std::optional last_modified = std::nullopt); } } From aefe7ffc31e4b2bb000d4df121472e85b93baa1c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 5 Jun 2024 21:34:29 +0000 Subject: [PATCH 333/856] time_virtual_col: slightly works --- src/IO/Archives/IArchiveReader.h | 2 ++ src/IO/Archives/LibArchiveReader.cpp | 1 + src/Storages/StorageFile.cpp | 4 +++- src/Storages/StorageFile.h | 1 + src/Storages/VirtualColumnUtils.cpp | 2 +- tests/integration/test_storage_hdfs/test.py | 2 +- tests/integration/test_storage_s3/test.py | 3 ++- 7 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/IO/Archives/IArchiveReader.h b/src/IO/Archives/IArchiveReader.h index ee516d2655b..d7758b9e401 100644 --- a/src/IO/Archives/IArchiveReader.h +++ b/src/IO/Archives/IArchiveReader.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -25,6 +26,7 @@ public: { UInt64 uncompressed_size; UInt64 compressed_size; + Poco::Timestamp last_modified; bool is_encrypted; }; diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index bec7f587180..e3fe63fa40d 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -157,6 +157,7 @@ public: file_info.emplace(); file_info->uncompressed_size = archive_entry_size(current_entry); file_info->compressed_size = archive_entry_size(current_entry); + file_info->last_modified = archive_entry_mtime(current_entry); file_info->is_encrypted = false; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 6744159d5dc..f84d3380c3f 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1341,6 +1341,7 @@ Chunk StorageFileSource::generate() chassert(file_enumerator); current_path = fmt::format("{}::{}", archive_reader->getPath(), *filename_override); current_file_size = file_enumerator->getFileInfo().uncompressed_size; + current_file_last_modified = file_enumerator->getFileInfo().last_modified; if (need_only_count && tryGetCountFromCache(current_archive_stat)) continue; @@ -1370,6 +1371,7 @@ Chunk StorageFileSource::generate() struct stat file_stat; file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); current_file_size = file_stat.st_size; + current_file_last_modified = Poco::Timestamp::fromEpochTime(file_stat.st_mtim.tv_sec); if (getContext()->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) continue; @@ -1437,7 +1439,7 @@ Chunk StorageFileSource::generate() /// Enrich with virtual columns. VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, current_path, current_file_size, filename_override.has_value() ? &filename_override.value() : nullptr); + chunk, requested_virtual_columns, current_path, current_file_size, filename_override.has_value() ? &filename_override.value() : nullptr, current_file_last_modified); return chunk; } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 37da59c3664..ac094aeb489 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -279,6 +279,7 @@ private: FilesIteratorPtr files_iterator; String current_path; std::optional current_file_size; + std::optional current_file_last_modified; struct stat current_archive_stat; std::optional filename_override; Block sample_block; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 5362cffece5..1e39d64fb18 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -221,7 +221,7 @@ void addRequestedPathFileAndSizeVirtualsToChunk( else if (virtual_column.name == "_time") { if (last_modified) - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), last_modified->utcTime())->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), last_modified->epochTime())->convertToFullColumnIfConst()); else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 44c0223e677..98ea79d6ee4 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -987,7 +987,7 @@ def test_read_subcolumns(started_cluster): assert res == "2\ttest_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( - f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select x.b.d, _path, x.b, _file, _time, now(), x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 09b27fff1e8..5becdf30476 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2117,7 +2117,7 @@ def test_read_subcolumns(started_cluster): assert res == "0\troot/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = instance.query( - f"select x.b.d, _path, x.b, _file, x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + f"select x.b.d, _path, x.b, _file, _time, now(), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" @@ -2148,6 +2148,7 @@ def test_read_subcolumns(started_cluster): res == "42\t/root/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" ) + logging.info("Some custom logging") def test_filtering_by_file_or_path(started_cluster): bucket = started_cluster.minio_bucket From 2f50e070efcceae8eba19287443e96ecdbfaf6b3 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Thu, 6 Jun 2024 10:34:59 +0000 Subject: [PATCH 334/856] Scale argument restricted to types (U)Int8 - (U)Int64 --- src/Functions/FunctionsRound.h | 22 +++++++++---------- .../03165_round_scale_as_column.sql | 1 + 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index a05619b54bb..1cc0b410514 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -626,7 +626,7 @@ public: }; /// Functions that round the value of an input parameter of type (U)Int8/16/32/64, Float32/64 or Decimal32/64/128. -/// Accept an additional optional parameter (0 by default). +/// Accept an additional optional parameter of type (U)Int8/16/32/64 (0 by default). template class FunctionRounding : public IFunction { @@ -640,19 +640,17 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if ((arguments.empty()) || (arguments.size() > 2)) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1 or 2.", - getName(), arguments.size()); + FunctionArgumentDescriptors mandatory_args{ + {"x", static_cast(&isNumber), nullptr, "A number to round"}, + }; + FunctionArgumentDescriptors optional_args{ + {"N", static_cast(&isNativeInteger), nullptr, "The number of decimal places to round to"}, + }; + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); - for (const auto & type : arguments) - if (!isNumber(type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0]->getName(), getName()); - - return arguments[0]; + return arguments[0].type; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.sql b/tests/queries/0_stateless/03165_round_scale_as_column.sql index 2304d2730a9..229f705808d 100644 --- a/tests/queries/0_stateless/03165_round_scale_as_column.sql +++ b/tests/queries/0_stateless/03165_round_scale_as_column.sql @@ -120,3 +120,4 @@ SELECT round(1, 1); SELECT round(materialize(1), materialize(1)); SELECT round(1, materialize(1)); --{serverError ILLEGAL_COLUMN} SELECT round(materialize(1), 1); +SELECT materialize(10.1) AS x, ceil(x, toUInt256(123)); --{serverError ILLEGAL_TYPE_OF_ARGUMENT} From 54ed3354f6be7aab304f55e253d8175a03d9a2b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 12:56:43 +0200 Subject: [PATCH 335/856] Better debugging --- ...23_zeros_generate_random_with_limit_progress_bar.sh | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh index 4bb8fc8880d..8aedf0bc0ff 100755 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -6,11 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_with_progress_and_match_total_rows() { - echo "$1" | \ - ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1 | \ - grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found" + CURL_RESPONSE=$(echo "$1" | \ + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) + + echo "$CURL_RESPONSE" | grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found: ${CURL_RESPONSE}" } run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros LIMIT 100' run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros_mt LIMIT 100' -run_with_progress_and_match_total_rows 'SELECT * FROM generateRandom() LIMIT 100' +run_with_progress_and_match_total_rows "SELECT * FROM generateRandom('number UInt64') LIMIT 100" From f0c21da1d8ed3327d4eb53f1e30bbc5b63c2e950 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Thu, 6 Jun 2024 11:06:16 +0000 Subject: [PATCH 336/856] Style fixed --- src/Functions/FunctionsRound.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 1cc0b410514..08e257de8ac 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -31,7 +31,6 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_COLUMN; From 8ebcd8860821537cec4e9e6a76653df3d8fb89ae Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 6 Jun 2024 13:08:40 +0200 Subject: [PATCH 337/856] In imported files should be only logging, no prints --- tests/ci/pr_info.py | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index ccf5dc23121..cd8d32d4e3c 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -59,7 +59,7 @@ def get_pr_for_commit(sha, ref): data = response.json() our_prs = [] # type: List[Dict] if len(data) > 1: - print("Got more than one pr for commit", sha) + logging.warning("Got more than one pr for commit %s", sha) for pr in data: # We need to check if the PR is created in our repo, because # https://github.com/kaynewu/ClickHouse/pull/2 @@ -71,13 +71,20 @@ def get_pr_for_commit(sha, ref): if pr["head"]["ref"] in ref: return pr our_prs.append(pr) - print( - f"Cannot find PR with required ref {ref}, sha {sha} - returning first one" + logging.warning( + "Cannot find PR with required ref %s, sha %s - returning first one", + ref, + sha, ) first_pr = our_prs[0] return first_pr except Exception as ex: - print(f"Cannot fetch PR info from commit {ref}, {sha}", ex) + logging.error( + "Cannot fetch PR info from commit ref %s, sha %s, exception: %s", + ref, + sha, + ex, + ) return None @@ -289,8 +296,10 @@ class PRInfo: else: # assume this is a dispatch self.event_type = EventType.DISPATCH - print("event.json does not match pull_request or push:") - print(json.dumps(github_event, sort_keys=True, indent=4)) + logging.warning( + "event.json does not match pull_request or push:\n%s", + json.dumps(github_event, sort_keys=True, indent=4), + ) self.sha = os.getenv( "GITHUB_SHA", "0000000000000000000000000000000000000000" ) @@ -357,7 +366,7 @@ class PRInfo: diff_object = PatchSet(response.text) self.changed_files.update({f.path for f in diff_object}) self.changed_files_requested = True - print(f"Fetched info about {len(self.changed_files)} changed files") + logging.info("Fetched info about %s changed files", len(self.changed_files)) def get_dict(self): return { From 407b86012c2f7c768202f0af0b54cd5b0426adbd Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 6 Jun 2024 13:23:48 +0200 Subject: [PATCH 338/856] stateless tests: add case for unequal joins --- ...006_join_on_inequal_expression_4.reference | 40 +++++++++++++++++++ .../03006_join_on_inequal_expression_4.sql.j2 | 10 +++-- 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference index f2a5901077a..0c747d04c58 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference @@ -121,3 +121,43 @@ SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = 1 10 0 0 2 15 2 10 3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 index 59df0b2abc2..3235019821b 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 @@ -42,6 +42,10 @@ SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 {% endfor -%} {% endfor -%} --- LOGICAL_ERROR Not-ready Set is passed as the second argument for function 'in' --- https://github.com/ClickHouse/ClickHouse/issues/64513 --- SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} From 3b593a7c5e69779a1324f2f4461b564e37a1fc1f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 6 Jun 2024 13:09:54 +0200 Subject: [PATCH 339/856] Make PRInfo.compare_pr_url persistent for the Github.PullReaquest objects --- tests/ci/pr_info.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index cd8d32d4e3c..dda5b30f1e3 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -266,12 +266,12 @@ class PRInfo: self.diff_urls.append( self.compare_url( pull_request["base"]["repo"]["default_branch"], - pull_request["head"]["label"], + pull_request["head"]["sha"], ) ) self.diff_urls.append( self.compare_url( - pull_request["head"]["label"], + pull_request["head"]["sha"], pull_request["base"]["repo"]["default_branch"], ) ) @@ -286,7 +286,7 @@ class PRInfo: # itself, but as well files changed since we branched out self.diff_urls.append( self.compare_url( - pull_request["head"]["label"], + pull_request["head"]["sha"], pull_request["base"]["repo"]["default_branch"], ) ) @@ -339,7 +339,7 @@ class PRInfo: return self.event_type == EventType.DISPATCH def compare_pr_url(self, pr_object: dict) -> str: - return self.compare_url(pr_object["base"]["label"], pr_object["head"]["label"]) + return self.compare_url(pr_object["base"]["sha"], pr_object["head"]["sha"]) @staticmethod def compare_url(first: str, second: str) -> str: From 0dff60821fd950d3e47f24e307ea8b3f5249f831 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 6 Jun 2024 13:10:51 +0200 Subject: [PATCH 340/856] Add context to the get_gh_api APIException --- tests/ci/build_download_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 0f6c8e5aa8a..036d3548eb9 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -110,7 +110,7 @@ def get_gh_api( logging.info("Exception '%s' while getting, retry %i", exc, try_cnt) time.sleep(sleep) - raise APIException("Unable to request data from GH API") from exc + raise APIException(f"Unable to request data from GH API: {url}") from exc def get_build_name_for_check(check_name: str) -> str: From a7230e3c6c3c35ff56836ed799f3f15bd037f8a0 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 27 May 2024 10:16:28 -0700 Subject: [PATCH 341/856] Add settings to allow parallel replicas custom key with range filter to use a custom range --- docs/en/operations/settings/settings.md | 17 ++++++ src/Core/Settings.h | 2 + src/Interpreters/InterpreterSelectQuery.cpp | 4 +- .../getCustomKeyFilterForParallelReplicas.cpp | 43 +++++++++----- .../getCustomKeyFilterForParallelReplicas.h | 10 +++- src/Planner/PlannerJoinTree.cpp | 4 +- src/Storages/StorageDistributed.cpp | 4 +- ...el_replicas_range_filter_min_max.reference | 7 +++ ...parallel_replicas_range_filter_min_max.sql | 57 +++++++++++++++++++ 9 files changed, 129 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference create mode 100644 tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ffaf53085c4..0dbd349525d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1590,6 +1590,23 @@ Possible values: 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 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]`. + +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]` when parallelizing the query. + +## 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 the custom range `[0, 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]` when parallelizing the query. + ## allow_experimental_parallel_reading_from_replicas Enables or disables sending SELECT queries to all replicas of a table (up to `max_parallel_replicas`). Reading is parallelized and coordinated dynamically. It will work for any kind of MergeTree table. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 27ce54c03a7..d6e4f0ae92b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -202,6 +202,8 @@ class IColumn; M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ + M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, std::numeric_limits::max(), "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 09f987a1c24..a418a4c9729 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -578,7 +578,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.parallel_replicas_count, settings.parallel_replica_offset, std::move(custom_key_ast), - settings.parallel_replicas_custom_key_filter_type, + {settings.parallel_replicas_custom_key_filter_type, + settings.parallel_replicas_custom_key_range_lower, + settings.parallel_replicas_custom_key_range_upper}, storage->getInMemoryMetadataPtr()->columns, context); } diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index d78b6ab0c4d..31669efb698 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -7,7 +7,6 @@ #include -#include #include @@ -18,18 +17,19 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; + extern const int INVALID_SETTING_VALUE; } ASTPtr getCustomKeyFilterForParallelReplica( size_t replicas_count, size_t replica_num, ASTPtr custom_key_ast, - ParallelReplicasCustomKeyFilterType filter_type, + ParallelReplicasCustomKeyFilter filter, const ColumnsDescription & columns, const ContextPtr & context) { chassert(replicas_count > 1); - if (filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) + if (filter.filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) { // first we do modulo with replica count auto modulo_function = makeASTFunction("positiveModulo", custom_key_ast, std::make_shared(replicas_count)); @@ -40,35 +40,48 @@ ASTPtr getCustomKeyFilterForParallelReplica( return equals_function; } - assert(filter_type == ParallelReplicasCustomKeyFilterType::RANGE); + assert(filter.filter_type == ParallelReplicasCustomKeyFilterType::RANGE); KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, columns, context); using RelativeSize = boost::rational; - RelativeSize size_of_universum = 0; + // get + RelativeSize range_upper = RelativeSize(filter.range_upper) + RelativeSize(1); + RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); if (custom_key_description.data_types.size() == 1) { if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); } - if (size_of_universum == RelativeSize(0)) + if (range_upper == RelativeSize(0)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); + if (range_lower < 0) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range Min must be a postive"); + + if (range_lower > range_upper) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key filter range: Range max {} must be larger than range min {}", + range_lower, + range_upper); + + RelativeSize size_of_universum = range_upper - range_lower; + RelativeSize relative_range_size = RelativeSize(1) / replicas_count; RelativeSize relative_range_offset = relative_range_size * RelativeSize(replica_num); @@ -76,16 +89,16 @@ ASTPtr getCustomKeyFilterForParallelReplica( bool has_lower_limit = false; bool has_upper_limit = false; - RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; - RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; + RelativeSize lower_limit_rational = range_lower + relative_range_offset * size_of_universum; + RelativeSize upper_limit_rational = range_lower + (relative_range_offset + relative_range_size) * size_of_universum; UInt64 lower = boost::rational_cast(lower_limit_rational); UInt64 upper = boost::rational_cast(upper_limit_rational); - if (lower > 0) + if (lower > range_lower) has_lower_limit = true; - if (upper_limit_rational < size_of_universum) + if (upper < range_upper) has_upper_limit = true; assert(has_lower_limit || has_upper_limit); diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h index 1506c1992c0..36198be8e51 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -6,16 +6,24 @@ #include #include #include +#include namespace DB { +struct ParallelReplicasCustomKeyFilter +{ + ParallelReplicasCustomKeyFilterType filter_type; + UInt64 range_lower; + UInt64 range_upper; +}; + /// Get AST for filter created from custom_key /// replica_num is the number of the replica for which we are generating filter starting from 0 ASTPtr getCustomKeyFilterForParallelReplica( size_t replicas_count, size_t replica_num, ASTPtr custom_key_ast, - ParallelReplicasCustomKeyFilterType filter_type, + ParallelReplicasCustomKeyFilter filter, const ColumnsDescription & columns, const ContextPtr & context); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 83b6f4f2c26..b092f3d0c26 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -501,7 +501,9 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, settings.parallel_replicas_count, settings.parallel_replica_offset, std::move(custom_key_ast), - settings.parallel_replicas_custom_key_filter_type, + {settings.parallel_replicas_custom_key_filter_type, + settings.parallel_replicas_custom_key_range_lower, + settings.parallel_replicas_custom_key_range_upper}, storage->getInMemoryMetadataPtr()->columns, query_context); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 9c58468c4a4..5048ef4788e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -904,11 +904,13 @@ void StorageDistributed::read( [my_custom_key_ast = std::move(custom_key_ast), column_description = this->getInMemoryMetadataPtr()->columns, custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, + custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value, + custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value, context = local_context, replica_count = modified_query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr { return getCustomKeyFilterForParallelReplica( - replica_count, replica_num - 1, my_custom_key_ast, custom_key_type, column_description, context); + replica_count, replica_num - 1, my_custom_key_ast, {custom_key_type, custom_key_range_lower, custom_key_range_upper}, column_description, context); }; } } diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference new file mode 100644 index 00000000000..5ba3f6bc471 --- /dev/null +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference @@ -0,0 +1,7 @@ +10 +10 +10 +10 +10 +10 +10 diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql new file mode 100644 index 00000000000..c9588d931a8 --- /dev/null +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql @@ -0,0 +1,57 @@ +DROP TABLE IF EXISTS range_filter_custom_range_test; + +CREATE TABLE range_filter_custom_range_test (k Int64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test SELECT number + 5 from numbers(10); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 5, parallel_replicas_custom_key_range_upper=15); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 4, parallel_replicas_custom_key_range_upper=14); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 6, parallel_replicas_custom_key_range_upper=17); + + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper=15); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 15, parallel_replicas_custom_key_range_upper=25); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper=5); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper=10000); + +DROP TABLE range_filter_custom_range_test; \ No newline at end of file From d455116b7c9e363dc6873221dd28c91bc6bf5f1e Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 29 May 2024 15:20:18 -0700 Subject: [PATCH 342/856] fix style --- docs/en/operations/settings/settings.md | 4 ++-- src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- .../getCustomKeyFilterForParallelReplicas.cpp | 5 ++--- .../getCustomKeyFilterForParallelReplicas.h | 4 ++-- src/Planner/PlannerJoinTree.cpp | 16 ++++++++-------- 5 files changed, 17 insertions(+), 18 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0dbd349525d..ff0da9eaa12 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1596,7 +1596,7 @@ Allows the filter type `range` to split the work evenly between replicas based t 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]` when parallelizing the query. +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} @@ -1605,7 +1605,7 @@ Allows the filter type `range` to split the work evenly between replicas based t 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]` when parallelizing the query. +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 diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a418a4c9729..b72399df2c1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -578,9 +578,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.parallel_replicas_count, settings.parallel_replica_offset, std::move(custom_key_ast), - {settings.parallel_replicas_custom_key_filter_type, - settings.parallel_replicas_custom_key_range_lower, - settings.parallel_replicas_custom_key_range_upper}, + {settings.parallel_replicas_custom_key_filter_type, + settings.parallel_replicas_custom_key_range_lower, + settings.parallel_replicas_custom_key_range_upper}, storage->getInMemoryMetadataPtr()->columns, context); } diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 31669efb698..6e45e806019 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -47,7 +47,6 @@ ASTPtr getCustomKeyFilterForParallelReplica( using RelativeSize = boost::rational; - // get RelativeSize range_upper = RelativeSize(filter.range_upper) + RelativeSize(1); RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; @@ -71,12 +70,12 @@ ASTPtr getCustomKeyFilterForParallelReplica( custom_key_column_type->getName()); if (range_lower < 0) - throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range Min must be a postive"); + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range lower bound must be a positive"); if (range_lower > range_upper) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, - "Invalid custom key filter range: Range max {} must be larger than range min {}", + "Invalid custom key filter range: Range upper bound {} must be larger than range lower bound {}", range_lower, range_upper); diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h index 36198be8e51..dfee5123ecb 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -14,8 +14,8 @@ struct ParallelReplicasCustomKeyFilter { ParallelReplicasCustomKeyFilterType filter_type; UInt64 range_lower; - UInt64 range_upper; -}; + UInt64 range_upper; +}; /// Get AST for filter created from custom_key /// replica_num is the number of the replica for which we are generating filter starting from 0 diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index b092f3d0c26..efc449402b9 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -498,14 +498,14 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, LOG_TRACE(getLogger("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); auto parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( - settings.parallel_replicas_count, - settings.parallel_replica_offset, - std::move(custom_key_ast), - {settings.parallel_replicas_custom_key_filter_type, - settings.parallel_replicas_custom_key_range_lower, - settings.parallel_replicas_custom_key_range_upper}, - storage->getInMemoryMetadataPtr()->columns, - query_context); + settings.parallel_replicas_count, + settings.parallel_replica_offset, + std::move(custom_key_ast), + {settings.parallel_replicas_custom_key_filter_type, + settings.parallel_replicas_custom_key_range_lower, + settings.parallel_replicas_custom_key_range_upper}, + storage->getInMemoryMetadataPtr()->columns, + query_context); return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context); } From 9ed52af56f51b24a9754a2d4d42060f5e9586712 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 29 May 2024 17:10:08 -0700 Subject: [PATCH 343/856] Add new settings to SettingsChangesHistory.h --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d6e4f0ae92b..092cbe35503 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -203,7 +203,7 @@ class IColumn; M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ - M(UInt64, parallel_replicas_custom_key_range_upper, std::numeric_limits::max(), "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, std::numeric_limits::max(), "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 814c08c5705..144cb9100e5 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -101,6 +101,8 @@ static const std::map Date: Thu, 30 May 2024 06:31:06 -0700 Subject: [PATCH 344/856] Fix issue with rational cast --- src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 6e45e806019..3cd8f3564a6 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -94,10 +94,10 @@ ASTPtr getCustomKeyFilterForParallelReplica( UInt64 lower = boost::rational_cast(lower_limit_rational); UInt64 upper = boost::rational_cast(upper_limit_rational); - if (lower > range_lower) + if (lower_limit_rational > range_lower) has_lower_limit = true; - if (upper < range_upper) + if (upper_limit_rational < range_upper) has_upper_limit = true; assert(has_lower_limit || has_upper_limit); From 0302f218e43fa0a6b8ba8c0696ec25169c508d47 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 30 May 2024 07:07:59 -0700 Subject: [PATCH 345/856] Add additional check to ensure range is larger than the number of replicas --- src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 3cd8f3564a6..a69d6cd6d21 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -72,7 +72,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( if (range_lower < 0) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range lower bound must be a positive"); - if (range_lower > range_upper) + if (range_lower >= range_upper) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range upper bound {} must be larger than range lower bound {}", @@ -81,6 +81,10 @@ ASTPtr getCustomKeyFilterForParallelReplica( RelativeSize size_of_universum = range_upper - range_lower; + if (size_of_universum <= RelativeSize(replicas_count)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range must be larger than than the number of replicas"); + RelativeSize relative_range_size = RelativeSize(1) / replicas_count; RelativeSize relative_range_offset = relative_range_size * RelativeSize(replica_num); From 677b2de5ed47f7940f100f9b53e171caa035899a Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Mon, 3 Jun 2024 08:43:09 -0400 Subject: [PATCH 346/856] Update docs/en/operations/settings/settings.md Co-authored-by: Antonio Andelic --- docs/en/operations/settings/settings.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ff0da9eaa12..6c94de3ac40 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1600,7 +1600,6 @@ Note: This setting will not cause any additional data to be filtered during quer ## 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 the custom range `[0, 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]`. From bde8d19f45f64492167c3b57e303b8d610a7f4aa Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 3 Jun 2024 10:12:24 -0700 Subject: [PATCH 347/856] Address review comments --- src/Core/Settings.h | 2 +- .../getCustomKeyFilterForParallelReplicas.cpp | 46 +++++++++++++++---- 2 files changed, 39 insertions(+), 9 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 092cbe35503..43879485f23 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -203,7 +203,7 @@ class IColumn; M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ - M(UInt64, parallel_replicas_custom_key_range_upper, std::numeric_limits::max(), "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index a69d6cd6d21..065460abda4 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -47,20 +47,53 @@ ASTPtr getCustomKeyFilterForParallelReplica( using RelativeSize = boost::rational; - RelativeSize range_upper = RelativeSize(filter.range_upper) + RelativeSize(1); + RelativeSize range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; if (custom_key_description.data_types.size() == 1) { if (typeid_cast(custom_key_column_type.get())) - range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); + { + range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (range_upper > RelativeSize(std::numeric_limits::max()) + RelativeSize(1)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt64) max value", + range_upper); + } else if (typeid_cast(custom_key_column_type.get())) - range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); + { + range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (range_upper > RelativeSize(std::numeric_limits::max()) + RelativeSize(1)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt32) max value", + range_upper); + } else if (typeid_cast(custom_key_column_type.get())) - range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); + { + range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (range_upper > RelativeSize(std::numeric_limits::max()) + RelativeSize(1)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt16) max value", + range_upper); + } else if (typeid_cast(custom_key_column_type.get())) - range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); + { + range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (range_upper > RelativeSize(std::numeric_limits::max()) + RelativeSize(1)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt8) max value", + range_upper); + } } if (range_upper == RelativeSize(0)) @@ -69,9 +102,6 @@ ASTPtr getCustomKeyFilterForParallelReplica( "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); - if (range_lower < 0) - throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range lower bound must be a positive"); - if (range_lower >= range_upper) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, From 0c0c6bee323dca926b7358bac927c52a815c3bdc Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 4 Jun 2024 06:32:38 -0700 Subject: [PATCH 348/856] Address review comments --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- .../getCustomKeyFilterForParallelReplicas.cpp | 6 +- ...parallel_replicas_range_filter_min_max.sql | 105 +++++++++++------- 4 files changed, 68 insertions(+), 47 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 43879485f23..cf572459cbc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -203,7 +203,7 @@ class IColumn; M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ - M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over. A value of 0 disables the upper bound, setting it to the max value of the custom key expression", 0) \ \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 144cb9100e5..35266249a2d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -102,7 +102,7 @@ static const std::map lower_function; @@ -156,7 +156,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( return upper_function; } - assert(upper_function && lower_function); + chassert(upper_function && lower_function); return makeASTFunction("and", std::move(lower_function), std::move(upper_function)); } diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql index c9588d931a8..b37948eccbd 100644 --- a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql @@ -4,54 +4,75 @@ CREATE TABLE range_filter_custom_range_test (k Int64) ENGINE=MergeTree ORDER BY INSERT INTO range_filter_custom_range_test SELECT number + 5 from numbers(10); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 5, parallel_replicas_custom_key_range_upper=15); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 5, parallel_replicas_custom_key_range_upper = 15 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 4, parallel_replicas_custom_key_range_upper=14); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 4, parallel_replicas_custom_key_range_upper = 14 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 6, parallel_replicas_custom_key_range_upper=17); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 6, parallel_replicas_custom_key_range_upper = 17 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper=15); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 15 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 15, parallel_replicas_custom_key_range_upper=25); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 15, parallel_replicas_custom_key_range_upper = 25 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper=5); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 5 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper=10000); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper = 10000 +); DROP TABLE range_filter_custom_range_test; \ No newline at end of file From e3e5f304b9b13a3b964d4a3e4483c45393973a37 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 4 Jun 2024 06:34:56 -0700 Subject: [PATCH 349/856] Update docs --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6c94de3ac40..65b0fa3f385 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1592,7 +1592,7 @@ 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 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]`. @@ -1600,7 +1600,7 @@ Note: This setting will not cause any additional data to be filtered during quer ## 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 the custom range `[0, 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]`. From e495bf074212bc3c5671b9fe754029eb1254eddf Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 5 Jun 2024 08:24:59 -0400 Subject: [PATCH 350/856] Update src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp Co-authored-by: Antonio Andelic --- src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 548aeba5398..7a908783f20 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -47,8 +47,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( using RelativeSize = boost::rational; - RelativeSize range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) - : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + RelativeSize range_upper = 0; RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; From 095508edb764af98706d860bd016f5309168c808 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 5 Jun 2024 06:57:02 -0700 Subject: [PATCH 351/856] Add additional tests --- .../getCustomKeyFilterForParallelReplicas.cpp | 2 +- ...el_replicas_range_filter_min_max.reference | 3 ++ ...parallel_replicas_range_filter_min_max.sql | 51 ++++++++++++++++++- 3 files changed, 53 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 7a908783f20..43be7c5f043 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -47,7 +47,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( using RelativeSize = boost::rational; - RelativeSize range_upper = 0; + RelativeSize range_upper = RelativeSize(0); RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference index 5ba3f6bc471..4dded9eda81 100644 --- a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference @@ -5,3 +5,6 @@ 10 10 10 +10 +13 +4 diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql index b37948eccbd..58143395e44 100644 --- a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS range_filter_custom_range_test; -CREATE TABLE range_filter_custom_range_test (k Int64) ENGINE=MergeTree ORDER BY k; +CREATE TABLE range_filter_custom_range_test (k UInt64) ENGINE=MergeTree ORDER BY k; INSERT INTO range_filter_custom_range_test SELECT number + 5 from numbers(10); @@ -75,4 +75,51 @@ FROM parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper = 10000 ); -DROP TABLE range_filter_custom_range_test; \ No newline at end of file + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 10, parallel_replicas_custom_key_range_upper = 13 +); + +DROP TABLE range_filter_custom_range_test; + +DROP TABLE IF EXISTS range_filter_custom_range_test_2; + +CREATE TABLE range_filter_custom_range_test_2 (k UInt64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test_2 SELECT number from numbers(13); + +SELECT count() +FROM +( + SELECT * + FROM cluster(parallel_replicas, currentDatabase(), range_filter_custom_range_test_2) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 12, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 13 +); + +DROP TABLE range_filter_custom_range_test_2; + +DROP TABLE IF EXISTS range_filter_custom_range_test_3; + +CREATE TABLE range_filter_custom_range_test_3 (k UInt64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test_3 SELECT number from numbers(4); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test_3) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 12, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 4 +); + +DROP TABLE range_filter_custom_range_test_3; \ No newline at end of file From 59f50ca2357e81c19c226b27a301c70b8ce09ffd Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 5 Jun 2024 10:53:48 -0700 Subject: [PATCH 352/856] Fix existing tests to use UInt rather than Int --- .../test_parallel_replicas_custom_key_failover/test.py | 4 ++-- .../0_stateless/02535_max_parallel_replicas_custom_key.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_parallel_replicas_custom_key_failover/test.py b/tests/integration/test_parallel_replicas_custom_key_failover/test.py index bbb8fd5abb0..3ba3ce092c3 100644 --- a/tests/integration/test_parallel_replicas_custom_key_failover/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_failover/test.py @@ -28,10 +28,10 @@ def create_tables(cluster, table_name): node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") node1.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" ) node3.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" ) # populate data diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 9850406eb3a..63644a51f8f 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -30,7 +30,7 @@ run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" -$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x)" +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y UInt32) ENGINE = MergeTree ORDER BY cityHash64(x)" $CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key SELECT toString(number), number % 3 FROM numbers(1000)" function run_count_with_custom_key { From 76db904de1d620f07d28135b53202ef8fcd14238 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 6 Jun 2024 05:15:41 -0700 Subject: [PATCH 353/856] Update integration tests to use UInt rather than Int --- tests/integration/test_parallel_replicas_custom_key/test.py | 2 +- .../test_parallel_replicas_custom_key_load_balancing/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index c646a678512..07a9e2badff 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -26,7 +26,7 @@ def create_tables(cluster): n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster}") n1.query( - f"CREATE TABLE test_table ON CLUSTER {cluster} (key Int32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" + f"CREATE TABLE test_table ON CLUSTER {cluster} (key UInt32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" ) n1.query( f""" diff --git a/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py b/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py index d5e17103296..ddfd07a0864 100644 --- a/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py @@ -33,7 +33,7 @@ def create_tables(table_name): for i in range(0, 4): nodes[i].query(f"DROP TABLE IF EXISTS {table_name} SYNC") nodes[i].query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r{i+1}') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r{i+1}') ORDER BY (key)" ) # populate data From 001ac9f8476dbcc2f04647b74fc8653dda9717fc Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 6 Jun 2024 09:48:48 -0300 Subject: [PATCH 354/856] fix no proxy hosts for remote and list resolvers --- src/Common/ProxyConfiguration.h | 2 +- src/Common/ProxyConfigurationResolverProvider.cpp | 5 +++-- src/Common/ProxyListConfigurationResolver.cpp | 1 - 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/ProxyConfiguration.h b/src/Common/ProxyConfiguration.h index 3398afce37c..a9921f1474d 100644 --- a/src/Common/ProxyConfiguration.h +++ b/src/Common/ProxyConfiguration.h @@ -55,7 +55,7 @@ struct ProxyConfiguration uint16_t port = 0; bool tunneling = false; Protocol original_request_protocol = Protocol::HTTP; - std::string no_proxy_hosts = ""; + std::string no_proxy_hosts = std::string{}; bool isEmpty() const { return host.empty(); } }; diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index f9eea1805fe..b06073121e7 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -54,7 +55,7 @@ namespace return std::make_shared( server_configuration, request_protocol, - getNoProxyHosts(configuration), + buildPocoNonProxyHosts(getNoProxyHosts(configuration)), std::make_shared(), isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); } @@ -97,7 +98,7 @@ namespace : std::make_shared( uris, request_protocol, - getNoProxyHosts(configuration), + buildPocoNonProxyHosts(getNoProxyHosts(configuration)), isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); } diff --git a/src/Common/ProxyListConfigurationResolver.cpp b/src/Common/ProxyListConfigurationResolver.cpp index a23c3a61951..2d5b5e97364 100644 --- a/src/Common/ProxyListConfigurationResolver.cpp +++ b/src/Common/ProxyListConfigurationResolver.cpp @@ -1,7 +1,6 @@ #include #include -#include #include namespace DB From f05a6577232e7061d13fe5888fb3da07cbfdfe39 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Jun 2024 12:56:34 +0000 Subject: [PATCH 355/856] add docs --- docs/en/operations/settings/settings.md | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ffaf53085c4..ada922cb037 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3172,7 +3172,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: @@ -4616,6 +4616,16 @@ Read more about [memory overcommit](memory-overcommit.md). Default value: `1GiB`. +## max_untracked_memory {#max_untracked_memory} +Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'. + +Default value: `4MiB`. + +## min_untracked_memory {#min_untracked_memory} +Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread memory usage devided by 16 and clamped between `min_untracked_memory` and `max_untracked_memory` for every thread. It guarantees that total untracked memory does not exceed 10% of current memory footprint even with a lot of small threads. To disable dynamic limit for untracked memory set value `4MiB`. + +Default value: `4KiB`. + ## Schema Inference settings See [schema inference](../../interfaces/schema-inference.md#schema-inference-modes) documentation for more details. From fab23d2a16573e3a9613cb53c2aba4639f8e33eb Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 6 Jun 2024 15:02:33 +0200 Subject: [PATCH 356/856] Add lowCardinalityKeys, lowCardinalityIndices and an example for blockSize --- .../functions/other-functions.md | 149 +++++++++++++++++- 1 file changed, 148 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 31df9e5627d..4639f9d0991 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -212,7 +212,7 @@ toTypeName(x) ## blockSize {#blockSize} -In ClickHouse, queries are processed in blocks (chunks). +In ClickHouse, queries are processed in [blocks](../../development/architecture.md/#block-block) (chunks). This function returns the size (row count) of the block the function is called on. **Syntax** @@ -221,6 +221,38 @@ This function returns the size (row count) of the block the function is called o blockSize() ``` +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (n UInt8) ENGINE = Memory; +-- Insert 3 blocks: +INSERT INTO test VALUES (1); +INSERT INTO test VALUES (1),(2); +INSERT INTO test VALUES (1),(2),(3); + +SELECT blockSize(), n FROM (SELECT * FROM test); +``` + +Result: + +```response + ┌─blockSize()─┬─n─┐ +1. │ 1 │ 1 │ + └─────────────┴───┘ + ┌─blockSize()─┬─n─┐ +2. │ 3 │ 1 │ +3. │ 3 │ 2 │ +4. │ 3 │ 3 │ + └─────────────┴───┘ + ┌─blockSize()─┬─n─┐ +5. │ 2 │ 1 │ +6. │ 2 │ 2 │ + └─────────────┴───┘ +``` + ## byteSize Returns an estimation of uncompressed byte size of its arguments in memory. @@ -3688,3 +3720,118 @@ Result: ```response {'version':'1','serial_number':'2D9071D64530052D48308473922C7ADAFA85D6C5','signature_algo':'sha256WithRSAEncryption','issuer':'/CN=marsnet.local CA','not_before':'May 7 17:01:21 2024 GMT','not_after':'May 7 17:01:21 2025 GMT','subject':'/CN=chnode1','pkey_algo':'rsaEncryption'} ``` + +## lowCardinalityIndices + +For each row in the current [block](../../development/architecture.md/#block-block), returns the index of the value in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. + +:::note +The first unique value encountered in the block is enumerated from 1. +::: + +**Syntax** + +```sql +lowCardinalityIndices(col) +``` + +**Arguments** + +- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md). + +**Returned value** + +- returns the index of the value in the dictionary of unique values, for each row in the current block. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; + +-- insert two blocks of data: + +INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); +INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); + +SELECT s, lowCardinalityIndices(s) FROM test; +``` + +Result: + +```response + ┌─s───┬─lowCardinalityIndices(s)─┐ +1. │ one │ 1 │ +2. │ two │ 2 │ +3. │ one │ 1 │ +4. │ one │ 1 │ +5. │ two │ 2 │ + └─────┴──────────────────────────┘ + ┌─s─────┬─lowCardinalityIndices(s)─┐ + 6. │ three │ 1 │ + 7. │ two │ 2 │ + 8. │ one │ 3 │ + 9. │ two │ 2 │ +10. │ two │ 2 │ +11. │ three │ 1 │ + └───────┴──────────────────────────┘ +``` +## lowCardinalityKeys + +For each row in the current [block](../../development/architecture.md/#block-block), returns the keys (unique values) in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. + +:::note +If the column size is less than the dictionary size, then values will be cut. If it is greater, then defaults will be added. +::: + +**Syntax** + +```sql +lowCardinalityIndices(col) +``` + +**Arguments** + +- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md). + +**Returned value** + +- returns the keys of the dictionary, for each row in the current block. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; + +-- insert two blocks of data: + +INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); +INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); + +SELECT s, lowCardinalityKeys(s) FROM test; +``` + +Result: + +```response + ┌─s───┬─lowCardinalityKeys(s)─┐ +1. │ one │ │ +2. │ two │ one │ +3. │ one │ two │ +4. │ one │ │ +5. │ two │ │ + └─────┴───────────────────────┘ + ┌─s─────┬─lowCardinalityKeys(s)─┐ + 6. │ three │ │ + 7. │ two │ three │ + 8. │ one │ two │ + 9. │ two │ one │ +10. │ two │ │ +11. │ three │ │ + └───────┴───────────────────────┘ +``` From baf3b6329d358eb024bacb54a863470b60337ee5 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 6 Jun 2024 15:07:45 +0200 Subject: [PATCH 357/856] Fix typo --- .../aggregate-functions/reference/flame_graph.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md b/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md index 4aa1cd0c8a8..e09769477f1 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md +++ b/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md @@ -5,7 +5,7 @@ sidebar_position: 110 # flameGraph -Aggregate function which builds a [flamegraph](https://www.brendangregg.com/flamegraphs.html) using the list of stacktraces. Outputs an array of strings which can be used by [flamegraph.pl util](https://github.com/brendangregg/FlameGraph) to render an SVG of the flamegraph. +Aggregate function which builds a [flamegraph](https://www.brendangregg.com/flamegraphs.html) using the list of stacktraces. Outputs an array of strings which can be used by [flamegraph.pl utility](https://github.com/brendangregg/FlameGraph) to render an SVG of the flamegraph. ## Syntax @@ -92,4 +92,4 @@ clickhouse client --allow_introspection_functions=1 -q "SELECT arrayJoin(flameGr ```text clickhouse client --allow_introspection_functions=1 -q "SELECT arrayJoin(flameGraph(trace, -size, ptr)) FROM (SELECT * FROM system.trace_log WHERE trace_type = 'MemorySample' AND query_id = 'xxx' AND event_time > 'yyy' ORDER BY event_time desc)" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_neg.svg -``` \ No newline at end of file +``` From 3c4f4d89813bac39b4eefa2011a82009b07723f2 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 6 Jun 2024 15:08:54 +0200 Subject: [PATCH 358/856] 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 49f43615c7e..09df4d98e71 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -222,6 +222,7 @@ DatabaseOrdinaryThreadsActive DateTime DateTimes DbCL +deallocated Decrypted Deduplicate Deduplication @@ -293,6 +294,7 @@ FilesystemMainPathUsedBytes FilesystemMainPathUsedINodes FixedString FlameGraph +flameGraph Flink ForEach FreeBSD From dbc27f0c34beb0efd7cbb20fd79cbab3669643e0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 6 Jun 2024 13:20:31 +0000 Subject: [PATCH 359/856] fix statistics --- src/Storages/MergeTree/MergeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 25e0d825778..56bd1181fef 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -58,7 +58,7 @@ static ColumnsStatistics getStatisticsForColumns( for (const auto & column : columns_to_read) { const auto * desc = all_columns.tryGet(column.name); - if (desc && desc->statistics.empty()) + if (desc && !desc->statistics.empty()) { auto statistics = MergeTreeStatisticsFactory::instance().get(desc->statistics); all_statistics.push_back(std::move(statistics)); From 387869488d80db6c645346c61b2470d37a026fe6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 15:54:02 +0200 Subject: [PATCH 360/856] Stabilize and adjust number of streams --- src/Storages/StorageGenerateRandom.cpp | 58 +++++++++++++++---- src/Storages/System/StorageSystemZeros.cpp | 33 ++++++----- ...generate_random_with_limit_progress_bar.sh | 2 +- 3 files changed, 65 insertions(+), 28 deletions(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index cdbade51695..5aceef78238 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -50,6 +50,12 @@ namespace ErrorCodes namespace { +struct GenerateRandomState +{ + std::atomic add_total_rows = 0; +}; +using GenerateRandomStatePtr = std::shared_ptr; + void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_of_type, pcg64 & rng, [[maybe_unused]] bool flip_bytes = false) { size_t size = limit * size_of_type; @@ -529,10 +535,24 @@ ColumnPtr fillColumnWithRandomData( class GenerateSource : public ISource { public: - GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_) + GenerateSource( + UInt64 block_size_, + UInt64 max_array_length_, + UInt64 max_string_length_, + UInt64 random_seed_, + Block block_header_, + ContextPtr context_, + GenerateRandomStatePtr state_) : ISource(Nested::flattenNested(prepareBlockToFill(block_header_))) - , block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_) - , block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {} + , block_size(block_size_) + , max_array_length(max_array_length_) + , max_string_length(max_string_length_) + , block_to_fill(std::move(block_header_)) + , rng(random_seed_) + , context(context_) + , shared_state(state_) + { + } String getName() const override { return "GenerateRandom"; } @@ -546,7 +566,15 @@ protected: columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context)); columns = Nested::flattenNested(block_to_fill.cloneWithColumns(columns)).getColumns(); - return {std::move(columns), block_size}; + + UInt64 total_rows = shared_state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + + auto chunk = Chunk{std::move(columns), block_size}; + progress(chunk.getNumRows(), chunk.bytes()); + + return chunk; } private: @@ -558,6 +586,7 @@ private: pcg64 rng; ContextPtr context; + GenerateRandomStatePtr shared_state; static Block & prepareBlockToFill(Block & block) { @@ -645,9 +674,6 @@ Pipe StorageGenerateRandom::read( { storage_snapshot->check(column_names); - Pipes pipes; - pipes.reserve(num_streams); - const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns(); Block block_header; for (const auto & name : column_names) @@ -676,16 +702,24 @@ Pipe StorageGenerateRandom::read( } } + UInt64 query_limit = query_info.limit; + if (query_limit && num_streams * max_block_size < query_limit) + { + /// We want to avoid spawning more streams than necessary + num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + } + Pipes pipes; + pipes.reserve(num_streams); + /// Will create more seed values for each source from initial seed. pcg64 generate(random_seed); + auto shared_state = std::make_shared(query_info.limit); + for (UInt64 i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context); - - if (i == 0 && query_info.limit) - source->addTotalRowsApprox(query_info.limit); - + auto source = std::make_shared( + max_block_size, max_array_length, max_string_length, generate(), block_header, context, shared_state); pipes.emplace_back(std::move(source)); } diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index a48b109fbbe..40faf2e265f 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -16,7 +16,9 @@ namespace struct ZerosState { + ZerosState(UInt64 limit) : add_total_rows(limit) { } std::atomic num_generated_rows = 0; + std::atomic add_total_rows = 0; }; using ZerosStatePtr = std::shared_ptr; @@ -49,6 +51,10 @@ protected: if (generated_rows >= limit) return {}; + UInt64 total_rows = state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + if (generated_rows + column_size > limit) { column_size = limit - generated_rows; @@ -105,10 +111,16 @@ Pipe StorageSystemZeros::read( bool use_multiple_streams = multithreaded; - if (limit && *limit < max_block_size) + UInt64 query_limit = limit ? *limit : query_info.limit; + if (query_limit && query_limit > max_block_size) + max_block_size = query_limit; + + if (use_multiple_streams && query_limit && num_streams * max_block_size < query_limit) { - max_block_size = static_cast(*limit); - use_multiple_streams = false; + /// We want to avoid spawning more streams than necessary + num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + if (num_streams <= 1) + use_multiple_streams = false; } if (!use_multiple_streams) @@ -118,21 +130,12 @@ Pipe StorageSystemZeros::read( ZerosStatePtr state; - if (limit) - state = std::make_shared(); + if (query_limit) + state = std::make_shared(query_limit); for (size_t i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, limit ? *limit : 0, state); - - if (i == 0) - { - if (limit) - source->addTotalRowsApprox(*limit); - else if (query_info.limit) - source->addTotalRowsApprox(query_info.limit); - } - + auto source = std::make_shared(max_block_size, query_limit, state); res.addSource(std::move(source)); } diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh index 8aedf0bc0ff..500a12587a2 100755 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_with_progress_and_match_total_rows() { CURL_RESPONSE=$(echo "$1" | \ - ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) echo "$CURL_RESPONSE" | grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found: ${CURL_RESPONSE}" } From 00167da1191bec5c2ab6df0323ccae34f2204016 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 6 Jun 2024 16:27:00 +0200 Subject: [PATCH 361/856] Disable transactions for unsupported storages even for materialized views --- src/Interpreters/InterpreterInsertQuery.cpp | 2 ++ ...03167_transactions_are_really_disabled.reference | 0 .../03167_transactions_are_really_disabled.sql | 13 +++++++++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03167_transactions_are_really_disabled.reference create mode 100644 tests/queries/0_stateless/03167_transactions_are_really_disabled.sql diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 128854e87ba..927bafe4bfb 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -279,6 +279,8 @@ Chain InterpreterInsertQuery::buildChain( std::atomic_uint64_t * elapsed_counter_ms, bool check_access) { + IInterpreter::checkStorageSupportsTransactionsIfNeeded(table, getContext()); + ProfileEvents::increment(ProfileEvents::InsertQueriesWithSubqueries); ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries); diff --git a/tests/queries/0_stateless/03167_transactions_are_really_disabled.reference b/tests/queries/0_stateless/03167_transactions_are_really_disabled.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql b/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql new file mode 100644 index 00000000000..e3c86a2d5be --- /dev/null +++ b/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS mv_table; +DROP TABLE IF EXISTS null_table; + +CREATE TABLE null_table (str String) ENGINE = Null; +CREATE MATERIALIZED VIEW mv_table (str String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/transactions_disabled_rmt', '{replica}') ORDER BY str AS SELECT str AS str FROM null_table; + +SET implicit_transaction=1; +set throw_on_unsupported_query_inside_transaction=0; + +INSERT INTO null_table VALUES ('test'); --{serverError NOT_IMPLEMENTED} + +DROP TABLE IF EXISTS mv_table; +DROP TABLE IF EXISTS null_table; From 05592fb5ba97dd86a744b146d15e9a1cb0422357 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 6 Jun 2024 14:27:23 +0000 Subject: [PATCH 362/856] additional log for cleanupDetachedTables --- src/Databases/DatabaseAtomic.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 8edc5b737a6..ccab72cfbae 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -1,20 +1,21 @@ +#include #include +#include #include #include -#include +#include #include #include -#include +#include +#include +#include +#include #include +#include +#include "Common/logger_useful.h" #include #include #include -#include -#include -#include -#include -#include -#include namespace fs = std::filesystem; @@ -393,6 +394,7 @@ DatabaseAtomic::DetachedTables DatabaseAtomic::cleanupDetachedTables() { DetachedTables not_in_use; auto it = detached_tables.begin(); + LOG_DEBUG(log, "There are {} detached tables. Start searching non used tables.", detached_tables.size()); while (it != detached_tables.end()) { if (it->second.unique()) @@ -403,6 +405,7 @@ DatabaseAtomic::DetachedTables DatabaseAtomic::cleanupDetachedTables() else ++it; } + LOG_DEBUG(log, "Found {} non used tables in detached tables.", not_in_use.size()); /// It should be destroyed in caller with released database mutex return not_in_use; } From 99de6bd6ecd71db4e132ca5fe997736affd3f306 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 6 Jun 2024 14:51:42 +0000 Subject: [PATCH 363/856] fix tests --- .../queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql | 2 +- .../queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql index 4f2fb8dea0c..ac987c9c75c 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql @@ -32,7 +32,7 @@ SELECT groups[2] AS merged, groups[3] AS gathered FROM system.text_log -WHERE (query_id = uuid || '::all_1_2_1') OR (query_id = currentDatabase() || '.t_ind_merge_1::all_1_2_1') AND notEmpty(groups) +WHERE ((query_id = uuid || '::all_1_2_1') OR (query_id = currentDatabase() || '.t_ind_merge_1::all_1_2_1')) AND notEmpty(groups) ORDER BY event_time_microseconds; DROP TABLE t_ind_merge_1; diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql index 0710b1069a8..2805059d918 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql @@ -35,7 +35,7 @@ SELECT groups[2] AS merged, groups[3] AS gathered FROM system.text_log -WHERE (query_id = uuid || '::all_1_2_1') OR (query_id = currentDatabase() || '.t_ind_merge_2::all_1_2_1') AND notEmpty(groups) +WHERE ((query_id = uuid || '::all_1_2_1') OR (query_id = currentDatabase() || '.t_ind_merge_2::all_1_2_1')) AND notEmpty(groups) ORDER BY event_time_microseconds; DROP TABLE t_ind_merge_2; From 9419a252315d053f443ebf9a4b24d0d5489fe8c8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 28 Feb 2024 01:41:40 +0800 Subject: [PATCH 364/856] Trivial count optimization with primary key --- .../optimizeUseAggregateProjection.cpp | 162 +++++++++++++++--- .../optimizeUseNormalProjection.cpp | 4 +- .../Optimizations/projectionsCommon.cpp | 3 +- .../QueryPlan/ReadFromMergeTree.cpp | 5 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 6 +- src/Storages/ProjectionsDescription.cpp | 4 - src/Storages/ProjectionsDescription.h | 2 - .../00636_partition_key_parts_pruning.sh | 22 +-- .../00653_verification_monotonic_data_load.sh | 1 + ...1710_projection_pk_trivial_count.reference | 3 + .../01710_projection_pk_trivial_count.sql | 8 + .../1_stateful/00166_explain_estimate.sql | 2 + ...00175_counting_resources_in_subqueries.sql | 2 + 13 files changed, 173 insertions(+), 51 deletions(-) create mode 100644 tests/queries/0_stateless/01710_projection_pk_trivial_count.reference create mode 100644 tests/queries/0_stateless/01710_projection_pk_trivial_count.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 4017670ad14..713f2f35fc8 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -421,6 +421,9 @@ struct AggregateProjectionCandidates /// This flag means that DAG for projection candidate should be used in FilterStep. bool has_filter = false; + + /// If not empty, try to find exact ranges from parts to speed up trivial count queries. + String only_count_column; }; AggregateProjectionCandidates getAggregateProjectionCandidates( @@ -502,6 +505,12 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( candidates.minmax_projection.emplace(std::move(minmax)); } } + else + { + /// Trivial count optimization only applies after @can_use_minmax_projection. + if (keys.empty() && aggregates.size() == 1 && typeid_cast(aggregates[0].function.get())) + candidates.only_count_column = aggregates[0].column_name; + } } if (!candidates.minmax_projection) @@ -584,13 +593,21 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu ContextPtr context = reading->getContext(); MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); AggregateProjectionCandidate * best_candidate = nullptr; + + /// Stores row count from exact ranges of parts. + size_t exact_count = 0; + if (candidates.minmax_projection) { best_candidate = &candidates.minmax_projection->candidate; } - else if (!candidates.real.empty()) + else if (!candidates.real.empty() || !candidates.only_count_column.empty()) { - auto ordinary_reading_select_result = reading->selectRangesToRead(); + auto ordinary_reading_select_result = reading->getAnalyzedResult(); + bool find_exact_ranges = !candidates.only_count_column.empty(); + if (!ordinary_reading_select_result || (!ordinary_reading_select_result->has_exact_ranges && find_exact_ranges)) + ordinary_reading_select_result = reading->selectRangesToRead(find_exact_ranges); + size_t ordinary_reading_marks = ordinary_reading_select_result->selected_marks; /// Nothing to read. Ignore projections. @@ -600,7 +617,49 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu return {}; } - const auto & parts_with_ranges = ordinary_reading_select_result->parts_with_ranges; + auto & parts_with_ranges = ordinary_reading_select_result->parts_with_ranges; + + if (!candidates.only_count_column.empty()) + { + for (auto & part_with_ranges : parts_with_ranges) + { + MarkRanges new_ranges; + auto & ranges = part_with_ranges.ranges; + const auto & exact_ranges = part_with_ranges.exact_ranges; + if (exact_ranges.empty()) + continue; + + size_t i = 0; + size_t len = exact_ranges.size(); + for (auto & range : ranges) + { + while (i < len && exact_ranges[i].begin < range.end) + { + chassert(exact_ranges[i].begin >= range.begin); + chassert(exact_ranges[i].end <= range.end); + + /// Found some marks which are not exact + if (range.begin < exact_ranges[i].begin) + new_ranges.emplace_back(range.begin, exact_ranges[i].begin); + + range.begin = exact_ranges[i].end; + ordinary_reading_marks -= exact_ranges[i].end - exact_ranges[i].begin; + exact_count += part_with_ranges.data_part->index_granularity.getRowsCountInRange(exact_ranges[i]); + ++i; + } + + /// Current range still contains some marks which are not exact + if (range.begin < range.end) + new_ranges.emplace_back(range); + } + chassert(i == len); + part_with_ranges.ranges = std::move(new_ranges); + } + + std::erase_if(parts_with_ranges, [&](const auto & part_with_ranges) { return part_with_ranges.ranges.empty(); }); + if (parts_with_ranges.empty()) + chassert(ordinary_reading_marks == 0); + } /// Selecting best candidate. for (auto & candidate : candidates.real) @@ -630,8 +689,20 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu if (!best_candidate) { - reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); - return {}; + if (exact_count > 0) + { + if (ordinary_reading_marks > 0) + { + ordinary_reading_select_result->selected_marks = ordinary_reading_marks; + ordinary_reading_select_result->selected_rows -= exact_count; + reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); + } + } + else + { + reading->setAnalyzedResult(std::move(ordinary_reading_select_result)); + return {}; + } } } else @@ -639,10 +710,11 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu return {}; } - chassert(best_candidate != nullptr); - QueryPlanStepPtr projection_reading; bool has_ordinary_parts; + String selected_projection_name; + if (best_candidate) + selected_projection_name = best_candidate->projection->name; /// Add reading from projection step. if (candidates.minmax_projection) @@ -654,6 +726,32 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu projection_reading = std::make_unique(std::move(pipe)); has_ordinary_parts = false; } + else if (best_candidate == nullptr) + { + chassert(exact_count > 0); + + auto agg_count = std::make_shared(DataTypes{}); + + std::vector state(agg_count->sizeOfData()); + AggregateDataPtr place = state.data(); + agg_count->create(place); + SCOPE_EXIT_MEMORY_SAFE(agg_count->destroy(place)); + agg_count->set(place, exact_count); + + auto column = ColumnAggregateFunction::create(agg_count); + column->insertFrom(place); + + Block block_with_count{ + {std::move(column), + std::make_shared(agg_count, DataTypes{}, Array{}), + candidates.only_count_column}}; + + Pipe pipe(std::make_shared(std::move(block_with_count))); + projection_reading = std::make_unique(std::move(pipe)); + + selected_projection_name = "Optimized trivial count"; + has_ordinary_parts = reading->getAnalyzedResult() != nullptr; + } else { auto storage_snapshot = reading->getStorageSnapshot(); @@ -694,46 +792,54 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu context->getQueryContext()->addQueryAccessInfo(Context::QualifiedProjectionName { .storage_id = reading->getMergeTreeData().getStorageID(), - .projection_name = best_candidate->projection->name, + .projection_name = selected_projection_name, }); } // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection reading header {}", // projection_reading->getOutputStream().header.dumpStructure()); - projection_reading->setStepDescription(best_candidate->projection->name); - + projection_reading->setStepDescription(selected_projection_name); auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); - auto & expr_or_filter_node = nodes.emplace_back(); - if (candidates.has_filter) + /// Root node of optimized child plan using @projection_name + QueryPlan::Node * aggregate_projection_node = nullptr; + + if (best_candidate) { - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - best_candidate->dag, - best_candidate->dag->getOutputs().front()->result_name, - true); - } - else - expr_or_filter_node.step = std::make_unique( - projection_reading_node.step->getOutputStream(), - best_candidate->dag); + aggregate_projection_node = &nodes.emplace_back(); + if (candidates.has_filter) + { + aggregate_projection_node->step = std::make_unique( + projection_reading_node.step->getOutputStream(), + best_candidate->dag, + best_candidate->dag->getOutputs().front()->result_name, + true); + } + else + aggregate_projection_node->step + = std::make_unique(projection_reading_node.step->getOutputStream(), best_candidate->dag); - expr_or_filter_node.children.push_back(&projection_reading_node); + aggregate_projection_node->children.push_back(&projection_reading_node); + } + else /// trivial count optimization + { + aggregate_projection_node = &projection_reading_node; + } if (!has_ordinary_parts) { /// All parts are taken from projection - aggregating->requestOnlyMergeForAggregateProjection(expr_or_filter_node.step->getOutputStream()); - node.children.front() = &expr_or_filter_node; + aggregating->requestOnlyMergeForAggregateProjection(aggregate_projection_node->step->getOutputStream()); + node.children.front() = aggregate_projection_node; } else { - node.step = aggregating->convertToAggregatingProjection(expr_or_filter_node.step->getOutputStream()); - node.children.push_back(&expr_or_filter_node); + node.step = aggregating->convertToAggregatingProjection(aggregate_projection_node->step->getOutputStream()); + node.children.push_back(aggregate_projection_node); } - return best_candidate->projection->name; + return selected_projection_name; } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 728aaaa6fc4..0af3869ccf1 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -139,7 +139,9 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod const auto & query_info = reading->getQueryInfo(); MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); - auto ordinary_reading_select_result = reading->selectRangesToRead(); + auto ordinary_reading_select_result = reading->getAnalyzedResult(); + if (!ordinary_reading_select_result) + ordinary_reading_select_result = reading->selectRangesToRead(); size_t ordinary_reading_marks = ordinary_reading_select_result->selected_marks; /// Nothing to read. Ignore projections. diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 3009460a468..af1578d6af8 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -25,8 +25,7 @@ namespace QueryPlanOptimizations bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) { - /// Probably some projection already was applied. - if (reading->hasAnalyzedResult()) + if (reading->getAnalyzedResult() && reading->getAnalyzedResult()->readFromProjection()) return false; if (reading->isQueryWithFinal()) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 887a95da60d..e469062d7e7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1358,9 +1358,9 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( return merging_pipes.empty() ? Pipe::unitePipes(std::move(no_merging_pipes)) : Pipe::unitePipes(std::move(merging_pipes)); } -ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead() const +ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(bool find_exact_ranges) const { - return selectRangesToRead(prepared_parts, alter_conversions_for_parts, false /* find_exact_ranges */); + return selectRangesToRead(prepared_parts, alter_conversions_for_parts, find_exact_ranges); } ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( @@ -1664,6 +1664,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( result.selected_marks_pk = sum_marks_pk; result.total_marks_pk = total_marks_pk; result.selected_rows = sum_rows; + result.has_exact_ranges = result.selected_parts == 0 || find_exact_ranges; if (query_info_.input_order_info) result.read_type = (query_info_.input_order_info->direction > 0) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index d1f88a6788f..caa8aa2e1bd 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -100,7 +100,9 @@ public: UInt64 selected_marks_pk = 0; UInt64 total_marks_pk = 0; UInt64 selected_rows = 0; + bool has_exact_ranges = false; + bool readFromProjection() const { return !parts_with_ranges.empty() && parts_with_ranges.front().data_part->isProjectionPart(); } void checkLimits(const Settings & settings, const SelectQueryInfo & query_info_) const; }; @@ -167,7 +169,7 @@ public: AnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, std::vector alter_conversions, bool find_exact_ranges = false) const; - AnalysisResultPtr selectRangesToRead() const; + AnalysisResultPtr selectRangesToRead(bool find_exact_ranges = false) const; StorageMetadataPtr getStorageMetadata() const { return metadata_for_reading; } @@ -182,7 +184,7 @@ public: bool requestOutputEachPartitionThroughSeparatePort(); bool willOutputEachPartitionThroughSeparatePort() const { return output_each_partition_through_separate_port; } - bool hasAnalyzedResult() const { return analyzed_result_ptr != nullptr; } + AnalysisResultPtr getAnalyzedResult() const { return analyzed_result_ptr; } void setAnalyzedResult(AnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 0bcbedee41a..f5d869cdea0 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -64,7 +64,6 @@ ProjectionDescription ProjectionDescription::clone() const other.sample_block_for_keys = sample_block_for_keys; other.metadata = metadata; other.key_size = key_size; - other.is_minmax_count_projection = is_minmax_count_projection; other.primary_key_max_column_name = primary_key_max_column_name; other.partition_value_indices = partition_value_indices; @@ -195,7 +194,6 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( ContextPtr query_context) { ProjectionDescription result; - result.is_minmax_count_projection = true; auto select_query = std::make_shared(); ASTPtr select_expression_list = std::make_shared(); @@ -282,13 +280,11 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( return result; } - void ProjectionDescription::recalculateWithNewColumns(const ColumnsDescription & new_columns, ContextPtr query_context) { *this = getProjectionFromAST(definition_ast, new_columns, query_context); } - Block ProjectionDescription::calculate(const Block & block, ContextPtr context) const { auto mut_context = Context::createCopy(context); diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index 75a97697e00..5f091b4421b 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -56,8 +56,6 @@ struct ProjectionDescription size_t key_size = 0; - bool is_minmax_count_projection = false; - /// If a primary key expression is used in the minmax_count projection, store the name of max expression. String primary_key_max_column_name; diff --git a/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh b/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh index 7ec4d99f028..f2b4cae8bb0 100755 --- a/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh +++ b/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh @@ -11,9 +11,11 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE single_col_partition_key(x UInt32) EN ${CLICKHOUSE_CLIENT} --query="INSERT INTO single_col_partition_key VALUES (1), (2), (3), (4), (11), (12), (20)" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x < 3 FORMAT XML" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x >= 11 FORMAT XML" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x = 20 FORMAT XML" | grep -F rows_read | sed 's/^[ \t]*//g' +DISABLE_COUNT_OPTIMIZATION="SETTINGS optimize_trivial_count_query = 0, optimize_use_implicit_projections = 0" + +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x < 3 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x >= 11 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x = 20 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' ${CLICKHOUSE_CLIENT} --query="DROP TABLE single_col_partition_key" @@ -31,14 +33,14 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO composite_partition_key VALUES \ ${CLICKHOUSE_CLIENT} --query="INSERT INTO composite_partition_key VALUES \ (301, 20, 3), (302, 21, 3), (303, 22, 3)" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a > 400 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE b = 11 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE c = 4 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a > 400 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE b = 11 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE c = 4 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a < 200 AND c = 2 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a = 301 AND b < 20 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE b >= 12 AND c = 2 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a < 200 AND c = 2 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a = 301 AND b < 20 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE b >= 12 AND c = 2 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a = 301 AND b = 21 AND c = 3 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a = 301 AND b = 21 AND c = 3 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' ${CLICKHOUSE_CLIENT} --query="DROP TABLE composite_partition_key" diff --git a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index e10b2f86145..7766d7720e1 100755 --- a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -45,6 +45,7 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO enum_test_table VALUES ('hello'), ('wo ${CLICKHOUSE_CLIENT} --query="INSERT INTO date_test_table VALUES (1), (2), (2), (256), (257), (257);" CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') +CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --optimize_use_implicit_projections 0" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64" diff --git a/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference b/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference new file mode 100644 index 00000000000..43316772467 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference @@ -0,0 +1,3 @@ + ReadFromMergeTree (default.x) + ReadFromPreparedSource (Optimized trivial count) +5 diff --git a/tests/queries/0_stateless/01710_projection_pk_trivial_count.sql b/tests/queries/0_stateless/01710_projection_pk_trivial_count.sql new file mode 100644 index 00000000000..ce9eadf06b2 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_pk_trivial_count.sql @@ -0,0 +1,8 @@ +drop table if exists x; + +create table x (i int) engine MergeTree order by i settings index_granularity = 3; +insert into x select * from numbers(10); +select * from (explain select count() from x where (i >= 3 and i <= 6) or i = 7) where explain like '%ReadFromPreparedSource%' or explain like '%ReadFromMergeTree%'; +select count() from x where (i >= 3 and i <= 6) or i = 7; + +drop table x; diff --git a/tests/queries/1_stateful/00166_explain_estimate.sql b/tests/queries/1_stateful/00166_explain_estimate.sql index c4071271736..136433c16ee 100644 --- a/tests/queries/1_stateful/00166_explain_estimate.sql +++ b/tests/queries/1_stateful/00166_explain_estimate.sql @@ -1,6 +1,8 @@ -- Tags: no-replicated-database -- Tag no-replicated-database: Requires investigation +SET optimize_use_implicit_projections = 0; + EXPLAIN ESTIMATE SELECT count() FROM test.hits WHERE CounterID = 29103473; EXPLAIN ESTIMATE SELECT count() FROM test.hits WHERE CounterID != 29103473; EXPLAIN ESTIMATE SELECT count() FROM test.hits WHERE CounterID > 29103473; diff --git a/tests/queries/1_stateful/00175_counting_resources_in_subqueries.sql b/tests/queries/1_stateful/00175_counting_resources_in_subqueries.sql index 63eca96414f..5733bf6dd18 100644 --- a/tests/queries/1_stateful/00175_counting_resources_in_subqueries.sql +++ b/tests/queries/1_stateful/00175_counting_resources_in_subqueries.sql @@ -1,3 +1,5 @@ +SET optimize_use_implicit_projections = 0; + -- the work for scalar subquery is properly accounted: SET max_rows_to_read = 1000000; SELECT 1 = (SELECT count() FROM test.hits WHERE NOT ignore(AdvEngineID)); -- { serverError TOO_MANY_ROWS } From dd9b15daf5accedc0e850e4d12b2ebc88b24bd86 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 6 Jun 2024 23:43:14 +0800 Subject: [PATCH 365/856] Fix tupleConcat of two empty tuples --- src/Functions/tupleConcat.cpp | 5 ++++- tests/queries/0_stateless/03167_empty_tuple_concat.reference | 1 + tests/queries/0_stateless/03167_empty_tuple_concat.sql | 1 + 3 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03167_empty_tuple_concat.reference create mode 100644 tests/queries/0_stateless/03167_empty_tuple_concat.sql diff --git a/src/Functions/tupleConcat.cpp b/src/Functions/tupleConcat.cpp index c48e4d61463..c9cdae10bcf 100644 --- a/src/Functions/tupleConcat.cpp +++ b/src/Functions/tupleConcat.cpp @@ -61,7 +61,7 @@ public: return std::make_shared(tuple_arg_types); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const size_t num_arguments = arguments.size(); Columns columns; @@ -92,6 +92,9 @@ public: columns.push_back(inner_col); } + if (columns.empty()) + return ColumnTuple::create(input_rows_count); + return ColumnTuple::create(columns); } }; diff --git a/tests/queries/0_stateless/03167_empty_tuple_concat.reference b/tests/queries/0_stateless/03167_empty_tuple_concat.reference new file mode 100644 index 00000000000..6a452c185a8 --- /dev/null +++ b/tests/queries/0_stateless/03167_empty_tuple_concat.reference @@ -0,0 +1 @@ +() diff --git a/tests/queries/0_stateless/03167_empty_tuple_concat.sql b/tests/queries/0_stateless/03167_empty_tuple_concat.sql new file mode 100644 index 00000000000..f6fce86f332 --- /dev/null +++ b/tests/queries/0_stateless/03167_empty_tuple_concat.sql @@ -0,0 +1 @@ +SELECT ()||(); From 15317b5b507aae95194f0cdb7d9678ce43a3784f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 6 Jun 2024 07:29:06 +0200 Subject: [PATCH 366/856] Fix possible loss of "Query was cancelled" message in client Previously "Query was cancelled" had been printed only on new blocks, so if no new blocks will be arrived, then it will not print this message, and this is indeed why 03023_zeros_generate_random_with_limit_progress_bar was flaky [1]: 2024-06-05 16:21:22 expect: does "\u258e\u001b[0m \u001b[2m(0.9 CPU)\u001b[0m 4%\u001b[K^C\r\u001b[1;31m\u2198\u001b[0m Progress: 404.74 thousand rows, 404.74 KB (806.77 thousand rows/s., 806.77 KB/s.) \u001b[0;32m\u2588\u258e\u001b[0m \u001b[2m(2.0 CPU)\u001b[0m 4%\u001b[K\r\u001b[KCancelling query.\r\n \r\u001b[1;32m\u2193\u001b[0m Progress: 404.74 thousand rows, 404.74 KB (806.15 thousand rows/s., 806.15 KB/s.) \u001b[0;32m\u2588\u258e\u001b[0m \u001b[2m(2.0 CPU)\u001b[0m 4%\u001b[K" (spawn_id exp4) match glob pattern "Query was cancelled."? no 2024-06-05 16:21:22 2024-06-05 16:21:22 [KOk. 2024-06-05 16:21:22 , result: As you can see it printed "Cancelling query" and "Ok" but not "Query was cancelled" due to this. [1]: https://s3.amazonaws.com/clickhouse-test-reports/61973/6cfd5b2165970a65a551117fe58e4b9d22237b8c/stateless_tests__tsan__s3_storage__[1_5].html A bit hackish with all this bool flags, but I guess it is OK. Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 13 +++++++++++-- src/Client/ClientBase.h | 1 + 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f8391c64d5a..854cc3fef8b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1188,7 +1188,10 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, b std::rethrow_exception(local_format_error); if (cancelled && is_interactive) + { std::cout << "Query was cancelled." << std::endl; + cancelled_printed = true; + } } @@ -1302,8 +1305,13 @@ void ClientBase::onEndOfStream() resetOutput(); - if (is_interactive && !written_first_block) - std::cout << "Ok." << std::endl; + if (is_interactive) + { + if (cancelled && !cancelled_printed) + std::cout << "Query was cancelled." << std::endl; + else if (!written_first_block) + std::cout << "Ok." << std::endl; + } } @@ -1866,6 +1874,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin resetOutput(); have_error = false; cancelled = false; + cancelled_printed = false; client_exception.reset(); server_exception.reset(); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 7a0489641c8..220fcddc038 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -329,6 +329,7 @@ protected: bool allow_merge_tree_settings = false; bool cancelled = false; + bool cancelled_printed = false; /// Does log_comment has specified by user? bool has_log_comment = false; From 833f031721cf40fc28d155a5df7cf23079bde9a9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 5 Jun 2024 18:32:04 +0200 Subject: [PATCH 367/856] Handle logs from rocksdb by ClickHouse internal logging Someone may not know about /path/to/rocksdb/LOG, and besides it requires an access to data dir, instead of logs dir. v2: fix use-after-free due to destruction order (https://s3.amazonaws.com/clickhouse-test-reports/64856/8cefc1a1ba5ddfdda033694a14e1f1847d497092/stateless_tests__asan__[2_4]/stderr.log) Signed-off-by: Azat Khuzhin --- programs/server/config.xml | 1 + .../RocksDB/StorageEmbeddedRocksDB.cpp | 71 ++++++++++++++++++- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 6 +- 3 files changed, 75 insertions(+), 3 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 4b3248d9d1c..b7a4b8dd0e9 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1561,6 +1561,7 @@ 8 + DEBUG_LEVEL 2 diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index c3b7ae64c7e..b9d3e071b6c 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -25,6 +25,7 @@ #include #include +#include #include #include #include @@ -42,6 +43,7 @@ #include #include +#include #include @@ -185,11 +187,11 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, bool read_only_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) + , log(getLogger(fmt::format("StorageEmbeddedRocksDB ({})", getStorageID().getNameForLogs()))) , primary_key{primary_key_} , rocksdb_dir(std::move(rocksdb_dir_)) , ttl(ttl_) , read_only(read_only_) - , log(getLogger(fmt::format("StorageEmbeddedRocksDB ({})", getStorageID().getNameForLogs()))) { setInMemoryMetadata(metadata_); setSettings(std::move(settings_)); @@ -352,6 +354,72 @@ bool StorageEmbeddedRocksDB::optimize( return true; } +static_assert(rocksdb::DEBUG_LEVEL == 0); +static_assert(rocksdb::HEADER_LEVEL == 5); +static constexpr std::array, 6> rocksdb_logger_map = { + std::make_pair(DB::LogsLevel::debug, Poco::Message::Priority::PRIO_DEBUG), + std::make_pair(DB::LogsLevel::information, Poco::Message::Priority::PRIO_INFORMATION), + std::make_pair(DB::LogsLevel::warning, Poco::Message::Priority::PRIO_WARNING), + std::make_pair(DB::LogsLevel::error, Poco::Message::Priority::PRIO_ERROR), + std::make_pair(DB::LogsLevel::fatal, Poco::Message::Priority::PRIO_FATAL), + /// Same as default logger does for HEADER_LEVEL + std::make_pair(DB::LogsLevel::information, Poco::Message::Priority::PRIO_INFORMATION), +}; +class StorageEmbeddedRocksDBLogger : public rocksdb::Logger +{ +public: + explicit StorageEmbeddedRocksDBLogger(const rocksdb::InfoLogLevel log_level, LoggerRawPtr log_) + : rocksdb::Logger(log_level) + , log(log_) + {} + + void Logv(const char * format, va_list ap) override + __attribute__((format(printf, 2, 0))) + { + Logv(rocksdb::InfoLogLevel::DEBUG_LEVEL, format, ap); + } + + void Logv(const rocksdb::InfoLogLevel log_level, const char * format, va_list ap) override + __attribute__((format(printf, 3, 0))) + { + if (log_level < GetInfoLogLevel()) + return; + + auto level = rocksdb_logger_map[log_level]; + + /// stack buffer was enough + { + va_list backup_ap; + va_copy(backup_ap, ap); + std::array stack; + if (vsnprintf(stack.data(), stack.size(), format, backup_ap) < static_cast(stack.size())) + { + va_end(backup_ap); + LOG_IMPL(log, level.first, level.second, "{}", stack.data()); + return; + } + va_end(backup_ap); + } + + /// let's try with a bigger dynamic buffer (but not too huge, since + /// some of rocksdb internal code has also such a limitation, i..e + /// HdfsLogger) + { + va_list backup_ap; + va_copy(backup_ap, ap); + static constexpr int buffer_size = 30000; + std::unique_ptr buffer(new char[buffer_size]); + if (vsnprintf(buffer.get(), buffer_size, format, backup_ap) >= buffer_size) + buffer[buffer_size - 1] = 0; + va_end(backup_ap); + LOG_IMPL(log, level.first, level.second, "{}", buffer.get()); + } + } + +private: + LoggerRawPtr log; +}; + void StorageEmbeddedRocksDB::initDB() { rocksdb::Status status; @@ -448,6 +516,7 @@ void StorageEmbeddedRocksDB::initDB() } } + merged.info_log = std::make_shared(merged.info_log_level, log.get()); merged.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); if (ttl > 0) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 61592398954..a6aa1ba36a4 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -114,17 +114,19 @@ public: private: SinkToStoragePtr getSink(ContextPtr context, const StorageMetadataPtr & metadata_snapshot); + LoggerPtr log; + MultiVersion storage_settings; const String primary_key; + using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; + mutable SharedMutex rocksdb_ptr_mx; String rocksdb_dir; Int32 ttl; bool read_only; void initDB(); - - LoggerPtr log; }; } From d10027cc3b7737c524f4cfce262d46753fd03036 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 5 Jun 2024 18:33:13 +0200 Subject: [PATCH 368/856] Enable debug logging for rocksdb on CI To understand the root cause of 02956_rocksdb_bulk_sink failures, this is likely due to some background merge had been done I would say, but it is hard to say without extra information. Signed-off-by: Azat Khuzhin --- tests/config/config.d/rocksdb.xml | 7 +++++++ tests/config/install.sh | 1 + 2 files changed, 8 insertions(+) create mode 100644 tests/config/config.d/rocksdb.xml diff --git a/tests/config/config.d/rocksdb.xml b/tests/config/config.d/rocksdb.xml new file mode 100644 index 00000000000..a3790a3dc1d --- /dev/null +++ b/tests/config/config.d/rocksdb.xml @@ -0,0 +1,7 @@ + + + + DEBUG_LEVEL + + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 6536683b6c2..e04392d893b 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -68,6 +68,7 @@ ln -sf $SRC_PATH/config.d/zero_copy_destructive_operations.xml $DEST_SERVER_PATH ln -sf $SRC_PATH/config.d/block_number.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/handlers.yaml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] From 4ba025c3ba8ceaa1551670f29b328983741f8052 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 15:48:36 +0000 Subject: [PATCH 369/856] Fix duplicating Delete events in blob_storage_log --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index ae719f5cde4..afc13251f5b 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -382,6 +382,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { std::vector current_chunk; String keys; + size_t first_position = current_position; for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position) { Aws::S3::Model::ObjectIdentifier obj; @@ -407,9 +408,9 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError(); auto time_now = std::chrono::system_clock::now(); - for (const auto & object : objects) + for (size_t i = first_position; i < current_position; ++i) blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, - uri.bucket, object.remote_path, object.local_path, object.bytes_size, + uri.bucket, objects[i].remote_path, objects[i].local_path, objects[i].bytes_size, outcome_error, time_now); } From 38d45a6bc99e738a051ccb386e3c0db706051c32 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 15:49:14 +0000 Subject: [PATCH 370/856] Verbose message for errors on inserting to system log --- src/Interpreters/SystemLog.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 5e0ce2cb0de..24e478053bc 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -499,7 +499,10 @@ void SystemLog::flushImpl(const std::vector & to_flush, auto log_element_names_and_types = LogElement::getColumnsDescription(); for (const auto & name_and_type : log_element_names_and_types.getAll()) - log_element_columns.emplace_back(name_and_type.type, name_and_type.name); + { + auto & column = log_element_columns.emplace_back(name_and_type.type, name_and_type.name); + column.column->reserve(to_flush.size()); + } Block block(std::move(log_element_columns)); @@ -532,7 +535,8 @@ void SystemLog::flushImpl(const std::vector & to_flush, } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to flush system log {} with {} entries up to offset {}", + table_id.getNameForLogs(), to_flush.size(), to_flush_end)); } queue->confirm(to_flush_end); From 81fe4c48944eda05c8aad628e0de65b47aeddead Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 17:59:16 +0200 Subject: [PATCH 371/856] Fix number of stream decision --- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/System/StorageSystemZeros.cpp | 35 ++++++++-------------- 2 files changed, 14 insertions(+), 23 deletions(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 5aceef78238..ca9c6fb3226 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -703,7 +703,7 @@ Pipe StorageGenerateRandom::read( } UInt64 query_limit = query_info.limit; - if (query_limit && num_streams * max_block_size < query_limit) + if (query_limit && num_streams * max_block_size > query_limit) { /// We want to avoid spawning more streams than necessary num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 40faf2e265f..7e1d7a14d0e 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -44,17 +44,16 @@ protected: auto column_ptr = column; size_t column_size = column_ptr->size(); - if (state) + UInt64 total_rows = state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + + if (limit) { auto generated_rows = state->num_generated_rows.fetch_add(column_size, std::memory_order_acquire); - if (generated_rows >= limit) return {}; - UInt64 total_rows = state->add_total_rows.fetch_and(0); - if (total_rows) - addTotalRowsApprox(total_rows); - if (generated_rows + column_size > limit) { column_size = limit - generated_rows; @@ -109,30 +108,22 @@ Pipe StorageSystemZeros::read( { storage_snapshot->check(column_names); - bool use_multiple_streams = multithreaded; + UInt64 query_limit = limit ? *limit : 0; + if (query_info.limit) + query_limit = query_limit ? std::min(query_limit, query_info.limit) : query_info.limit; - UInt64 query_limit = limit ? *limit : query_info.limit; - if (query_limit && query_limit > max_block_size) + if (query_limit && query_limit < max_block_size) max_block_size = query_limit; - if (use_multiple_streams && query_limit && num_streams * max_block_size < query_limit) - { + if (!multithreaded) + num_streams = 1; + else if (query_limit && num_streams * max_block_size > query_limit) /// We want to avoid spawning more streams than necessary num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); - if (num_streams <= 1) - use_multiple_streams = false; - } - if (!use_multiple_streams) - num_streams = 1; + ZerosStatePtr state = std::make_shared(query_limit); Pipe res; - - ZerosStatePtr state; - - if (query_limit) - state = std::make_shared(query_limit); - for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared(max_block_size, query_limit, state); From ad2d3979628c5043ad64a127388337f4ac983172 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Jun 2024 16:03:58 +0000 Subject: [PATCH 372/856] Use supportsReplication() --- src/Planner/PlannerJoinTree.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 8d3c8557cd1..d383224b1b2 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -49,7 +49,6 @@ #include #include -#include #include #include @@ -872,8 +871,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!table->isMergeTree()) return false; - if (std::dynamic_pointer_cast(table) - && !query_settings.parallel_replicas_for_non_replicated_merge_tree) + if (!table->supportsReplication() && !query_settings.parallel_replicas_for_non_replicated_merge_tree) return false; return true; From 1881107f0773d4aa0e195ef5cd6bb8f8a17b8cc3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 16:04:46 +0000 Subject: [PATCH 373/856] fix build --- src/Interpreters/SystemLog.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 24e478053bc..3b25deeb59d 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -499,14 +499,15 @@ void SystemLog::flushImpl(const std::vector & to_flush, auto log_element_names_and_types = LogElement::getColumnsDescription(); for (const auto & name_and_type : log_element_names_and_types.getAll()) - { - auto & column = log_element_columns.emplace_back(name_and_type.type, name_and_type.name); - column.column->reserve(to_flush.size()); - } + log_element_columns.emplace_back(name_and_type.type, name_and_type.name); Block block(std::move(log_element_columns)); MutableColumns columns = block.mutateColumns(); + + for (auto & column : columns) + column->reserve(to_flush.size()); + for (const auto & elem : to_flush) elem.appendToBlock(columns); From 2c193a793d197ddf459a0ba0461d5ae908c4db89 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Jun 2024 16:17:36 +0000 Subject: [PATCH 374/856] typo --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ada922cb037..b3e9da816ab 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4622,7 +4622,7 @@ Small allocations and deallocations are grouped in thread local variable and tra Default value: `4MiB`. ## min_untracked_memory {#min_untracked_memory} -Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread memory usage devided by 16 and clamped between `min_untracked_memory` and `max_untracked_memory` for every thread. It guarantees that total untracked memory does not exceed 10% of current memory footprint even with a lot of small threads. To disable dynamic limit for untracked memory set value `4MiB`. +Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread memory usage divided by 16 and clamped between `min_untracked_memory` and `max_untracked_memory` for every thread. It guarantees that total untracked memory does not exceed 10% of current memory footprint even with a lot of small threads. To disable dynamic limit for untracked memory set value `4MiB`. Default value: `4KiB`. From a2c3c215f50d6372ebbe35f15612a65a6906fb93 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Jun 2024 18:43:45 +0200 Subject: [PATCH 375/856] More user friendly --- .../NamedCollectionsFactory.cpp | 2 +- .../NamedCollectionsMetadataStorage.cpp | 19 +++++++++++++++---- .../NamedCollectionsMetadataStorage.h | 2 +- 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.cpp b/src/Common/NamedCollections/NamedCollectionsFactory.cpp index e46fa26896d..4edeb7251b8 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.cpp +++ b/src/Common/NamedCollections/NamedCollectionsFactory.cpp @@ -235,7 +235,7 @@ bool NamedCollectionFactory::loadIfNot(std::lock_guard & lock) loadFromConfig(context->getConfigRef(), lock); loadFromSQL(lock); - if (metadata_storage->requiresPeriodicUpdate()) + if (metadata_storage->supportsPeriodicUpdate()) { update_task = context->getMessageBrokerSchedulePool().createTask("NamedCollectionsMetadataStorage", [this]{ updateFunc(); }); update_task->activate(); diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index a82c0110023..3c0a62e3b0b 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes extern const int NAMED_COLLECTION_DOESNT_EXIST; extern const int INVALID_CONFIG_PARAMETER; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } static const std::string named_collections_storage_config_path = "named_collections_storage"; @@ -66,6 +67,8 @@ public: virtual bool removeIfExists(const std::string & path) = 0; + virtual bool supportsPeriodicUpdate() const = 0; + virtual void waitUpdate(size_t /* timeout */) {} }; @@ -86,6 +89,8 @@ public: ~LocalStorage() override = default; + bool supportsPeriodicUpdate() const override { return false; } + std::vector list() const override { if (!fs::exists(root_path)) @@ -215,6 +220,8 @@ public: ~ZooKeeperStorage() override = default; + bool supportsPeriodicUpdate() const override { return true; } + void waitUpdate(size_t timeout) override { if (wait_event) @@ -430,16 +437,20 @@ void NamedCollectionsMetadataStorage::writeCreateQuery(const ASTCreateNamedColle storage->write(getFileName(query.collection_name), serializeAST(*normalized_query), replace); } -bool NamedCollectionsMetadataStorage::requiresPeriodicUpdate() const +bool NamedCollectionsMetadataStorage::supportsPeriodicUpdate() const { - const auto & config = Context::getGlobalContextInstance()->getConfigRef(); - return config.has(named_collections_storage_config_path + ".update_timeout_ms"); + return storage->supportsPeriodicUpdate(); } void NamedCollectionsMetadataStorage::waitUpdate() { + if (!storage->supportsPeriodicUpdate()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Periodic updates are not supported"); + const auto & config = Context::getGlobalContextInstance()->getConfigRef(); - storage->waitUpdate(config.getUInt(named_collections_storage_config_path + ".update_timeout_ms")); + const size_t timeout = config.getUInt(named_collections_storage_config_path + ".update_timeout_ms", 5000); + + storage->waitUpdate(timeout); } std::unique_ptr NamedCollectionsMetadataStorage::create(const ContextPtr & context_) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h index 059a3c7cf76..4762d55bf99 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h @@ -31,7 +31,7 @@ public: void waitUpdate(); - bool requiresPeriodicUpdate() const; + bool supportsPeriodicUpdate() const; private: class INamedCollectionsStorage; From e42791e9e8dc619efc12e24c505bc14d7a1a2399 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 6 Jun 2024 18:45:43 +0200 Subject: [PATCH 376/856] Add info about MutationsInterpreter --- docs/en/operations/analyzer.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index f5b900d01f7..f89a4416375 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -148,6 +148,12 @@ SELECT toTypeName(if(0, [2, 3, 4], 'String')) The new analyzer significantly changed the communication protocol between servers in the cluster. Thus, it's impossible to run distributed queries on servers with different `allow_experimental_analyzer` setting values. +### Mutations are interpreted by previous analyzer + +Mutations are still using the old analyzer. +This means some new ClickHouse SQL features can't be used in mutations. For example, the `QUALIFY` clause. +Status can be checked [here](https://github.com/ClickHouse/ClickHouse/issues/61563). + ### Unsupported features The list of features new analyzer currently doesn't support: From ef9ef53fde6d3d1c543fd33a6c278956b7b43450 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 6 Jun 2024 18:53:28 +0200 Subject: [PATCH 377/856] Add info about fix --- docs/en/operations/analyzer.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index f89a4416375..2a79653fa00 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -19,6 +19,7 @@ Optimizations could rewrite the initial query so it becomes valid and can be exe In the new infrastructure, query validation takes place before the optimization step. This means that invalid queries that were possible to execute before are now unsupported. +In such cases, the query must be fixed manually. **Example 1:** @@ -45,6 +46,16 @@ HAVING number > 5 The same problem occurs in this query: column `number` is used after aggregation with another key. The previous query analyzer fixed this query by moving the `number > 5` filter from the `HAVING` clause to the `WHERE` clause. +To fix the query, you should move all conditions that apply to non-aggregated columns to the `WHERE` section to conform to standard SQL syntax: +```sql +SELECT + number % 2 AS n, + sum(number) +FROM numbers(10) +WHERE number > 5 +GROUP BY n +``` + ### Known incompatibilities of JOIN clause #### Join using column from projection From 21294013f2df27b9abb56b0fb5a017176689f8ee Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 6 Jun 2024 10:03:52 -0700 Subject: [PATCH 378/856] Update SettingsChangesHistory --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 35266249a2d..7eb14047ace 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -100,7 +100,7 @@ static const std::map Date: Thu, 6 Jun 2024 19:17:34 +0200 Subject: [PATCH 379/856] Add info about invalid VIEWs --- docs/en/operations/analyzer.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 2a79653fa00..7b3e18666ec 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -56,6 +56,23 @@ WHERE number > 5 GROUP BY n ``` +### CREATE VIEW with invalid query + +The new infrastructure always performs type-checking. +Previously, it was possible to create a `VIEW` with an invalid `SELECT` query, and it'd fail during the first SELECT or insert (in the case of `MATERIALIZED VIEW`). + +Now, it's not possible to create such `VIEW`s anymore. + +**Example:** + +```sql +CREATE TABLE source (data String) ENGINE=MergeTree ORDER BY tuple(); + +CREATE VIEW some_view +AS SELECT JSONExtract(data, 'test', 'DateTime64(3)') +FROM source; +``` + ### Known incompatibilities of JOIN clause #### Join using column from projection From 823a7d37f3bab8d4aa9cb8b2ff8b7b087e8a3037 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 6 Jun 2024 20:18:42 +0200 Subject: [PATCH 380/856] support statistics on replicated merge tree --- src/Interpreters/InterpreterCreateQuery.cpp | 6 ++--- src/Parsers/ASTColumnDeclaration.cpp | 10 +++---- src/Parsers/ASTColumnDeclaration.h | 2 +- src/Parsers/ParserCreateQuery.h | 10 +++---- src/Storages/ColumnsDescription.cpp | 4 +++ src/Storages/StatisticsDescription.cpp | 2 +- .../test_manipulate_statistics/test.py | 27 ++++++++++++++++++- 7 files changed, 45 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 66936dc25d7..a51d3e6dade 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -450,8 +450,8 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) if (!column.statistics.empty()) { - column_declaration->stat_type = column.statistics.getAST(); - column_declaration->children.push_back(column_declaration->stat_type); + column_declaration->statistics_desc = column.statistics.getAST(); + column_declaration->children.push_back(column_declaration->statistics_desc); } if (column.ttl) @@ -676,7 +676,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( } column.statistics.column_name = column.name; /// We assign column name here for better exception error message. - if (col_decl.stat_type) + if (col_decl.statistics_desc) { if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics) throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistics is now disabled. Turn on allow_experimental_statistics"); diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 6c29e0bf9d5..4a8a3d2967d 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -39,10 +39,10 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->codec); } - if (stat_type) + if (statistics_desc) { - res->stat_type = stat_type->clone(); - res->children.push_back(res->stat_type); + res->statistics_desc = statistics_desc->clone(); + res->children.push_back(res->statistics_desc); } if (ttl) @@ -111,10 +111,10 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo codec->formatImpl(format_settings, state, frame); } - if (stat_type) + if (statistics_desc) { format_settings.ostr << ' '; - stat_type->formatImpl(format_settings, state, frame); + statistics_desc->formatImpl(format_settings, state, frame); } if (ttl) diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index d775928d05c..914916d5074 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -19,7 +19,7 @@ public: bool ephemeral_default = false; ASTPtr comment; ASTPtr codec; - ASTPtr stat_type; + ASTPtr statistics_desc; ASTPtr ttl; ASTPtr collation; ASTPtr settings; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 27bb524970d..5f6df33176f 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -193,7 +193,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr default_expression; ASTPtr comment_expression; ASTPtr codec_expression; - ASTPtr stat_type_expression; + ASTPtr statistics_desc_expression; ASTPtr ttl_expression; ASTPtr collation_expression; ASTPtr settings; @@ -325,7 +325,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (s_stat.ignore(pos, expected)) { - if (!stat_type_parser.parse(pos, stat_type_expression, expected)) + if (!stat_type_parser.parse(pos, statistics_desc_expression, expected)) return false; } @@ -398,10 +398,10 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(settings)); } - if (stat_type_expression) + if (statistics_desc_expression) { - column_declaration->stat_type = stat_type_expression; - column_declaration->children.push_back(std::move(stat_type_expression)); + column_declaration->statistics_desc = statistics_desc_expression; + column_declaration->children.push_back(std::move(statistics_desc_expression)); } if (ttl_expression) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 69e39323219..d1babb817bf 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -25,6 +25,7 @@ #include #include #include "Parsers/ASTSetQuery.h" +#include "Storages/StatisticsDescription.h" #include #include #include @@ -207,6 +208,9 @@ void ColumnDescription::readText(ReadBuffer & buf) if (col_ast->settings) settings = col_ast->settings->as().changes; + + if (col_ast->statistics_desc) + statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast); } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index dff1b7d3602..fc06c66b50e 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -171,7 +171,7 @@ std::vector ColumnStatisticsDescription::fromAST(co ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column) { - const auto & stat_type_list_ast = column.stat_type->as().arguments; + const auto & stat_type_list_ast = column.statistics_desc->as().arguments; if (stat_type_list_ast->children.empty()) throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column)); ColumnStatisticsDescription stats; diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 2b26af940d1..bffee89ffc6 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -6,9 +6,12 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", user_configs=["config/config.xml"], with_zookeeper=False + "node1", user_configs=["config/config.xml"], with_zookeeper=True ) +node2 = cluster.add_instance( + "node2", user_configs=["config/config.xml"], with_zookeeper=True +) @pytest.fixture(scope="module") def started_cluster(): @@ -122,3 +125,25 @@ def test_single_node_normal(started_cluster): """ ) run_test_single_node(started_cluster) + +def test_replicated_table_ddl(started_cluster): + node1.query("DROP TABLE IF EXISTS test_stat") + node2.query("DROP TABLE IF EXISTS test_stat") + + node1.query( + """ + CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest)) + ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '1') ORDER BY a; + """ + ) + node2.query( + """ + CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest)) + ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '2') ORDER BY a; + """ + ) + + node1.query("ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq", settings={"alter_sync":"2"}); + node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync":"2"}); + + assert node2.query("SHOW CREATE TABLE test_stat") == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" From 0bd5164f83b1751999c667de4ec6327f9a59c5d6 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 6 Jun 2024 21:30:16 +0200 Subject: [PATCH 381/856] fix style --- src/Storages/ColumnsDescription.cpp | 3 +-- .../integration/test_manipulate_statistics/test.py | 14 +++++++++++--- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index d1babb817bf..556f8a6e42d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -24,8 +25,6 @@ #include #include #include -#include "Parsers/ASTSetQuery.h" -#include "Storages/StatisticsDescription.h" #include #include #include diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index bffee89ffc6..53ab9682ad2 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -13,6 +13,7 @@ node2 = cluster.add_instance( "node2", user_configs=["config/config.xml"], with_zookeeper=True ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -126,6 +127,7 @@ def test_single_node_normal(started_cluster): ) run_test_single_node(started_cluster) + def test_replicated_table_ddl(started_cluster): node1.query("DROP TABLE IF EXISTS test_stat") node2.query("DROP TABLE IF EXISTS test_stat") @@ -143,7 +145,13 @@ def test_replicated_table_ddl(started_cluster): """ ) - node1.query("ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq", settings={"alter_sync":"2"}); - node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync":"2"}); + node1.query( + "ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq", + settings={"alter_sync": "2"}, + ) + node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync": "2"}) - assert node2.query("SHOW CREATE TABLE test_stat") == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" + assert ( + node2.query("SHOW CREATE TABLE test_stat") + == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" + ) From d36bf5bac86fb9bfde4fe761a1ab2ab7079f4f1a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 21:40:24 +0200 Subject: [PATCH 382/856] Compiler complains --- src/Storages/System/StorageSystemZeros.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 7e1d7a14d0e..09a2bb5d963 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -16,7 +16,7 @@ namespace struct ZerosState { - ZerosState(UInt64 limit) : add_total_rows(limit) { } + explicit ZerosState(UInt64 limit) : add_total_rows(limit) { } std::atomic num_generated_rows = 0; std::atomic add_total_rows = 0; }; @@ -119,7 +119,7 @@ Pipe StorageSystemZeros::read( num_streams = 1; else if (query_limit && num_streams * max_block_size > query_limit) /// We want to avoid spawning more streams than necessary - num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + num_streams = std::min(num_streams, static_cast(((query_limit + max_block_size - 1) / max_block_size))); ZerosStatePtr state = std::make_shared(query_limit); From bd0ef01ffdcdf1828dd56dde25f8b0d0e65dd1b4 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 6 Jun 2024 16:45:34 -0300 Subject: [PATCH 383/856] try to fix ut --- .../tests/gtest_poco_no_proxy_regex.cpp | 24 +++++++++++++ ...oxy_configuration_to_poco_proxy_config.cpp | 36 ------------------- .../gtest_proxy_environment_configuration.cpp | 7 +++- 3 files changed, 30 insertions(+), 37 deletions(-) create mode 100644 src/Common/tests/gtest_poco_no_proxy_regex.cpp delete mode 100644 src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp diff --git a/src/Common/tests/gtest_poco_no_proxy_regex.cpp b/src/Common/tests/gtest_poco_no_proxy_regex.cpp new file mode 100644 index 00000000000..1f70a483ab4 --- /dev/null +++ b/src/Common/tests/gtest_poco_no_proxy_regex.cpp @@ -0,0 +1,24 @@ +#include + +#include + +TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuild) +{ + ASSERT_EQ( + DB::buildPocoNonProxyHosts("localhost,127.0.0.1,some_other_domain:8080,sub-domain.domain.com"), + R"((?:.*\.)?localhost|(?:.*\.)?127\.0\.0\.1|(?:.*\.)?some_other_domain\:8080|(?:.*\.)?sub\-domain\.domain\.com)"); +} + +TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildMatchAnything) +{ + ASSERT_EQ( + DB::buildPocoNonProxyHosts("*"), + "(.*?)"); +} + +TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildEmpty) +{ + ASSERT_EQ( + DB::buildPocoNonProxyHosts(""), + ""); +} diff --git a/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp b/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp deleted file mode 100644 index db87f23fc65..00000000000 --- a/src/Common/tests/gtest_proxy_configuration_to_poco_proxy_config.cpp +++ /dev/null @@ -1,36 +0,0 @@ -#include - -#include - -TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuild) -{ - DB::ProxyConfiguration proxy_configuration; - - proxy_configuration.no_proxy_hosts = "localhost,127.0.0.1,some_other_domain:8080,sub-domain.domain.com"; - - auto poco_proxy_configuration = DB::proxyConfigurationToPocoProxyConfig(proxy_configuration); - - ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, R"((?:.*\.)?localhost|(?:.*\.)?127\.0\.0\.1|(?:.*\.)?some_other_domain\:8080|(?:.*\.)?sub\-domain\.domain\.com)"); -} - -TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildMatchAnything) -{ - DB::ProxyConfiguration proxy_configuration; - - proxy_configuration.no_proxy_hosts = "*"; - - auto poco_proxy_configuration = DB::proxyConfigurationToPocoProxyConfig(proxy_configuration); - - ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, "(.*?)"); -} - -TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildEmpty) -{ - DB::ProxyConfiguration proxy_configuration; - - proxy_configuration.no_proxy_hosts = ""; - - auto poco_proxy_configuration = DB::proxyConfigurationToPocoProxyConfig(proxy_configuration); - - ASSERT_EQ(poco_proxy_configuration.nonProxyHosts, ""); -} diff --git a/src/Common/tests/gtest_proxy_environment_configuration.cpp b/src/Common/tests/gtest_proxy_environment_configuration.cpp index 095e44dfb86..ac31e33e129 100644 --- a/src/Common/tests/gtest_proxy_environment_configuration.cpp +++ b/src/Common/tests/gtest_proxy_environment_configuration.cpp @@ -16,7 +16,12 @@ namespace TEST(EnvironmentProxyConfigurationResolver, TestHTTPandHTTPS) { - std::string no_proxy_string = "localhost,,127.0.0.1,some_other_domain,,,, last_domain,"; + // Some other tests rely on HTTP clients (e.g, gtest_aws_s3_client), which depend on proxy configuration + // since in https://github.com/ClickHouse/ClickHouse/pull/63314 the environment proxy resolver reads only once + // from the environment, the proxy configuration will always be there. + // The problem is that the proxy server does not exist, causing the test to fail. + // To work around this issue, `no_proxy` is set to bypass all domains. + std::string no_proxy_string = "*"; std::string poco_no_proxy_regex = buildPocoNonProxyHosts(no_proxy_string); EnvironmentProxySetter setter(http_proxy_server, https_proxy_server, no_proxy_string); From 258b1f9559673b3ebe6da0eae728aa783955045e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 6 Jun 2024 21:00:47 +0000 Subject: [PATCH 384/856] time_virtual_col: tests, doc, small refactoring --- .../integrations/azureBlobStorage.md | 1 + .../table-engines/integrations/hdfs.md | 1 + .../engines/table-engines/integrations/s3.md | 11 +++++---- docs/en/engines/table-engines/special/file.md | 1 + docs/en/engines/table-engines/special/url.md | 1 + .../table-functions/azureBlobStorage.md | 1 + docs/en/sql-reference/table-functions/file.md | 1 + docs/en/sql-reference/table-functions/hdfs.md | 1 + docs/en/sql-reference/table-functions/s3.md | 1 + docs/en/sql-reference/table-functions/url.md | 1 + .../StorageObjectStorageSource.cpp | 14 ++++++----- src/Storages/S3Queue/S3QueueSource.cpp | 10 ++++++-- src/Storages/StorageFile.cpp | 11 +++++++-- src/Storages/StorageURL.cpp | 7 +++++- src/Storages/VirtualColumnUtils.cpp | 23 ++++++++++--------- src/Storages/VirtualColumnUtils.h | 14 +++++++++-- .../test_storage_azure_blob_storage/test.py | 4 ++-- tests/integration/test_storage_hdfs/test.py | 4 ++-- tests/integration/test_storage_s3/test.py | 4 ++-- ...e_structure_from_insertion_table.reference | 1 - ...lumn_use_structure_from_insertion_table.sh | 13 ----------- ...e_structure_from_insertion_table.reference | 1 + ...lumn_use_structure_from_insertion_table.sh | 14 +++++++++++ 23 files changed, 91 insertions(+), 49 deletions(-) delete mode 100644 tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference delete mode 100755 tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh create mode 100644 tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference create mode 100755 tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index 0843ff1ac47..dfc27d6b8cf 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -54,6 +54,7 @@ SELECT * FROM test_table; - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## See also diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 2749fa7e479..c9df713231a 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -235,6 +235,7 @@ libhdfs3 support HDFS namenode HA. - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index cb1da1c8e68..93f4a187656 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -53,14 +53,14 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da This example uses the [docker compose recipe](https://github.com/ClickHouse/examples/tree/5fdc6ff72f4e5137e23ea075c88d3f44b0202490/docker-compose-recipes/recipes/ch-and-minio-S3), which integrates ClickHouse and MinIO. You should be able to reproduce the same queries using S3 by replacing the endpoint and authentication values. -Notice that the S3 endpoint in the `ENGINE` configuration uses the parameter token `{_partition_id}` as part of the S3 object (filename), and that the SELECT queries select against those resulting object names (e.g., `test_3.csv`). +Notice that the S3 endpoint in the `ENGINE` configuration uses the parameter token `{_partition_id}` as part of the S3 object (filename), and that the SELECT queries select against those resulting object names (e.g., `test_3.csv`). :::note As shown in the example, querying from S3 tables that are partitioned is not directly supported at this time, but can be accomplished by querying the individual partitions using the S3 table function. -The primary use-case for writing +The primary use-case for writing partitioned data in S3 is to enable transferring that data into another ClickHouse system (for example, moving from on-prem systems to ClickHouse Cloud). Because ClickHouse datasets are often very large, and network @@ -78,9 +78,9 @@ CREATE TABLE p ) ENGINE = S3( # highlight-next-line - 'http://minio:10000/clickhouse//test_{_partition_id}.csv', - 'minioadmin', - 'minioadminpassword', + 'http://minio:10000/clickhouse//test_{_partition_id}.csv', + 'minioadmin', + 'minioadminpassword', 'CSV') PARTITION BY column3 ``` @@ -145,6 +145,7 @@ Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Reading fr - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index 0d422f64762..957b18b5305 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -102,6 +102,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Settings {#settings} diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index f6183a779ae..c906830d0e9 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -108,6 +108,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - `_path` — Path to the `URL`. Type: `LowCardinalty(String)`. - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 1510489ce83..f59fedeb3a2 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -72,6 +72,7 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. **See Also** diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index f66178afbb2..4fec772c373 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -196,6 +196,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Settings {#settings} diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index d65615e7588..28cba5ccc6a 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -97,6 +97,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index cbef80371a3..1a7e2b8d66a 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -272,6 +272,7 @@ FROM s3( - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 4dc6e435b50..3bb7aff53a7 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -53,6 +53,7 @@ Character `|` inside patterns is used to specify failover addresses. They are it - `_path` — Path to the `URL`. Type: `LowCardinalty(String)`. - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 8d5c03ae11e..2fc6993369d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -195,12 +195,14 @@ Chunk StorageObjectStorageSource::generate() const auto & object_info = reader.getObjectInfo(); const auto & filename = object_info.getFileName(); chassert(object_info.metadata); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, - read_from_format_info.requested_virtual_columns, - getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), - object_info.metadata->size_bytes, &filename, object_info.metadata->last_modified); - + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, read_from_format_info.requested_virtual_columns, + { + .path = getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), + .size = object_info.metadata->size_bytes, + .filename = &filename, + .last_modified = object_info.metadata->last_modified + }); return chunk; } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index d8633037ed9..b5b1a8dd992 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -421,8 +421,14 @@ Chunk StorageS3QueueSource::generate() file_status->processed_rows += chunk.getNumRows(); processed_rows_from_file += chunk.getNumRows(); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, path, reader.getObjectInfo().metadata->size_bytes); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = path, + .size = reader.getObjectInfo().metadata->size_bytes + }); + + return chunk; } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f84d3380c3f..09f0bd60859 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1438,8 +1438,15 @@ Chunk StorageFileSource::generate() progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); /// Enrich with virtual columns. - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, current_path, current_file_size, filename_override.has_value() ? &filename_override.value() : nullptr, current_file_last_modified); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = current_path, + .size = current_file_size, + .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), + .last_modified = current_file_last_modified + }); + return chunk; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 8d1c6933503..9302e7ef3e5 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -411,7 +411,12 @@ Chunk StorageURLSource::generate() if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, curr_uri.getPath(), current_file_size); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = curr_uri.getPath(), + .size = current_file_size + }); return chunk; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 1e39d64fb18..778c9e13adb 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -189,39 +189,40 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } -void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename, std::optional last_modified) +void addRequestedFileLikeStorageVirtualsToChunk( + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, + VirtualsForFileLikeStorage virtual_values) { for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") { - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), path)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), virtual_values.path)->convertToFullColumnIfConst()); } else if (virtual_column.name == "_file") { - if (filename) + if (virtual_values.filename) { - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *filename)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), (*virtual_values.filename))->convertToFullColumnIfConst()); } else { - size_t last_slash_pos = path.find_last_of('/'); - auto filename_from_path = path.substr(last_slash_pos + 1); + size_t last_slash_pos = virtual_values.path.find_last_of('/'); + auto filename_from_path = virtual_values.path.substr(last_slash_pos + 1); chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), filename_from_path)->convertToFullColumnIfConst()); } } else if (virtual_column.name == "_size") { - if (size) - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *size)->convertToFullColumnIfConst()); + if (virtual_values.size) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *virtual_values.size)->convertToFullColumnIfConst()); else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } else if (virtual_column.name == "_time") { - if (last_modified) - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), last_modified->epochTime())->convertToFullColumnIfConst()); + if (virtual_values.last_modified) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), virtual_values.last_modified->epochTime())->convertToFullColumnIfConst()); else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 65826e1f1e7..fbfbdd6c6cc 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -68,8 +68,18 @@ void filterByPathOrFile(std::vector & sources, const std::vector & pa sources = std::move(filtered_sources); } -void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr, std::optional last_modified = std::nullopt); +struct VirtualsForFileLikeStorage +{ + const String & path; + std::optional size { std::nullopt }; + const String * filename { nullptr }; + std::optional last_modified { std::nullopt }; + +}; + +void addRequestedFileLikeStorageVirtualsToChunk( + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, + VirtualsForFileLikeStorage virtual_values); } } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f836c58ce30..9f5aef1489c 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -758,12 +758,12 @@ def test_read_subcolumns(cluster): ) res = node.query( - f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv'," + f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()), a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv'," f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" + assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t0\t3\n" res = node.query( f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 98ea79d6ee4..cda2b8694c6 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -987,10 +987,10 @@ def test_read_subcolumns(started_cluster): assert res == "2\ttest_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( - f"select x.b.d, _path, x.b, _file, _time, now(), x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select x.b.d, _path, x.b, _file, dateDiff('minute', _time, now()), x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" + assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\t0\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 5becdf30476..f7f7bbe335c 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2117,10 +2117,10 @@ def test_read_subcolumns(started_cluster): assert res == "0\troot/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = instance.query( - f"select x.b.d, _path, x.b, _file, _time, now(), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + f"select x.b.d, _path, x.b, _file, dateDiff('minute', _time, now()), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) - assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" + assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t0\t42\n" res = instance.query( f"select a.b.d, _path, a.b, _file, a.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" diff --git a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference deleted file mode 100644 index 35ef86f5339..00000000000 --- a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference +++ /dev/null @@ -1 +0,0 @@ -1 2 4 diff --git a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh deleted file mode 100755 index d9e4a2c8f8b..00000000000 --- a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " -create table test (x UInt64, y UInt32, size UInt64) engine=Memory; -insert into test select c1, c2, _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') settings use_structure_from_insertion_table_in_table_functions=1; -select * from test; -" -rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv diff --git a/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference new file mode 100644 index 00000000000..93acdc34842 --- /dev/null +++ b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference @@ -0,0 +1 @@ +1 2 4 1 1 diff --git a/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh new file mode 100755 index 00000000000..ebdda0cc1d3 --- /dev/null +++ b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv +sleep 1 +$CLICKHOUSE_LOCAL -nm -q " +create table test (x UInt64, y UInt32, size UInt64, d32 DateTime32, d64 DateTime64) engine=Memory; +insert into test select c1, c2, _size, _time, _time from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') settings use_structure_from_insertion_table_in_table_functions=1; +select x, y, size, (dateDiff('millisecond', d32, now()) < 4000 AND dateDiff('millisecond', d32, now()) > 0), (dateDiff('second', d64, now()) < 4 AND dateDiff('second', d64, now()) > 0) from test; +" +rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv From b82ab5a4028f620d4abe920745945983ae9f264c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 6 Jun 2024 21:21:12 +0000 Subject: [PATCH 385/856] time_virtual_col: style check --- tests/integration/test_storage_s3/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index f7f7bbe335c..61c6d95f123 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2120,7 +2120,9 @@ def test_read_subcolumns(started_cluster): f"select x.b.d, _path, x.b, _file, dateDiff('minute', _time, now()), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) - assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t0\t42\n" + assert ( + res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t0\t42\n" + ) res = instance.query( f"select a.b.d, _path, a.b, _file, a.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" @@ -2150,6 +2152,7 @@ def test_read_subcolumns(started_cluster): logging.info("Some custom logging") + def test_filtering_by_file_or_path(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] From eb72c12b31560dad49caff2e532472e8920f38d5 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 00:51:37 +0200 Subject: [PATCH 386/856] CI: Minor fixes in ci scripts --- tests/ci/ci_settings.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index 62e7826dac5..7807cc7ac10 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -211,12 +211,15 @@ class CiSettings: ): res[job] = job_config + add_parents = [] for job in list(res): parent_jobs = CI_CONFIG.get_job_parents(job) for parent_job in parent_jobs: if parent_job not in res: + add_parents.append(parent_job) print(f"Job [{job}] requires [{parent_job}] - add") - res[parent_job] = job_configs[parent_job] + for job in add_parents: + res[job] = job_configs[job] for job, job_config in res.items(): batches = [] From 0deb862c93824146cf9012f95fa247e459c3683d Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 01:22:47 +0200 Subject: [PATCH 387/856] Re-enable Fast test in MQ --- tests/ci/ci.py | 6 +++++- tests/ci/ci_settings.py | 6 ++++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 55a18a2f335..ec6e84dea8c 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -442,7 +442,11 @@ def _configure_jobs( # filter jobs in accordance with ci settings job_configs = ci_settings.apply( - job_configs, pr_info.is_release, is_pr=pr_info.is_pr, labels=pr_info.labels + job_configs, + pr_info.is_release, + is_pr=pr_info.is_pr, + is_mq=pr_info.is_merge_queue, + labels=pr_info.labels, ) # check jobs in ci cache diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index 7807cc7ac10..83d4ddb4211 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -134,6 +134,7 @@ class CiSettings: job_config: JobConfig, is_release: bool, is_pr: bool, + is_mq: bool, labels: Iterable[str], ) -> bool: # type: ignore #too-many-return-statements if self.do_not_test: @@ -189,7 +190,7 @@ class CiSettings: if job_config.release_only and not is_release: return False - elif job_config.pr_only and not is_pr: + elif job_config.pr_only and not is_pr and not is_mq: return False return not to_deny @@ -199,6 +200,7 @@ class CiSettings: job_configs: Dict[str, JobConfig], is_release: bool, is_pr: bool, + is_mq: bool, labels: Iterable[str], ) -> Dict[str, JobConfig]: """ @@ -207,7 +209,7 @@ class CiSettings: res = {} for job, job_config in job_configs.items(): if self._check_if_selected( - job, job_config, is_release=is_release, is_pr=is_pr, labels=labels + job, job_config, is_release=is_release, is_pr=is_pr, is_mq=is_mq, labels=labels ): res[job] = job_config From 8f26f77505a08197f236f6e2cc069cd4111d71ec Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 01:35:39 +0200 Subject: [PATCH 388/856] fix unit test --- tests/ci/test_ci_options.py | 58 +++++++++++++++++++++++++++---------- 1 file changed, 42 insertions(+), 16 deletions(-) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index c00cd0b9216..e6fa0389649 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -167,19 +167,19 @@ class TestCIOptions(unittest.TestCase): ) jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST} - jobs_configs[ - "fuzzers" - ].run_by_label = ( + jobs_configs["fuzzers"].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs[ - "Integration tests (asan)" - ].release_only = ( + jobs_configs["Integration tests (asan)"].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( ci_options.apply( - jobs_configs, is_release=False, is_pr=True, labels=["TEST_LABEL"] + jobs_configs, + is_release=False, + is_pr=True, + is_mq=False, + labels=["TEST_LABEL"], ) ) self.assertCountEqual( @@ -212,7 +212,9 @@ class TestCIOptions(unittest.TestCase): jobs_configs["fuzzers"].run_by_label = "TEST_LABEL" # no settings are set filtered_jobs = list( - CiSettings().apply(jobs_configs, is_release=False, is_pr=True, labels=[]) + CiSettings().apply( + jobs_configs, is_release=False, is_pr=False, is_mq=True, labels=[] + ) ) self.assertCountEqual( filtered_jobs, @@ -220,9 +222,21 @@ class TestCIOptions(unittest.TestCase): "Fast test", ], ) - filtered_jobs = list( - CiSettings().apply(jobs_configs, is_release=True, is_pr=False, labels=[]) + CiSettings().apply( + jobs_configs, is_release=False, is_pr=True, is_mq=False, labels=[] + ) + ) + self.assertCountEqual( + filtered_jobs, + [ + "Fast test", + ], + ) + filtered_jobs = list( + CiSettings().apply( + jobs_configs, is_release=True, is_pr=False, is_mq=False, labels=[] + ) ) self.assertCountEqual( filtered_jobs, @@ -240,7 +254,11 @@ class TestCIOptions(unittest.TestCase): # no settings are set filtered_jobs = list( ci_settings.apply( - jobs_configs, is_release=False, is_pr=True, labels=["TEST_LABEL"] + jobs_configs, + is_release=False, + is_pr=True, + is_mq=False, + labels=["TEST_LABEL"], ) ) self.assertCountEqual( @@ -253,7 +271,11 @@ class TestCIOptions(unittest.TestCase): ci_settings.include_keywords = ["Fast"] filtered_jobs = list( ci_settings.apply( - jobs_configs, is_release=True, is_pr=False, labels=["TEST_LABEL"] + jobs_configs, + is_release=True, + is_pr=False, + is_mq=False, + labels=["TEST_LABEL"], ) ) self.assertCountEqual( @@ -271,13 +293,17 @@ class TestCIOptions(unittest.TestCase): self.assertCountEqual(ci_options.include_keywords, ["analyzer"]) self.assertIsNone(ci_options.exclude_keywords) jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST} - jobs_configs[ - "fuzzers" - ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result + jobs_configs["fuzzers"].run_by_label = ( + "TEST_LABEL" # check "fuzzers" does not appears in the result + ) jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( - jobs_configs, is_release=False, is_pr=True, labels=["TEST_LABEL"] + jobs_configs, + is_release=False, + is_pr=True, + is_mq=False, + labels=["TEST_LABEL"], ) ) self.assertCountEqual( From 367d41e7f042137d4a25b55c740ba3835b5d5435 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 6 Jun 2024 23:41:25 +0000 Subject: [PATCH 389/856] Automatic style fix --- tests/ci/ci_settings.py | 7 ++++++- tests/ci/test_ci_options.py | 14 +++++++++----- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index 83d4ddb4211..f25344c7701 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -209,7 +209,12 @@ class CiSettings: res = {} for job, job_config in job_configs.items(): if self._check_if_selected( - job, job_config, is_release=is_release, is_pr=is_pr, is_mq=is_mq, labels=labels + job, + job_config, + is_release=is_release, + is_pr=is_pr, + is_mq=is_mq, + labels=labels, ): res[job] = job_config diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index e6fa0389649..60888932803 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -167,10 +167,14 @@ class TestCIOptions(unittest.TestCase): ) jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST} - jobs_configs["fuzzers"].run_by_label = ( + jobs_configs[ + "fuzzers" + ].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs["Integration tests (asan)"].release_only = ( + jobs_configs[ + "Integration tests (asan)" + ].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -293,9 +297,9 @@ class TestCIOptions(unittest.TestCase): self.assertCountEqual(ci_options.include_keywords, ["analyzer"]) self.assertIsNone(ci_options.exclude_keywords) jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST} - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" does not appears in the result - ) + jobs_configs[ + "fuzzers" + ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 05ea297c0d79f55f65ded88ec5284fa9e8c34753 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Jun 2024 03:47:35 +0200 Subject: [PATCH 390/856] Fix error message (it was strange) --- src/Functions/FunctionHelpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index c7a0c3c58ca..593646240ca 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -314,7 +314,7 @@ void checkFunctionArgumentSizes(const ColumnsWithTypeAndName & arguments, size_t if (current_size != input_rows_count) throw Exception( ErrorCodes::LOGICAL_ERROR, - "Expected the argument nº#{} ('{}' of type {}) to have {} rows, but it has {}", + "Expected the argument №{} ('{}' of type {}) to have {} rows, but it has {}", i + 1, arguments[i].name, arguments[i].type->getName(), From 8a2e1870c757f3018698efc82f589cfc33c1c84a Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 7 Jun 2024 02:20:03 -0400 Subject: [PATCH 391/856] 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: Fri, 7 Jun 2024 08:43:49 +0200 Subject: [PATCH 392/856] 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 49f43615c7e..826c6953a10 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1934,6 +1934,8 @@ london lookups loongarch lowcardinality +lowCardinalityIndices +lowCardinalityKeys lowerUTF lowercased lttb From de5258128e48061514a82bdfdc5368852d6a5062 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 7 Jun 2024 06:44:36 +0000 Subject: [PATCH 393/856] update fmtlib version to 9.1.0 Signed-off-by: Duc Canh Le --- base/base/EnumReflection.h | 2 +- base/base/wide_integer_to_string.h | 2 +- contrib/fmtlib | 2 +- contrib/fmtlib-cmake/CMakeLists.txt | 1 - src/Analyzer/Identifier.h | 4 ++-- src/Client/TestHint.h | 2 +- src/Common/TransactionID.h | 2 +- src/Common/ZooKeeper/IKeeper.h | 2 +- src/Common/filesystemHelpers.cpp | 13 ++++++++----- src/Common/formatReadable.h | 2 +- src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/RaftServerConfig.h | 4 ++-- src/Core/Field.h | 2 +- src/Core/QualifiedTableName.h | 2 +- src/DataTypes/IDataType.h | 2 +- src/Functions/FunctionsStringDistance.cpp | 4 ++-- src/IO/WriteHelpers.h | 2 +- src/Interpreters/StorageID.h | 2 +- src/Parsers/formatAST.h | 2 +- .../Formats/Impl/BSONEachRowRowInputFormat.cpp | 4 ++-- src/Processors/Transforms/WindowTransform.cpp | 2 +- src/Storages/Kafka/KafkaConsumer.cpp | 5 +---- src/Storages/Kafka/KafkaConsumer.h | 5 +++++ src/Storages/MergeTree/MarkRange.h | 2 +- .../ParallelReplicasReadingCoordinator.cpp | 2 +- src/Storages/MergeTree/RangesInDataPart.cpp | 2 +- src/Storages/NamedCollectionsHelpers.h | 2 +- 27 files changed, 41 insertions(+), 37 deletions(-) diff --git a/base/base/EnumReflection.h b/base/base/EnumReflection.h index 4a9de4d17a3..e4e0ef672fd 100644 --- a/base/base/EnumReflection.h +++ b/base/base/EnumReflection.h @@ -32,7 +32,7 @@ constexpr void static_for(F && f) template struct fmt::formatter : fmt::formatter { - constexpr auto format(T value, auto& format_context) + constexpr auto format(T value, auto& format_context) const { return formatter::format(magic_enum::enum_name(value), format_context); } diff --git a/base/base/wide_integer_to_string.h b/base/base/wide_integer_to_string.h index c2cbe8d82e3..f703a722afa 100644 --- a/base/base/wide_integer_to_string.h +++ b/base/base/wide_integer_to_string.h @@ -62,7 +62,7 @@ struct fmt::formatter> } template - auto format(const wide::integer & value, FormatContext & ctx) + auto format(const wide::integer & value, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", to_string(value)); } diff --git a/contrib/fmtlib b/contrib/fmtlib index b6f4ceaed0a..a33701196ad 160000 --- a/contrib/fmtlib +++ b/contrib/fmtlib @@ -1 +1 @@ -Subproject commit b6f4ceaed0a0a24ccf575fab6c56dd50ccf6f1a9 +Subproject commit a33701196adfad74917046096bf5a2aa0ab0bb50 diff --git a/contrib/fmtlib-cmake/CMakeLists.txt b/contrib/fmtlib-cmake/CMakeLists.txt index fe399ddc6e1..6625e411295 100644 --- a/contrib/fmtlib-cmake/CMakeLists.txt +++ b/contrib/fmtlib-cmake/CMakeLists.txt @@ -13,7 +13,6 @@ set (SRCS ${FMT_SOURCE_DIR}/include/fmt/core.h ${FMT_SOURCE_DIR}/include/fmt/format.h ${FMT_SOURCE_DIR}/include/fmt/format-inl.h - ${FMT_SOURCE_DIR}/include/fmt/locale.h ${FMT_SOURCE_DIR}/include/fmt/os.h ${FMT_SOURCE_DIR}/include/fmt/ostream.h ${FMT_SOURCE_DIR}/include/fmt/printf.h diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index cbd8f5e7694..91190dc7cdb 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -406,7 +406,7 @@ struct fmt::formatter } template - auto format(const DB::Identifier & identifier, FormatContext & ctx) + auto format(const DB::Identifier & identifier, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", identifier.getFullName()); } @@ -428,7 +428,7 @@ struct fmt::formatter } template - auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx) + auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", identifier_view.getFullName()); } diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index eaf854be5df..b76c4245df4 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -112,7 +112,7 @@ struct fmt::formatter } template - auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx) + auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx) const { if (ErrorVector.empty()) return fmt::format_to(ctx.out(), "{}", 0); diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 97d0072bc14..466f3f5343b 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -108,7 +108,7 @@ struct fmt::formatter } template - auto format(const DB::TransactionID & tid, FormatContext & context) + auto format(const DB::TransactionID & tid, FormatContext & context) const { return fmt::format_to(context.out(), "({}, {}, {})", tid.start_csn, tid.local_tid, tid.host_id); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index ddd30c4eef2..7d574247aa5 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -647,7 +647,7 @@ public: template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(Coordination::Error code, auto & ctx) + constexpr auto format(Coordination::Error code, auto & ctx) const { return formatter::format(Coordination::errorMessage(code), ctx); } diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 2d053c615d9..f8d209bc11f 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -1,4 +1,6 @@ #include "filesystemHelpers.h" +#include +#include #if defined(OS_LINUX) # include @@ -11,7 +13,7 @@ #include #include #include -#include +#include #include #include #include @@ -369,10 +371,11 @@ Poco::Timestamp getModificationTimestamp(const std::string & path) void setModificationTime(const std::string & path, time_t time) { - struct utimbuf tb; - tb.actime = time; - tb.modtime = time; - if (utime(path.c_str(), &tb) != 0) + struct timeval times[2]; + times[0].tv_usec = times[1].tv_usec = 0; + times[0].tv_sec = ::time(nullptr); + times[1].tv_sec = time; + if (utimes(path.c_str(), times) != 0) DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot set modification time to file: {}", path); } diff --git a/src/Common/formatReadable.h b/src/Common/formatReadable.h index a05a2a7f9e2..0d7a437219a 100644 --- a/src/Common/formatReadable.h +++ b/src/Common/formatReadable.h @@ -49,7 +49,7 @@ struct fmt::formatter } template - auto format(const ReadableSize & size, FormatContext & ctx) + auto format(const ReadableSize & size, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", formatReadableSizeWithBinarySuffix(size.value)); } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index f25ccab86b1..23ff714a929 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -466,7 +466,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial #ifdef NDEBUG /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key); + " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key.toView()); #else throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" " is different from actual children size {} for node {}", diff --git a/src/Coordination/RaftServerConfig.h b/src/Coordination/RaftServerConfig.h index 0ecbd6464c1..37b6a92ba70 100644 --- a/src/Coordination/RaftServerConfig.h +++ b/src/Coordination/RaftServerConfig.h @@ -57,7 +57,7 @@ using ClusterUpdateActions = std::vector; template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) + constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) const { return fmt::format_to( ctx.out(), "server.{}={};{};{}", server.id, server.endpoint, server.learner ? "learner" : "participant", server.priority); @@ -67,7 +67,7 @@ struct fmt::formatter : fmt::formatter template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) + constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) const { if (const auto * add = std::get_if(&action)) return fmt::format_to(ctx.out(), "(Add server {})", add->id); diff --git a/src/Core/Field.h b/src/Core/Field.h index a78b589c883..f1bb4a72b0d 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -1038,7 +1038,7 @@ struct fmt::formatter } template - auto format(const DB::Field & x, FormatContext & ctx) + auto format(const DB::Field & x, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", toString(x)); } diff --git a/src/Core/QualifiedTableName.h b/src/Core/QualifiedTableName.h index bf05bd59caf..0fd72c32a54 100644 --- a/src/Core/QualifiedTableName.h +++ b/src/Core/QualifiedTableName.h @@ -125,7 +125,7 @@ namespace fmt } template - auto format(const DB::QualifiedTableName & name, FormatContext & ctx) + auto format(const DB::QualifiedTableName & name, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}.{}", DB::backQuoteIfNeed(name.database), DB::backQuoteIfNeed(name.table)); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 46c30240ef8..bd6065ca270 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -623,7 +623,7 @@ struct fmt::formatter } template - auto format(const DB::DataTypePtr & type, FormatContext & ctx) + auto format(const DB::DataTypePtr & type, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", type->getName()); } diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 6cb23bbea9f..d0d8ebc946d 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -159,7 +159,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(haystack, haystack_end - haystack)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(haystack, haystack_end - haystack)); } } } @@ -186,7 +186,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(needle, needle_end - needle)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(needle, needle_end - needle)); } } } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index d4b2d8ea0dc..6b0de441e94 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1420,7 +1420,7 @@ struct fmt::formatter } template - auto format(const DB::UUID & uuid, FormatContext & context) + auto format(const DB::UUID & uuid, FormatContext & context) const { return fmt::format_to(context.out(), "{}", toString(uuid)); } diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index 96e3cefe00c..69dac8ea32d 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -136,7 +136,7 @@ namespace fmt } template - auto format(const DB::StorageID & storage_id, FormatContext & ctx) + auto format(const DB::StorageID & storage_id, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", storage_id.getNameForLogs()); } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index dd72a59b4a2..e34902663dd 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -40,7 +40,7 @@ struct fmt::formatter } template - auto format(const DB::ASTPtr & ast, FormatContext & context) + auto format(const DB::ASTPtr & ast, FormatContext & context) const { return fmt::format_to(context.out(), "{}", DB::serializeAST(*ast)); } diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 6a3475a1830..56c774782c2 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -421,7 +421,7 @@ void BSONEachRowRowInputFormat::readTuple(IColumn & column, const DataTypePtr & "Cannot parse tuple column with type {} from BSON array/embedded document field: " "tuple doesn't have element with name \"{}\"", data_type->getName(), - name); + name.toView()); index = *try_get_index; } @@ -806,7 +806,7 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi else { if (seen_columns[index]) - throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name); + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name.toView()); seen_columns[index] = true; read_columns[index] = readField(*columns[index], types[index], BSONType(type)); diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index af340c4aab8..b9f61d30182 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -37,7 +37,7 @@ struct fmt::formatter } template - auto format(const DB::RowNumber & x, FormatContext & ctx) + auto format(const DB::RowNumber & x, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}:{}", x.block, x.row); } diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 7075dcb71ca..9ba42b9875e 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -1,7 +1,4 @@ -// Needs to go first because its partial specialization of fmt::formatter -// should be defined before any instantiation -#include - +#include #include #include diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index a3bc97779b3..a2d047933be 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include @@ -197,3 +199,6 @@ private: }; } + +template <> struct fmt::formatter : fmt::ostream_formatter {}; +template <> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/src/Storages/MergeTree/MarkRange.h b/src/Storages/MergeTree/MarkRange.h index 626d4e9e689..6b111f348bb 100644 --- a/src/Storages/MergeTree/MarkRange.h +++ b/src/Storages/MergeTree/MarkRange.h @@ -69,7 +69,7 @@ struct fmt::formatter } template - auto format(const DB::MarkRange & range, FormatContext & ctx) + auto format(const DB::MarkRange & range, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", fmt::format("({}, {})", range.begin, range.end)); } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f3318a48883..79c0e6ad262 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -112,7 +112,7 @@ struct fmt::formatter static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); } template - auto format(const DB::Part & part, FormatContext & ctx) + auto format(const DB::Part & part, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{} in replicas [{}]", part.description.describe(), fmt::join(part.replicas, ", ")); } diff --git a/src/Storages/MergeTree/RangesInDataPart.cpp b/src/Storages/MergeTree/RangesInDataPart.cpp index c46385e84ef..50e0781b4e6 100644 --- a/src/Storages/MergeTree/RangesInDataPart.cpp +++ b/src/Storages/MergeTree/RangesInDataPart.cpp @@ -13,7 +13,7 @@ struct fmt::formatter static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); } template - auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx) + auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", range.describe()); } diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index a1909f514ea..b4aea096c59 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -158,7 +158,7 @@ struct fmt::formatter> } template - auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) + auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) const { return fmt::format_to(context.out(), "{}", elem.value); } From 18ced447efe9ab612362a584e05dd4edb02a3b87 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 7 Jun 2024 08:45:39 +0200 Subject: [PATCH 394/856] Fix typo II --- .../sql-reference/aggregate-functions/reference/flame_graph.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md b/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md index e09769477f1..ae17153085c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md +++ b/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md @@ -26,7 +26,7 @@ Only allocations which were not freed are shown. Non mapped deallocations are ig ## Returned value -- An array of strings for use with [flamegraph.pl util](https://github.com/brendangregg/FlameGraph). [Array](../../data-types/array.md)([String](../../data-types/string.md)). +- An array of strings for use with [flamegraph.pl utility](https://github.com/brendangregg/FlameGraph). [Array](../../data-types/array.md)([String](../../data-types/string.md)). ## Examples From f2700f551034f5c45f80ab4aeb5777198625a74d Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 7 Jun 2024 01:48:02 -0700 Subject: [PATCH 395/856] Analyzer docs changes --- docs/en/operations/analyzer.md | 41 ++++++++++++++++++---------------- 1 file changed, 22 insertions(+), 19 deletions(-) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 7b3e18666ec..e85c5fa9e8d 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -1,6 +1,9 @@ --- slug: /en/operations/analyzer sidebar_label: Analyzer +title: Analyzer +description: Details about ClickHouse's query analyzer +keywords: [analyzer] --- # Analyzer @@ -9,15 +12,15 @@ sidebar_label: Analyzer ## Known incompatibilities -In ClickHouse version `24.3`, new query analysis was enabled by default. -Despite fixing a large number of bugs and introducing new optimizations, it also introduces some breaking changes in ClickHouse behaviour. +In ClickHouse version `24.3`, the new query analyzer was enabled by default. +Despite fixing a large number of bugs and introducing new optimizations, it also introduces some breaking changes in ClickHouse behaviour. Please read the following changes to determine how to rewrite your queries for the new analyzer. -### Invalid before optimization queries +### Invalid queries are no longer optimized The previous query planning infrastructure applied AST-level optimizations before the query validation step. Optimizations could rewrite the initial query so it becomes valid and can be executed. -In the new infrastructure, query validation takes place before the optimization step. +In the new analyzer, query validation takes place before the optimization step. This means that invalid queries that were possible to execute before are now unsupported. In such cases, the query must be fixed manually. @@ -30,7 +33,7 @@ GROUP BY toString(number) ``` The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. -In the old infrastructure, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. +In the old analyzer, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. **Example 2:** @@ -58,8 +61,8 @@ GROUP BY n ### CREATE VIEW with invalid query -The new infrastructure always performs type-checking. -Previously, it was possible to create a `VIEW` with an invalid `SELECT` query, and it'd fail during the first SELECT or insert (in the case of `MATERIALIZED VIEW`). +The new analyzer always performs type-checking. +Previously, it was possible to create a `VIEW` with an invalid `SELECT` query. It would then fail during the first `SELECT` or `INSERT` (in the case of `MATERIALIZED VIEW`). Now, it's not possible to create such `VIEW`s anymore. @@ -73,15 +76,15 @@ AS SELECT JSONExtract(data, 'test', 'DateTime64(3)') FROM source; ``` -### Known incompatibilities of JOIN clause +### Known incompatibilities of the `JOIN` clause #### Join using column from projection -Alias from the `SELECT` list can not be used as a `JOIN USING` key by default. +Alias from the `SELECT` list can not be used as a `JOIN USING` key by default. -A new setting, `analyzer_compatibility_join_using_top_level_identifier`, when enabled, alters the behavior of `JOIN USING` to prefer to resolve identifiers based on expressions from the projection list of the SELECT query, rather than using the columns from left table directly. +A new setting, `analyzer_compatibility_join_using_top_level_identifier`, when enabled, alters the behavior of `JOIN USING` to prefer to resolve identifiers based on expressions from the projection list of the `SELECT` query, rather than using the columns from left table directly. -*Example:* +**Example:** ```sql SELECT a + 1 AS b, t2.s @@ -90,15 +93,15 @@ JOIN Values('b UInt64, s String', (1, 'one'), (2, 'two')) t2 USING (b); ``` -With `analyzer_compatibility_join_using_top_level_identifier` set to `true`, the join condition is interpreted as `t1.a + 1 = t2.b`, matching the behavior of earlier versions. So, the result will be `2, 'two'` +With `analyzer_compatibility_join_using_top_level_identifier` set to `true`, the join condition is interpreted as `t1.a + 1 = t2.b`, matching the behavior of earlier versions. So, the result will be `2, 'two'`. When the setting is `false`, the join condition defaults to `t1.b = t2.b`, and the query will return `2, 'one'`. -In case then `b` is not present in `t1`, the query will fail with an error. +If `b` is not present in `t1`, the query will fail with an error. -#### Changes in Behavior with `JOIN USING` and `ALIAS/MATERIALIZED` Columns +#### Changes in behavior with `JOIN USING` and `ALIAS`/`MATERIALIZED` columns -In the new analyzer, using `*` in a `JOIN USING` query that involves `ALIAS` or `MATERIALIZED` columns will include that columns in the result set by default. +In the new analyzer, using `*` in a `JOIN USING` query that involves `ALIAS` or `MATERIALIZED` columns will include those columns in the result set by default. -*Example:* +**Example:** ```sql CREATE TABLE t1 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id; @@ -123,7 +126,7 @@ In the new version of the analyzer, the rules for determining the common superty - `Nullable(T)` and `T`: When a column of type `Nullable(T)` is joined with a column of type `T`, the resulting common supertype will be `Nullable(T)`, ensuring that the nullable property is preserved. -*Example:* +**Example:** ```sql SELECT id, toTypeName(id) FROM Values('id LowCardinality(String)', ('a')) AS t1 @@ -135,7 +138,7 @@ In this query, the common supertype for `id` is determined as `String`, discardi ### Projection column names changes -During projection names computation aliases are not substituted. +During projection names, computation aliases are not substituted. ```sql SELECT @@ -161,7 +164,7 @@ FORMAT PrettyCompact ### Incompatible function arguments types -In the new infrastructure type inference happens during initial query analysis. +In the new analyzer, type inference happens during initial query analysis. This change means that type checks are done before short-circuit evaluation; thus, `if` function arguments must always have a common supertype. **Example:** From 43e6482af0a6e14a2aee3abe007ff23a084c7acc Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 7 Jun 2024 11:36:24 +0200 Subject: [PATCH 396/856] Update docs/en/operations/analyzer.md --- docs/en/operations/analyzer.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index e85c5fa9e8d..298c6dacd06 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -138,7 +138,7 @@ In this query, the common supertype for `id` is determined as `String`, discardi ### Projection column names changes -During projection names, computation aliases are not substituted. +During projection names computation, aliases are not substituted. ```sql SELECT From 2131877819a514f037fe37a32eef0aa37e862380 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 7 Jun 2024 10:44:59 +0000 Subject: [PATCH 397/856] Fixing other tests. --- .../gtest_transform_query_for_external_database.cpp | 12 ++++++++---- .../0_stateless/02892_orc_filter_pushdown.reference | 2 +- .../03164_early_constant_folding_analyzer.sql | 2 +- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 7e2d393c3d1..6765e112bb9 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -368,17 +368,21 @@ TEST(TransformQueryForExternalDatabase, Null) check(state, 1, {"field"}, "SELECT field FROM table WHERE field IS NULL", - R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)"); + R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)", + R"(SELECT "field" FROM "test"."table" WHERE 1 = 0)"); check(state, 1, {"field"}, "SELECT field FROM table WHERE field IS NOT NULL", - R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)"); + R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)", + R"(SELECT "field" FROM "test"."table")"); check(state, 1, {"field"}, "SELECT field FROM table WHERE isNull(field)", - R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)"); + R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)", + R"(SELECT "field" FROM "test"."table" WHERE 1 = 0)"); check(state, 1, {"field"}, "SELECT field FROM table WHERE isNotNull(field)", - R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)"); + R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)", + R"(SELECT "field" FROM "test"."table")"); } TEST(TransformQueryForExternalDatabase, ToDate) diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference index 9059b403a34..f029b1d405f 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference @@ -210,7 +210,7 @@ select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null < -500); 596 -1099 -501 select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null is null); -1000 499500 +0 0 select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null is null); 0 0 0 select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null in (0, -1, -10, -100, -1000)); diff --git a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql index 37675c96dd1..dbffbc1af71 100644 --- a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql +++ b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql @@ -27,4 +27,4 @@ ENGINE = MergeTree ORDER BY (date, pull_request_number, commit_sha, check_name, insert into checks select * from generateRandom() limit 1; -select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL SETTINGS allow_experimental_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 0) where explain like '%ReadFromPreparedSource%'; \ No newline at end of file +select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL) where explain like '%ReadFromPreparedSource%' SETTINGS allow_experimental_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 0; From 44127fdd35aa1d4aff8d19b3804f2f970c345572 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 Jun 2024 13:08:01 +0200 Subject: [PATCH 398/856] Add documentation --- docs/en/operations/named-collections.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index c9d94dd95ee..fa8a936c59f 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -67,6 +67,23 @@ To manage named collections with DDL a user must have the `named_control_collect In the above example the `password_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plain text and sha256 hex passwords set for a user. ::: +### Storage for named collections + +Named collections can either be stored on local disk or in zookeeper/keeper. By default local storage is used. + +To configure named collections storage in keeper and a `type` (equal to either `keeper` or `zookeeper`) and `path` (path in keeper, where named collections will be stored) to `named_collections_storage` section in configuration file: +``` + + + zookeeper + /named_collections_path/ + 1000 + + +``` + +An optional configuration parameter `update_timeout_ms` by default is equal to `5000`. + ## Storing named collections in configuration files ### XML example From 078f5f4ee0fff1d55a5924df8340a905ef32f0f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 13:10:12 +0200 Subject: [PATCH 399/856] Fix bug in short circuit evaluation --- src/Columns/MaskOperations.cpp | 6 +++++- .../0_stateless/03168_fuzz_multiIf_short_circuit.reference | 0 .../0_stateless/03168_fuzz_multiIf_short_circuit.sql | 6 ++++++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference create mode 100644 tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index 1f5f94beee9..873a4060872 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -289,10 +289,14 @@ void executeColumnIfNeeded(ColumnWithTypeAndName & column, bool empty) if (!column_function) return; + size_t original_size = column.column->size(); + if (!empty) column = column_function->reduce(); else - column.column = column_function->getResultType()->createColumn(); + column.column = column_function->getResultType()->createColumnConstWithDefaultValue(original_size)->convertToFullColumnIfConst(); + + chassert(column.column->size() == original_size); } int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments) diff --git a/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql new file mode 100644 index 00000000000..4e4cc291e9b --- /dev/null +++ b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql @@ -0,0 +1,6 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/64946 +SELECT + multiIf((number % toLowCardinality(toNullable(toUInt128(2)))) = (number % toNullable(2)), toInt8(1), (number % materialize(toLowCardinality(3))) = toUInt128(toNullable(0)), toInt8(materialize(materialize(2))), toInt64(toUInt128(3))) +FROM system.numbers +LIMIT 44857 +FORMAT Null; From 4b010dc478310b65d26cbe114e15f3cb73af4bb4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 7 Jun 2024 13:11:52 +0200 Subject: [PATCH 400/856] Disable test with ASAN --- .../0_stateless/02908_many_requests_to_system_replicas.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh index 144831a2cdc..a247c99a818 100755 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-parallel, no-fasttest, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 9920c3d17f9f64e24b46addd746c4bdcc49e6972 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Jun 2024 21:19:28 +0000 Subject: [PATCH 401/856] 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 4d88f103469e8176229f3a258847eb9cca374309 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 7 Jun 2024 11:28:36 +0000 Subject: [PATCH 402/856] Update version_date.tsv and changelogs after v24.4.2.141-stable --- docs/changelogs/v24.4.2.141-stable.md | 101 ++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 102 insertions(+) create mode 100644 docs/changelogs/v24.4.2.141-stable.md diff --git a/docs/changelogs/v24.4.2.141-stable.md b/docs/changelogs/v24.4.2.141-stable.md new file mode 100644 index 00000000000..656d0854392 --- /dev/null +++ b/docs/changelogs/v24.4.2.141-stable.md @@ -0,0 +1,101 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.4.2.141-stable (9e23d27bd11) FIXME as compared to v24.4.1.2088-stable (6d4b31322d1) + +#### Improvement +* Backported in [#63467](https://github.com/ClickHouse/ClickHouse/issues/63467): Make rabbitmq nack broken messages. Closes [#45350](https://github.com/ClickHouse/ClickHouse/issues/45350). [#60312](https://github.com/ClickHouse/ClickHouse/pull/60312) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Build/Testing/Packaging Improvement +* Backported in [#63612](https://github.com/ClickHouse/ClickHouse/issues/63612): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#64279](https://github.com/ClickHouse/ClickHouse/issues/64279): Fix queries with FINAL give wrong result when table does not use adaptive granularity. [#62432](https://github.com/ClickHouse/ClickHouse/pull/62432) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#63295](https://github.com/ClickHouse/ClickHouse/issues/63295): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63978](https://github.com/ClickHouse/ClickHouse/issues/63978): Fix intersect parts when restart after drop range. [#63202](https://github.com/ClickHouse/ClickHouse/pull/63202) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#63413](https://github.com/ClickHouse/ClickHouse/issues/63413): Fix a misbehavior when SQL security defaults don't load for old tables during server startup. [#63209](https://github.com/ClickHouse/ClickHouse/pull/63209) ([pufit](https://github.com/pufit)). +* Backported in [#63388](https://github.com/ClickHouse/ClickHouse/issues/63388): JOIN filter push down filled join fix. Closes [#63228](https://github.com/ClickHouse/ClickHouse/issues/63228). [#63234](https://github.com/ClickHouse/ClickHouse/pull/63234) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63618](https://github.com/ClickHouse/ClickHouse/issues/63618): Fix bug which could potentially lead to rare LOGICAL_ERROR during SELECT query with message: `Unexpected return type from materialize. Expected type_XXX. Got type_YYY.` Introduced in [#59379](https://github.com/ClickHouse/ClickHouse/issues/59379). [#63353](https://github.com/ClickHouse/ClickHouse/pull/63353) ([alesapin](https://github.com/alesapin)). +* Backported in [#63451](https://github.com/ClickHouse/ClickHouse/issues/63451): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)). +* Backported in [#63605](https://github.com/ClickHouse/ClickHouse/issues/63605): Fix backup of projection part in case projection was removed from table metadata, but part still has projection. [#63426](https://github.com/ClickHouse/ClickHouse/pull/63426) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#63510](https://github.com/ClickHouse/ClickHouse/issues/63510): Fix 'Every derived table must have its own alias' error for MYSQL dictionary source, close [#63341](https://github.com/ClickHouse/ClickHouse/issues/63341). [#63481](https://github.com/ClickHouse/ClickHouse/pull/63481) ([vdimir](https://github.com/vdimir)). +* Backported in [#63592](https://github.com/ClickHouse/ClickHouse/issues/63592): Avoid segafult in `MergeTreePrefetchedReadPool` while fetching projection parts. [#63513](https://github.com/ClickHouse/ClickHouse/pull/63513) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63750](https://github.com/ClickHouse/ClickHouse/issues/63750): Read only the necessary columns from VIEW (new analyzer). Closes [#62594](https://github.com/ClickHouse/ClickHouse/issues/62594). [#63688](https://github.com/ClickHouse/ClickHouse/pull/63688) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63772](https://github.com/ClickHouse/ClickHouse/issues/63772): Fix [#63539](https://github.com/ClickHouse/ClickHouse/issues/63539). Forbid WINDOW redefinition in new analyzer. [#63694](https://github.com/ClickHouse/ClickHouse/pull/63694) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#63872](https://github.com/ClickHouse/ClickHouse/issues/63872): Flatten_nested is broken with replicated database. [#63695](https://github.com/ClickHouse/ClickHouse/pull/63695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63854](https://github.com/ClickHouse/ClickHouse/issues/63854): Fix `Not found column` and `CAST AS Map from array requires nested tuple of 2 elements` exceptions for distributed queries which use `Map(Nothing, Nothing)` type. Fixes [#63637](https://github.com/ClickHouse/ClickHouse/issues/63637). [#63753](https://github.com/ClickHouse/ClickHouse/pull/63753) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63847](https://github.com/ClickHouse/ClickHouse/issues/63847): Fix possible `ILLEGAL_COLUMN` error in `partial_merge` join, close [#37928](https://github.com/ClickHouse/ClickHouse/issues/37928). [#63755](https://github.com/ClickHouse/ClickHouse/pull/63755) ([vdimir](https://github.com/vdimir)). +* Backported in [#63908](https://github.com/ClickHouse/ClickHouse/issues/63908): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#63955](https://github.com/ClickHouse/ClickHouse/issues/63955): Fix possible crash with SYSTEM UNLOAD PRIMARY KEY. [#63778](https://github.com/ClickHouse/ClickHouse/pull/63778) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63938](https://github.com/ClickHouse/ClickHouse/issues/63938): Allow JOIN filter push down to both streams if only single equivalent column is used in query. Closes [#63799](https://github.com/ClickHouse/ClickHouse/issues/63799). [#63819](https://github.com/ClickHouse/ClickHouse/pull/63819) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63991](https://github.com/ClickHouse/ClickHouse/issues/63991): Fix incorrect select query result when parallel replicas were used to read from a Materialized View. [#63861](https://github.com/ClickHouse/ClickHouse/pull/63861) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#64033](https://github.com/ClickHouse/ClickHouse/issues/64033): Fix a error `Database name is empty` for remote queries with lambdas over the cluster with modified default database. Fixes [#63471](https://github.com/ClickHouse/ClickHouse/issues/63471). [#63864](https://github.com/ClickHouse/ClickHouse/pull/63864) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64561](https://github.com/ClickHouse/ClickHouse/issues/64561): Fix SIGSEGV due to CPU/Real (`query_profiler_real_time_period_ns`/`query_profiler_cpu_time_period_ns`) profiler (has been an issue since 2022, that leads to periodic server crashes, especially if you were using distributed engine). [#63865](https://github.com/ClickHouse/ClickHouse/pull/63865) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#64011](https://github.com/ClickHouse/ClickHouse/issues/64011): Fix analyzer - IN function with arbitrary deep sub-selects in materialized view to use insertion block. [#63930](https://github.com/ClickHouse/ClickHouse/pull/63930) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#64238](https://github.com/ClickHouse/ClickHouse/issues/64238): Fix resolve of unqualified COLUMNS matcher. Preserve the input columns order and forbid usage of unknown identifiers. [#63962](https://github.com/ClickHouse/ClickHouse/pull/63962) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64103](https://github.com/ClickHouse/ClickHouse/issues/64103): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64170](https://github.com/ClickHouse/ClickHouse/issues/64170): Add missing settings to recoverLostReplica. [#64040](https://github.com/ClickHouse/ClickHouse/pull/64040) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64322](https://github.com/ClickHouse/ClickHouse/issues/64322): 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)). +* Backported in [#64382](https://github.com/ClickHouse/ClickHouse/issues/64382): 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)). +* Backported in [#64568](https://github.com/ClickHouse/ClickHouse/issues/64568): 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)). +* Backported in [#64272](https://github.com/ClickHouse/ClickHouse/issues/64272): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64330](https://github.com/ClickHouse/ClickHouse/issues/64330): 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)). +* Backported in [#64254](https://github.com/ClickHouse/ClickHouse/issues/64254): Ignore `text_log` config when using Keeper. [#64218](https://github.com/ClickHouse/ClickHouse/pull/64218) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#64690](https://github.com/ClickHouse/ClickHouse/issues/64690): 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)). +* Backported in [#64409](https://github.com/ClickHouse/ClickHouse/issues/64409): 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)). +* Backported in [#64727](https://github.com/ClickHouse/ClickHouse/issues/64727): Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#64623](https://github.com/ClickHouse/ClickHouse/issues/64623): 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)). +* Backported in [#64680](https://github.com/ClickHouse/ClickHouse/issues/64680): 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)). +* Backported in [#64942](https://github.com/ClickHouse/ClickHouse/issues/64942): Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64871](https://github.com/ClickHouse/ClickHouse/issues/64871): 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)). + +#### CI Fix or Improvement (changelog entry is not required) + +* Backported in [#63364](https://github.com/ClickHouse/ClickHouse/issues/63364): Implement cumulative A Sync status. [#61464](https://github.com/ClickHouse/ClickHouse/pull/61464) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63338](https://github.com/ClickHouse/ClickHouse/issues/63338): Use `/commit/` to have the URLs in [reports](https://play.clickhouse.com/play?user=play#c2VsZWN0IGRpc3RpbmN0IGNvbW1pdF91cmwgZnJvbSBjaGVja3Mgd2hlcmUgY2hlY2tfc3RhcnRfdGltZSA+PSBub3coKSAtIGludGVydmFsIDEgbW9udGggYW5kIHB1bGxfcmVxdWVzdF9udW1iZXI9NjA1MzI=) like https://github.com/ClickHouse/ClickHouse/commit/44f8bc5308b53797bec8cccc3bd29fab8a00235d and not like https://github.com/ClickHouse/ClickHouse/commits/44f8bc5308b53797bec8cccc3bd29fab8a00235d. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63376](https://github.com/ClickHouse/ClickHouse/issues/63376):. [#63366](https://github.com/ClickHouse/ClickHouse/pull/63366) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#63571](https://github.com/ClickHouse/ClickHouse/issues/63571):. [#63551](https://github.com/ClickHouse/ClickHouse/pull/63551) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Backported in [#63651](https://github.com/ClickHouse/ClickHouse/issues/63651): Fix 02362_part_log_merge_algorithm flaky test. [#63635](https://github.com/ClickHouse/ClickHouse/pull/63635) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Backported in [#63828](https://github.com/ClickHouse/ClickHouse/issues/63828): Fix test_odbc_interaction from aarch64 [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63787](https://github.com/ClickHouse/ClickHouse/pull/63787) ([alesapin](https://github.com/alesapin)). +* Backported in [#63897](https://github.com/ClickHouse/ClickHouse/issues/63897): Fix test `test_catboost_evaluate` for aarch64. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63789](https://github.com/ClickHouse/ClickHouse/pull/63789) ([alesapin](https://github.com/alesapin)). +* Backported in [#63889](https://github.com/ClickHouse/ClickHouse/issues/63889): Remove HDFS from disks config for one integration test for arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63832](https://github.com/ClickHouse/ClickHouse/pull/63832) ([alesapin](https://github.com/alesapin)). +* Backported in [#63881](https://github.com/ClickHouse/ClickHouse/issues/63881): Bump version for old image in test_short_strings_aggregation to make it work on arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63836](https://github.com/ClickHouse/ClickHouse/pull/63836) ([alesapin](https://github.com/alesapin)). +* Backported in [#63919](https://github.com/ClickHouse/ClickHouse/issues/63919): Disable test `test_non_default_compression/test.py::test_preconfigured_deflateqpl_codec` on arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63839](https://github.com/ClickHouse/ClickHouse/pull/63839) ([alesapin](https://github.com/alesapin)). +* Backported in [#63971](https://github.com/ClickHouse/ClickHouse/issues/63971): Fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#64049](https://github.com/ClickHouse/ClickHouse/issues/64049): Add `ClickHouseVersion.copy` method. Create a branch release in advance without spinning out the release to increase the stability. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64078](https://github.com/ClickHouse/ClickHouse/issues/64078): The mime type is not 100% reliable for Python and shell scripts without shebangs; add a check for file extension. [#64062](https://github.com/ClickHouse/ClickHouse/pull/64062) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64161](https://github.com/ClickHouse/ClickHouse/issues/64161): Add retries in git submodule update. [#64125](https://github.com/ClickHouse/ClickHouse/pull/64125) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#64589](https://github.com/ClickHouse/ClickHouse/issues/64589): 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)). +* Backported in [#64880](https://github.com/ClickHouse/ClickHouse/issues/64880): 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)). + +#### NO CL CATEGORY + +* Backported in [#63306](https://github.com/ClickHouse/ClickHouse/issues/63306):. [#63297](https://github.com/ClickHouse/ClickHouse/pull/63297) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#63710](https://github.com/ClickHouse/ClickHouse/issues/63710):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#64363](https://github.com/ClickHouse/ClickHouse/issues/64363) to 24.4: Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts"'. [#64905](https://github.com/ClickHouse/ClickHouse/pull/64905) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* group_by_use_nulls strikes back [#62922](https://github.com/ClickHouse/ClickHouse/pull/62922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add `FROM` keyword to `TRUNCATE ALL TABLES` [#63241](https://github.com/ClickHouse/ClickHouse/pull/63241) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* More checks for concurrently deleted files and dirs in system.remote_data_paths [#63274](https://github.com/ClickHouse/ClickHouse/pull/63274) ([Alexander Gololobov](https://github.com/davenger)). +* Try fix segfault in `MergeTreeReadPoolBase::createTask` [#63323](https://github.com/ClickHouse/ClickHouse/pull/63323) ([Antonio Andelic](https://github.com/antonio2368)). +* Skip unaccessible table dirs in system.remote_data_paths [#63330](https://github.com/ClickHouse/ClickHouse/pull/63330) ([Alexander Gololobov](https://github.com/davenger)). +* Workaround for `oklch()` inside canvas bug for firefox [#63404](https://github.com/ClickHouse/ClickHouse/pull/63404) ([Sergei Trifonov](https://github.com/serxa)). +* Cancel S3 reads properly when parallel reads are used [#63687](https://github.com/ClickHouse/ClickHouse/pull/63687) ([Antonio Andelic](https://github.com/antonio2368)). +* Userspace page cache: don't collect stats if cache is unused [#63730](https://github.com/ClickHouse/ClickHouse/pull/63730) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix sanitizers [#64090](https://github.com/ClickHouse/ClickHouse/pull/64090) ([Azat Khuzhin](https://github.com/azat)). +* Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts [#64363](https://github.com/ClickHouse/ClickHouse/pull/64363) ([Kruglov Pavel](https://github.com/Avogar)). +* CI: Critical bugfix category in PR template [#64480](https://github.com/ClickHouse/ClickHouse/pull/64480) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f7d84cce4b1..2f96daf4887 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,4 +1,5 @@ v24.5.1.1763-stable 2024-06-01 +v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 v24.3.3.102-lts 2024-05-01 v24.3.2.23-lts 2024-04-03 From 2d9ac2e8169957ba73f477befd9aef8e753e86f2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 7 Jun 2024 14:03:56 +0200 Subject: [PATCH 403/856] Revert "Add dynamic untracked memory limits for more precise memory tracking" --- docs/en/operations/settings/settings.md | 12 +----------- src/Common/CurrentMemoryTracker.cpp | 12 +----------- src/Common/CurrentMemoryTracker.h | 2 -- src/Common/ThreadStatus.h | 12 ------------ src/Core/Settings.h | 1 - src/Core/SettingsChangesHistory.h | 1 - src/Interpreters/ThreadStatusExt.cpp | 10 +++------- tests/integration/test_failed_async_inserts/test.py | 4 +++- .../test_settings_constraints_distributed/test.py | 5 +---- .../0_stateless/01017_uniqCombined_memory_usage.sql | 3 +-- .../03030_system_flush_distributed_settings.sql | 2 -- 11 files changed, 10 insertions(+), 54 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b3e9da816ab..ffaf53085c4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3172,7 +3172,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: @@ -4616,16 +4616,6 @@ Read more about [memory overcommit](memory-overcommit.md). Default value: `1GiB`. -## max_untracked_memory {#max_untracked_memory} -Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'. - -Default value: `4MiB`. - -## min_untracked_memory {#min_untracked_memory} -Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread memory usage divided by 16 and clamped between `min_untracked_memory` and `max_untracked_memory` for every thread. It guarantees that total untracked memory does not exceed 10% of current memory footprint even with a lot of small threads. To disable dynamic limit for untracked memory set value `4MiB`. - -Default value: `4KiB`. - ## Schema Inference settings See [schema inference](../../interfaces/schema-inference.md#schema-inference-modes) documentation for more details. diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index b1dcded0b23..02c7dc6e224 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -57,7 +57,6 @@ AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory { auto res = memory_tracker->allocImpl(will_be, throw_if_memory_exceeded); current_thread->untracked_memory = 0; - current_thread->updateUntrackedMemoryLimit(memory_tracker->get()); return res; } else @@ -85,13 +84,6 @@ void CurrentMemoryTracker::check() std::ignore = memory_tracker->allocImpl(0, true); } -Int64 CurrentMemoryTracker::get() -{ - if (auto * memory_tracker = getMemoryTracker()) - return memory_tracker->get(); - return 0; -} - AllocationTrace CurrentMemoryTracker::alloc(Int64 size) { bool throw_if_memory_exceeded = true; @@ -111,12 +103,10 @@ AllocationTrace CurrentMemoryTracker::free(Int64 size) if (current_thread) { current_thread->untracked_memory -= size; - // Note that we use `max_untracked_memory` and not `untracked_memory_limit` to create hysteresis to avoid track/untrack cycles - if (current_thread->untracked_memory < -current_thread->max_untracked_memory) + if (current_thread->untracked_memory < -current_thread->untracked_memory_limit) { Int64 untracked_memory = current_thread->untracked_memory; current_thread->untracked_memory = 0; - current_thread->updateUntrackedMemoryLimit(memory_tracker->get() + untracked_memory); return memory_tracker->free(-untracked_memory); } } diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index 401eeed93dd..18a1e3f49b1 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -12,9 +12,7 @@ struct CurrentMemoryTracker /// This function should be called after memory deallocation. [[nodiscard]] static AllocationTrace free(Int64 size); - static void check(); - [[nodiscard]] static Int64 get(); /// Throws MEMORY_LIMIT_EXCEEDED (if it's allowed to throw exceptions) static void injectFault(); diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index db4854da707..0c02ab8fdb0 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -183,12 +183,6 @@ public: Int64 untracked_memory = 0; /// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters. Int64 untracked_memory_limit = 4 * 1024 * 1024; - /// To keep total untracked memory limited to `untracked_memory_ratio * RSS` we have to account threads with small and large memory footprint differently. - /// For this purpose we dynamically change `untracked_memory_limit` after every tracking event using a simple formula: - /// untracked_memory_limit = clamp(untracked_memory_ratio * cur_memory_bytes, min_untracked_memory, max_untracked_memory) - /// Note that this values are updated when thread is attached to a group - Int64 min_untracked_memory = 4 * 1024 * 1024; // Default value is kept 4MB mostly for tests and client (should be changed to 4KB as default value a setting) - Int64 max_untracked_memory = 4 * 1024 * 1024; /// Statistics of read and write rows/bytes Progress progress_in; @@ -315,12 +309,6 @@ public: void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period); - void updateUntrackedMemoryLimit(Int64 current) - { - constexpr Int64 untracked_memory_ratio_bits = 4; // untracked_memory_ratio = 1.0 / (1 << untracked_memory_ratio_bits) = 1.0 / 16 = 6.25% - untracked_memory_limit = std::clamp(current >> untracked_memory_ratio_bits, min_untracked_memory, max_untracked_memory); - } - private: void applyGlobalSettings(); void applyQuerySettings(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d63d5e9d181..27ce54c03a7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -491,7 +491,6 @@ class IColumn; M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ - M(UInt64, min_untracked_memory, (4 * 1024), "Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread_memory_usage/16 and clamped between min_untracked_memory and max_untracked_memory for every thread.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 630ffa54b49..814c08c5705 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -96,7 +96,6 @@ static const std::map #include #include -#include #include #include #include @@ -211,12 +210,9 @@ void ThreadStatus::applyQuerySettings() query_id_from_query_context = query_context_ptr->getCurrentQueryId(); initQueryProfiler(); - max_untracked_memory = settings.max_untracked_memory; - if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast(max_untracked_memory)) - max_untracked_memory = settings.memory_profiler_step; - min_untracked_memory = std::min(settings.min_untracked_memory, max_untracked_memory); - - updateUntrackedMemoryLimit(CurrentMemoryTracker::get()); + untracked_memory_limit = settings.max_untracked_memory; + if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast(untracked_memory_limit)) + untracked_memory_limit = settings.memory_profiler_step; #if defined(OS_LINUX) /// Set "nice" value if required. diff --git a/tests/integration/test_failed_async_inserts/test.py b/tests/integration/test_failed_async_inserts/test.py index e7e504e565f..ecb506c36bc 100644 --- a/tests/integration/test_failed_async_inserts/test.py +++ b/tests/integration/test_failed_async_inserts/test.py @@ -45,7 +45,9 @@ def test_failed_async_inserts(started_cluster): ignore_error=True, ) - select_query = "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = '4Mi'" + select_query = ( + "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery'" + ) assert node.query(select_query) == "4\n" diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index d29b66b43bb..fbebbac276e 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -136,10 +136,7 @@ def test_select_clamps_settings(): ) assert ( - distributed.query( - query, - settings={"max_memory_usage": 1, "min_untracked_memory": 4 * 1024 * 1024}, - ) + distributed.query(query, settings={"max_memory_usage": 1}) == "node1\tmax_memory_usage\t11111111\n" "node1\treadonly\t0\n" "node2\tmax_memory_usage\t0\n" diff --git a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql index de84846c1d7..c13a0859183 100644 --- a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql +++ b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql @@ -7,8 +7,7 @@ -- sizeof(HLL) is (2^K * 6 / 8) -- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400 -SET use_uncompressed_cache = 0; -SET min_untracked_memory = '4Mi'; +SET use_uncompressed_cache = 0; -- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements SELECT 'UInt32'; diff --git a/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql b/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql index 7961444dbc2..da2a387e07c 100644 --- a/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql +++ b/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql @@ -13,8 +13,6 @@ create table dist_out as data engine=Distributed(test_shard_localhost, currentDa set prefer_localhost_replica=0; -set min_untracked_memory='4Mi'; -- Disable precise memory tracking - insert into dist_in select number/100, number from system.numbers limit 1e6 settings max_memory_usage='20Mi'; system flush distributed dist_in; -- { serverError MEMORY_LIMIT_EXCEEDED } system flush distributed dist_in settings max_memory_usage=0; From 69045adc10df36e0e1bd9097b2865e278ed16aea Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 7 Jun 2024 14:09:54 +0200 Subject: [PATCH 404/856] fix bad test --- .../test_attach_partition_using_copy/test.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_attach_partition_using_copy/test.py b/tests/integration/test_attach_partition_using_copy/test.py index e7163b1eb32..d5b07603dff 100644 --- a/tests/integration/test_attach_partition_using_copy/test.py +++ b/tests/integration/test_attach_partition_using_copy/test.py @@ -98,7 +98,8 @@ def create_destination_table(node, table_name, replicated): ) -def test_both_mergtree(start_cluster): +def test_both_mergetree(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", False) create_destination_table(replica1, "destination", False) @@ -120,12 +121,13 @@ def test_both_mergtree(start_cluster): def test_all_replicated(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", True) create_destination_table(replica1, "destination", True) create_destination_table(replica2, "destination", True) - replica1.query("SYSTEM SYNC REPLICA destination") replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + replica2.query("SYSTEM SYNC REPLICA destination") assert_eq_with_retry( replica1, @@ -154,12 +156,13 @@ def test_all_replicated(start_cluster): def test_only_destination_replicated(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", False) create_destination_table(replica1, "destination", True) create_destination_table(replica2, "destination", True) - replica1.query("SYSTEM SYNC REPLICA destination") replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + replica2.query("SYSTEM SYNC REPLICA destination") assert_eq_with_retry( replica1, @@ -188,6 +191,7 @@ def test_only_destination_replicated(start_cluster): def test_not_work_on_different_disk(start_cluster): + cleanup([replica1, replica2]) # Replace and move should not work on replace create_source_table(replica1, "source", False) create_destination_table(replica2, "destination", False) From f77b6096c405aaf8862582bf87a897e925de7a6a Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 7 Jun 2024 12:14:21 +0000 Subject: [PATCH 405/856] time_virtual_col: st_mtime, not st_mtim, because of darwin --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 09f0bd60859..aaf84f6f82c 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1371,7 +1371,7 @@ Chunk StorageFileSource::generate() struct stat file_stat; file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); current_file_size = file_stat.st_size; - current_file_last_modified = Poco::Timestamp::fromEpochTime(file_stat.st_mtim.tv_sec); + current_file_last_modified = Poco::Timestamp::fromEpochTime(file_stat.st_mtime); if (getContext()->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) continue; From 441279a979d3fd4f4e6257c21a82f4d162de3505 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 7 Jun 2024 14:47:22 +0200 Subject: [PATCH 406/856] Disable GWP for sanitizer builds --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 455adc24182..c4f093b1c99 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -399,7 +399,7 @@ option (ENABLE_GWP_ASAN "Enable Gwp-Asan" ON) # but GWP-ASan also wants to use mmap frequently, # and due to a large number of memory mappings, # it does not work together well. -if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) +if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") OR SANITIZE) set(ENABLE_GWP_ASAN OFF) endif () From b04fb116a5cd8c6064804a1949b97d31ccc95a56 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 11:26:12 +0200 Subject: [PATCH 407/856] Compiler happy. Developer unhappy --- src/Storages/StorageGenerateRandom.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index ca9c6fb3226..8852e468c5e 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -706,7 +706,7 @@ Pipe StorageGenerateRandom::read( if (query_limit && num_streams * max_block_size > query_limit) { /// We want to avoid spawning more streams than necessary - num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + num_streams = std::min(num_streams, static_cast(((query_limit + max_block_size - 1) / max_block_size))); } Pipes pipes; pipes.reserve(num_streams); From 2b200b566107f8023a4b6cc323ee1a3e4d1f06d3 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 7 Jun 2024 10:03:07 -0300 Subject: [PATCH 408/856] modify match everything regex --- src/Common/proxyConfigurationToPocoProxyConfig.cpp | 2 +- src/Common/tests/gtest_poco_no_proxy_regex.cpp | 2 +- .../tests/gtest_proxy_configuration_resolver_provider.cpp | 7 ++++++- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.cpp b/src/Common/proxyConfigurationToPocoProxyConfig.cpp index c1a439d2f36..f64dbc3bc02 100644 --- a/src/Common/proxyConfigurationToPocoProxyConfig.cpp +++ b/src/Common/proxyConfigurationToPocoProxyConfig.cpp @@ -54,7 +54,7 @@ std::string buildPocoRegexpEntryWithoutLeadingDot(const std::string & host) std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts_string) { static constexpr auto OR_SEPARATOR = "|"; - static constexpr auto MATCH_ANYTHING = R"((.*?))"; + static constexpr auto MATCH_ANYTHING = R"(.*)"; static constexpr auto MATCH_SUBDOMAINS_REGEX = R"((?:.*\.)?)"; bool match_any_host = no_proxy_hosts_string.size() == 1 && no_proxy_hosts_string[0] == '*'; diff --git a/src/Common/tests/gtest_poco_no_proxy_regex.cpp b/src/Common/tests/gtest_poco_no_proxy_regex.cpp index 1f70a483ab4..c3c1b512c08 100644 --- a/src/Common/tests/gtest_poco_no_proxy_regex.cpp +++ b/src/Common/tests/gtest_poco_no_proxy_regex.cpp @@ -13,7 +13,7 @@ TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildMatchA { ASSERT_EQ( DB::buildPocoNonProxyHosts("*"), - "(.*?)"); + ".*"); } TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildEmpty) diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index 368ce12cd7b..8bb2100561f 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -34,7 +34,12 @@ Poco::URI https_env_proxy_server = Poco::URI("http://https_environment_proxy:312 Poco::URI http_list_proxy_server = Poco::URI("http://http_list_proxy:3128"); Poco::URI https_list_proxy_server = Poco::URI("http://https_list_proxy:3128"); -static std::string no_proxy_hosts = "localhost,,127.0.0.1,some_other_domain,,,, sub-domain.domain.com,"; +// Some other tests rely on HTTP clients (e.g, gtest_aws_s3_client), which depend on proxy configuration +// since in https://github.com/ClickHouse/ClickHouse/pull/63314 the environment proxy resolver reads only once +// from the environment, the proxy configuration will always be there. +// The problem is that the proxy server does not exist, causing the test to fail. +// To work around this issue, `no_proxy` is set to bypass all domains. +static std::string no_proxy_hosts = "*"; TEST_F(ProxyConfigurationResolverProviderTests, EnvironmentResolverShouldBeUsedIfNoSettings) { From c02cb392bfebeff6cae375eede4990c5fff20fa4 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 7 Jun 2024 13:32:08 +0000 Subject: [PATCH 409/856] 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 48de600770aa5ad08720c365d8a705b48375a647 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 7 Jun 2024 10:47:54 -0300 Subject: [PATCH 410/856] unify environment proxy set] --- src/Common/tests/gtest_helper_functions.h | 29 +++++++++---------- ..._proxy_configuration_resolver_provider.cpp | 16 +--------- .../gtest_proxy_environment_configuration.cpp | 20 ++++--------- 3 files changed, 20 insertions(+), 45 deletions(-) diff --git a/src/Common/tests/gtest_helper_functions.h b/src/Common/tests/gtest_helper_functions.h index 1e5c2b21d99..90c5d4d2088 100644 --- a/src/Common/tests/gtest_helper_functions.h +++ b/src/Common/tests/gtest_helper_functions.h @@ -76,25 +76,22 @@ inline std::string xmlNodeAsString(Poco::XML::Node *pNode) struct EnvironmentProxySetter { - EnvironmentProxySetter( - const Poco::URI & http_proxy, - const Poco::URI & https_proxy, - const std::string & no_proxy = {}) + static constexpr auto * NO_PROXY = "*"; + static constexpr auto * HTTP_PROXY = "http://proxy_server:3128"; + static constexpr auto * HTTPS_PROXY = "https://proxy_server:3128"; + + EnvironmentProxySetter() { - if (!http_proxy.empty()) - { - setenv("http_proxy", http_proxy.toString().c_str(), 1); // NOLINT(concurrency-mt-unsafe) - } + setenv("http_proxy", HTTP_PROXY, 1); // NOLINT(concurrency-mt-unsafe) - if (!https_proxy.empty()) - { - setenv("https_proxy", https_proxy.toString().c_str(), 1); // NOLINT(concurrency-mt-unsafe) - } + setenv("https_proxy", HTTPS_PROXY, 1); // NOLINT(concurrency-mt-unsafe) - if (!no_proxy.empty()) - { - setenv("no_proxy", no_proxy.c_str(), 1); // NOLINT(concurrency-mt-unsafe) - } + // Some other tests rely on HTTP clients (e.g, gtest_aws_s3_client), which depend on proxy configuration + // since in https://github.com/ClickHouse/ClickHouse/pull/63314 the environment proxy resolver reads only once + // from the environment, the proxy configuration will always be there. + // The problem is that the proxy server does not exist, causing the test to fail. + // To work around this issue, `no_proxy` is set to bypass all domains. + setenv("no_proxy", NO_PROXY, 1); // NOLINT(concurrency-mt-unsafe) } ~EnvironmentProxySetter() diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index 8bb2100561f..7bc48203998 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -28,22 +28,12 @@ protected: DB::ContextMutablePtr ProxyConfigurationResolverProviderTests::context; -Poco::URI http_env_proxy_server = Poco::URI("http://http_environment_proxy:3128"); -Poco::URI https_env_proxy_server = Poco::URI("http://https_environment_proxy:3128"); - Poco::URI http_list_proxy_server = Poco::URI("http://http_list_proxy:3128"); Poco::URI https_list_proxy_server = Poco::URI("http://https_list_proxy:3128"); -// Some other tests rely on HTTP clients (e.g, gtest_aws_s3_client), which depend on proxy configuration -// since in https://github.com/ClickHouse/ClickHouse/pull/63314 the environment proxy resolver reads only once -// from the environment, the proxy configuration will always be there. -// The problem is that the proxy server does not exist, causing the test to fail. -// To work around this issue, `no_proxy` is set to bypass all domains. -static std::string no_proxy_hosts = "*"; - TEST_F(ProxyConfigurationResolverProviderTests, EnvironmentResolverShouldBeUsedIfNoSettings) { - EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server, no_proxy_hosts); + EnvironmentProxySetter setter; const auto & config = getContext().context->getConfigRef(); auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, config); @@ -58,7 +48,6 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPOnly) ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); config->setString("proxy", ""); - config->setString("proxy.no_proxy", no_proxy_hosts); config->setString("proxy.http", ""); config->setString("proxy.http.uri", http_list_proxy_server.toString()); context->setConfig(config); @@ -74,7 +63,6 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPSOnly) { ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); - config->setString("proxy.no_proxy", no_proxy_hosts); config->setString("proxy", ""); config->setString("proxy.https", ""); config->setString("proxy.https.uri", https_list_proxy_server.toString()); @@ -91,7 +79,6 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListBoth) { ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); - config->setString("proxy.no_proxy", no_proxy_hosts); config->setString("proxy", ""); config->setString("proxy.http", ""); config->setString("proxy.http.uri", http_list_proxy_server.toString()); @@ -137,7 +124,6 @@ TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverHTTPSOnly) ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); config->setString("proxy", ""); - config->setString("proxy.no_proxy", no_proxy_hosts); config->setString("proxy.https", ""); config->setString("proxy.https.resolver", ""); config->setString("proxy.https.resolver.endpoint", "http://resolver:8080/hostname"); diff --git a/src/Common/tests/gtest_proxy_environment_configuration.cpp b/src/Common/tests/gtest_proxy_environment_configuration.cpp index ac31e33e129..708c7194785 100644 --- a/src/Common/tests/gtest_proxy_environment_configuration.cpp +++ b/src/Common/tests/gtest_proxy_environment_configuration.cpp @@ -8,22 +8,14 @@ namespace DB { -namespace -{ - auto http_proxy_server = Poco::URI("http://proxy_server:3128"); - auto https_proxy_server = Poco::URI("https://proxy_server:3128"); -} - TEST(EnvironmentProxyConfigurationResolver, TestHTTPandHTTPS) { - // Some other tests rely on HTTP clients (e.g, gtest_aws_s3_client), which depend on proxy configuration - // since in https://github.com/ClickHouse/ClickHouse/pull/63314 the environment proxy resolver reads only once - // from the environment, the proxy configuration will always be there. - // The problem is that the proxy server does not exist, causing the test to fail. - // To work around this issue, `no_proxy` is set to bypass all domains. - std::string no_proxy_string = "*"; - std::string poco_no_proxy_regex = buildPocoNonProxyHosts(no_proxy_string); - EnvironmentProxySetter setter(http_proxy_server, https_proxy_server, no_proxy_string); + const auto http_proxy_server = Poco::URI(EnvironmentProxySetter::HTTP_PROXY); + const auto https_proxy_server = Poco::URI(EnvironmentProxySetter::HTTPS_PROXY); + + std::string poco_no_proxy_regex = buildPocoNonProxyHosts(EnvironmentProxySetter::NO_PROXY); + + EnvironmentProxySetter setter; EnvironmentProxyConfigurationResolver http_resolver(ProxyConfiguration::Protocol::HTTP); From a12fec7f41203049ea6a454d6ac8832499ef7958 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 7 Jun 2024 14:22:23 +0000 Subject: [PATCH 411/856] 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 01ece1403520af82e5ba02892a938a44d0b5b45f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 7 Jun 2024 16:23:40 +0200 Subject: [PATCH 412/856] fix initialization order (ServerUUID/ZooKeeper) --- programs/server/Server.cpp | 28 ++++++++++++++-------------- src/Core/ServerUUID.cpp | 8 ++++++++ src/Core/ServerUUID.h | 2 +- 3 files changed, 23 insertions(+), 15 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..a35a8ab6de6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -944,6 +944,18 @@ try } } + std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); + fs::path path = path_str; + + /// Check that the process user id matches the owner of the data. + assertProcessUserMatchesDataOwner(path_str, [&](const std::string & message){ global_context->addWarningMessage(message); }); + + global_context->setPath(path_str); + + StatusFile status{path / "status", StatusFile::write_full_info}; + + ServerUUID::load(path / "uuid", log); + zkutil::validateZooKeeperConfig(config()); bool has_zookeeper = zkutil::hasZooKeeperConfig(config()); @@ -955,7 +967,7 @@ try ConfigProcessor config_processor(config_path); loaded_config = config_processor.loadConfigWithZooKeeperIncludes( main_config_zk_node_cache, main_config_zk_changed_event, /* fallback_to_preprocessed = */ true); - config_processor.savePreprocessedConfig(loaded_config, config().getString("path", DBMS_DEFAULT_PATH)); + config_processor.savePreprocessedConfig(loaded_config, path_str); config().removeConfiguration(old_configuration.get()); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); global_context->setConfig(loaded_config.configuration); @@ -1089,19 +1101,6 @@ try global_context->setRemoteHostFilter(config()); global_context->setHTTPHeaderFilter(config()); - std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); - fs::path path = path_str; - std::string default_database = server_settings.default_database.toString(); - - /// Check that the process user id matches the owner of the data. - assertProcessUserMatchesDataOwner(path_str, [&](const std::string & message){ global_context->addWarningMessage(message); }); - - global_context->setPath(path_str); - - StatusFile status{path / "status", StatusFile::write_full_info}; - - ServerUUID::load(path / "uuid", log); - /// Try to increase limit on number of open files. { rlimit rlim; @@ -1889,6 +1888,7 @@ try /// Set current database name before loading tables and databases because /// system logs may copy global context. + std::string default_database = server_settings.default_database.toString(); global_context->setCurrentDatabaseNameInGlobalContext(default_database); LOG_INFO(log, "Loading metadata from {}", path_str); diff --git a/src/Core/ServerUUID.cpp b/src/Core/ServerUUID.cpp index c2de6be7794..159aa8faadf 100644 --- a/src/Core/ServerUUID.cpp +++ b/src/Core/ServerUUID.cpp @@ -11,6 +11,14 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_CREATE_FILE; + extern const int LOGICAL_ERROR; +} + +UUID ServerUUID::get() +{ + if (server_uuid == UUIDHelpers::Nil) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ServerUUID is not initialized yet"); + return server_uuid; } void ServerUUID::load(const fs::path & server_uuid_file, Poco::Logger * log) diff --git a/src/Core/ServerUUID.h b/src/Core/ServerUUID.h index 71ae9edc00e..9b9963ceeeb 100644 --- a/src/Core/ServerUUID.h +++ b/src/Core/ServerUUID.h @@ -15,7 +15,7 @@ class ServerUUID public: /// Returns persistent UUID of current clickhouse-server or clickhouse-keeper instance. - static UUID get() { return server_uuid; } + static UUID get(); /// Loads server UUID from file or creates new one. Should be called on daemon startup. static void load(const fs::path & server_uuid_file, Poco::Logger * log); From dd0f38db05a04509de548ec149a48cbb763ed4a6 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 16:37:05 +0200 Subject: [PATCH 413/856] CI: Fix nightly workflow --- tests/ci/ci.py | 37 ++++++++++++++++++++++--------------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 55a18a2f335..73d1b1e4155 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -424,6 +424,7 @@ def _configure_jobs( s3: S3Helper, pr_info: PRInfo, ci_settings: CiSettings, + skip_jobs: bool, ) -> CiCache: """ returns CICache instance with configured job's data @@ -434,11 +435,14 @@ def _configure_jobs( """ # get all jobs - job_configs = CI_CONFIG.get_workflow_jobs_with_configs( - is_mq=pr_info.is_merge_queue, - is_docs_only=pr_info.has_changes_in_documentation_only(), - is_master=pr_info.is_master, - ) + if not skip_jobs: + job_configs = CI_CONFIG.get_workflow_jobs_with_configs( + is_mq=pr_info.is_merge_queue, + is_docs_only=pr_info.has_changes_in_documentation_only(), + is_master=pr_info.is_master, + ) + else: + job_configs = {} # filter jobs in accordance with ci settings job_configs = ci_settings.apply( @@ -447,7 +451,9 @@ def _configure_jobs( # check jobs in ci cache ci_cache = CiCache.calc_digests_and_create( - s3, job_configs, cache_enabled=not ci_settings.no_ci_cache and CI + s3, + job_configs, + cache_enabled=not ci_settings.no_ci_cache and not skip_jobs and CI, ) ci_cache.update() ci_cache.apply(job_configs, is_release=pr_info.is_release) @@ -971,6 +977,7 @@ def main() -> int: s3, pr_info, ci_settings, + args.skip_jobs, ) ci_cache.print_status() @@ -989,15 +996,15 @@ def main() -> int: 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["jobs_data"] = { - "jobs_to_do": list(ci_cache.jobs_to_do), - "jobs_to_skip": ci_cache.jobs_to_skip, - "digests": ci_cache.job_digests, - "jobs_params": { - job: {"batches": config.batches, "num_batches": config.num_batches} - for job, config in ci_cache.jobs_to_do.items() - }, - } + result["jobs_data"] = { + "jobs_to_do": list(ci_cache.jobs_to_do), + "jobs_to_skip": ci_cache.jobs_to_skip, + "digests": ci_cache.job_digests, + "jobs_params": { + job: {"batches": config.batches, "num_batches": config.num_batches} + for job, config in ci_cache.jobs_to_do.items() + }, + } result["docker_data"] = docker_data ### CONFIGURE action: end From f50a951e8e3cf80652ea525b4232833c846507f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 16:49:07 +0200 Subject: [PATCH 414/856] Fix innocuous data race in detectLanguage --- contrib/cld2 | 2 +- tests/queries/0_stateless/03168_cld2_tsan.reference | 2 ++ tests/queries/0_stateless/03168_cld2_tsan.sql | 10 ++++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03168_cld2_tsan.reference create mode 100644 tests/queries/0_stateless/03168_cld2_tsan.sql diff --git a/contrib/cld2 b/contrib/cld2 index bc6d493a2f6..217ba8b8805 160000 --- a/contrib/cld2 +++ b/contrib/cld2 @@ -1 +1 @@ -Subproject commit bc6d493a2f64ed1fc1c4c4b4294a542a04e04217 +Subproject commit 217ba8b8805b41557faadaa47bb6e99f2242eea3 diff --git a/tests/queries/0_stateless/03168_cld2_tsan.reference b/tests/queries/0_stateless/03168_cld2_tsan.reference new file mode 100644 index 00000000000..6c3cafd4a6d --- /dev/null +++ b/tests/queries/0_stateless/03168_cld2_tsan.reference @@ -0,0 +1,2 @@ +{'ja':0.62,'fr':0.36} +{'ja':0.62,'fr':0.36} diff --git a/tests/queries/0_stateless/03168_cld2_tsan.sql b/tests/queries/0_stateless/03168_cld2_tsan.sql new file mode 100644 index 00000000000..701a781c472 --- /dev/null +++ b/tests/queries/0_stateless/03168_cld2_tsan.sql @@ -0,0 +1,10 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: depends on cld2 + +-- https://github.com/ClickHouse/ClickHouse/issues/64931 +SELECT detectLanguageMixed(materialize('二兎を追う者は一兎をも得ず二兎を追う者は一兎をも得ず A vaincre sans peril, on triomphe sans gloire.')) +GROUP BY + GROUPING SETS ( + ('a', toUInt256(1)), + (stringToH3(toFixedString(toFixedString('85283473ffffff', 14), 14)))) +SETTINGS allow_experimental_nlp_functions = 1; From 8c4f5c65aa21569dbaea1d409d273a842a736437 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 7 Jun 2024 17:07:06 +0200 Subject: [PATCH 415/856] Use a named logger in build_download_helper --- tests/ci/build_download_helper.py | 34 ++++++++++++++++--------------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 036d3548eb9..0532d618802 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -15,6 +15,8 @@ from ci_config import CI_CONFIG DOWNLOAD_RETRIES_COUNT = 5 +logger = logging.getLogger(__name__) + class DownloadException(Exception): pass @@ -30,7 +32,7 @@ def get_with_retries( sleep: int = 3, **kwargs: Any, ) -> requests.Response: - logging.info( + logger.info( "Getting URL with %i tries and sleep %i in between: %s", retries, sleep, url ) exc = Exception("A placeholder to satisfy typing and avoid nesting") @@ -42,7 +44,7 @@ def get_with_retries( return response except Exception as e: if i + 1 < retries: - logging.info("Exception '%s' while getting, retry %i", e, i + 1) + logger.info("Exception '%s' while getting, retry %i", e, i + 1) time.sleep(sleep) exc = e @@ -96,7 +98,7 @@ def get_gh_api( ) try_auth = e.response.status_code == 404 if (ratelimit_exceeded or try_auth) and not token_is_set: - logging.warning( + logger.warning( "Received rate limit exception, setting the auth header and retry" ) set_auth_header() @@ -107,7 +109,7 @@ def get_gh_api( exc = e if try_cnt < retries: - logging.info("Exception '%s' while getting, retry %i", exc, try_cnt) + logger.info("Exception '%s' while getting, retry %i", exc, try_cnt) time.sleep(sleep) raise APIException(f"Unable to request data from GH API: {url}") from exc @@ -121,25 +123,25 @@ def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str for root, _, files in os.walk(reports_path): for file in files: if file.endswith(f"_{build_name}.json"): - logging.info("Found build report json %s for %s", file, build_name) + logger.info("Found build report json %s for %s", file, build_name) with open( os.path.join(root, file), "r", encoding="utf-8" ) as file_handler: build_report = json.load(file_handler) return build_report["build_urls"] # type: ignore - logging.info("A build report is not found for %s", build_name) + logger.info("A build report is not found for %s", build_name) return [] def download_build_with_progress(url: str, path: Path) -> None: - logging.info("Downloading from %s to temp path %s", url, path) + logger.info("Downloading from %s to temp path %s", url, path) for i in range(DOWNLOAD_RETRIES_COUNT): try: response = get_with_retries(url, retries=1, stream=True) total_length = int(response.headers.get("content-length", 0)) if path.is_file() and total_length and path.stat().st_size == total_length: - logging.info( + logger.info( "The file %s already exists and have a proper size %s", path, total_length, @@ -148,14 +150,14 @@ def download_build_with_progress(url: str, path: Path) -> None: with open(path, "wb") as f: if total_length == 0: - logging.info( + logger.info( "No content-length, will download file without progress" ) f.write(response.content) else: dl = 0 - logging.info("Content length is %ld bytes", total_length) + logger.info("Content length is %ld bytes", total_length) for data in response.iter_content(chunk_size=4096): dl += len(data) f.write(data) @@ -170,8 +172,8 @@ def download_build_with_progress(url: str, path: Path) -> None: except Exception as e: if sys.stdout.isatty(): sys.stdout.write("\n") - if os.path.exists(path): - os.remove(path) + if path.exists(): + path.unlink() if i + 1 < DOWNLOAD_RETRIES_COUNT: time.sleep(3) @@ -182,7 +184,7 @@ def download_build_with_progress(url: str, path: Path) -> None: if sys.stdout.isatty(): sys.stdout.write("\n") - logging.info("Downloading finished") + logger.info("Downloading finished") def download_builds( @@ -191,7 +193,7 @@ def download_builds( for url in build_urls: if filter_fn(url): fname = os.path.basename(url.replace("%2B", "+").replace("%20", " ")) - logging.info("Will download %s to %s", fname, result_path) + logger.info("Will download %s to %s", fname, result_path) download_build_with_progress(url, result_path / fname) @@ -203,7 +205,7 @@ def download_builds_filter( ) -> None: build_name = get_build_name_for_check(check_name) urls = read_build_urls(build_name, reports_path) - logging.info("The build report for %s contains the next URLs: %s", build_name, urls) + logger.info("The build report for %s contains the next URLs: %s", build_name, urls) if not urls: raise DownloadException("No build URLs found") @@ -240,7 +242,7 @@ def get_clickhouse_binary_url( ) -> Optional[str]: build_name = get_build_name_for_check(check_name) urls = read_build_urls(build_name, reports_path) - logging.info("The build report for %s contains the next URLs: %s", build_name, urls) + logger.info("The build report for %s contains the next URLs: %s", build_name, urls) for url in urls: check_url = url if "?" in check_url: From 98235ba2b1df3ce121959a0c09db2a40993302e6 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Fri, 7 Jun 2024 15:47:12 +0000 Subject: [PATCH 416/856] Dedicated support for Base64URL encoding --- .../functions/string-functions.md | 8 ++ .../functions/string-functions.md | 8 ++ src/Functions/FunctionBase64Conversion.h | 79 +++++++++++++++++++ src/Functions/base64UrlDecode.cpp | 14 ++++ src/Functions/base64UrlEncode.cpp | 14 ++++ .../03167_base64_url_functions.reference | 11 +++ .../03167_base64_url_functions.sql | 28 +++++++ 7 files changed, 162 insertions(+) create mode 100644 src/Functions/base64UrlDecode.cpp create mode 100644 src/Functions/base64UrlEncode.cpp create mode 100644 tests/queries/0_stateless/03167_base64_url_functions.reference create mode 100644 tests/queries/0_stateless/03167_base64_url_functions.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 342ca2b9f03..7365e0f4d27 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1140,12 +1140,20 @@ Encodes a String or FixedString as base64. Alias: `TO_BASE64`. +## base64UrlEncode + +Encodes an URL (String or FixedString) as base64 according to [RFC 4648](https://tools.ietf.org/html/rfc4648). + ## base64Decode Decodes a base64-encoded String or FixedString. Throws an exception in case of error. Alias: `FROM_BASE64`. +## base64UrlDecode + +Decodes a base64-encoded URL (String or FixedString) according to [RFC 4648](https://tools.ietf.org/html/rfc4648). Throws an exception in case of error. + ## tryBase64Decode Like `base64Decode` but returns an empty string in case of error. diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index fc258f7b4cf..c44cf94876f 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -538,12 +538,20 @@ SELECT base58Decode('3dc8KtHrwM'); Синоним: `TO_BASE64`. +## base64UrlEncode(s) + +Производит кодирование URL (String или FixedString) в base64-представление в соответствии с [RFC 4648](https://tools.ietf.org/html/rfc4648). + ## base64Decode(s) {#base64decode} Декодирует base64-представление s в исходную строку. При невозможности декодирования выбрасывает исключение Синоним: `FROM_BASE64`. +## base64UrlDecode(s) + +Декодирует base64-представление URL в исходную строку в соответствии с [RFC 4648](https://tools.ietf.org/html/rfc4648). При невозможности декодирования выбрасывает исключение + ## tryBase64Decode(s) {#trybase64decode} Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку. diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 3906563a254..008ce0b7338 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -85,6 +85,85 @@ struct TryBase64Decode } }; +struct Base64UrlEncode : Base64Encode +{ + static constexpr auto name = "base64UrlEncode"; + + static size_t perform(const std::span src, UInt8 * dst) + { + auto out_len = Base64Encode::perform(src, dst); + + // Do postprocessing as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 + for (size_t i = 0; i < out_len; ++i) + { + switch (dst[i]) + { + case '/': + dst[i] = '_'; + break; + case '+': + dst[i] = '-'; + break; + case '=': // stop when padding is detected + return i; + default: + break; + } + } + return out_len; + } +}; + +struct Base64UrlDecode : Base64Decode +{ + static constexpr auto name = "base64UrlDecode"; + + static size_t perform(const std::span src, UInt8 * dst) + { + std::vector tmp{}; + // insert padding to please alcomp library + auto size = src.size(); + auto remainder = size % 4; + switch (remainder) + { + case 0: + break; // no padding needed + case 1: + break; // invalid input, let it be detected by alcomp library + case 2: + size += 2; // two bytes padding + break; + default: // remainder == 3 + ++size; // one byte padding + } + tmp.resize(size); + + size_t i = 0; + for (; i < src.size(); ++i) + { + switch (src[i]) + { + case '_': + tmp[i] = '/'; + break; + case '-': + tmp[i] = '+'; + break; + default: + tmp[i] = src[i]; + break; + } + } + if (remainder == 2 || remainder == 3) + tmp[i++] = '='; + if (remainder == 2) + tmp[i++] = '='; + + return Base64Decode::perform(tmp, dst); + } +}; + + template class FunctionBase64Conversion : public IFunction { diff --git a/src/Functions/base64UrlDecode.cpp b/src/Functions/base64UrlDecode.cpp new file mode 100644 index 00000000000..fa2adac3c5f --- /dev/null +++ b/src/Functions/base64UrlDecode.cpp @@ -0,0 +1,14 @@ +#include + +#if USE_BASE64 +#include + +namespace DB +{ +REGISTER_FUNCTION(Base64UrlDecode) +{ + factory.registerFunction>(); +} +} + +#endif diff --git a/src/Functions/base64UrlEncode.cpp b/src/Functions/base64UrlEncode.cpp new file mode 100644 index 00000000000..a3775324145 --- /dev/null +++ b/src/Functions/base64UrlEncode.cpp @@ -0,0 +1,14 @@ +#include + +#if USE_BASE64 +#include + +namespace DB +{ +REGISTER_FUNCTION(Base64UrlEncode) +{ + factory.registerFunction>(); +} +} + +#endif diff --git a/tests/queries/0_stateless/03167_base64_url_functions.reference b/tests/queries/0_stateless/03167_base64_url_functions.reference new file mode 100644 index 00000000000..075d1729cef --- /dev/null +++ b/tests/queries/0_stateless/03167_base64_url_functions.reference @@ -0,0 +1,11 @@ +aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ +https://clickhouse.com +MTI_ +12? +aHR0cHM6Ly93d3cuZ29vZ2xlLmNvbS9zZWFyY2g_cT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmc2NhX2Vzdj03MzlmOGJiMzgwZTRjN2VkJmVpPVRmUmlacUNESXJtbndQQVAyS0xSa0E4JnZlZD0wYWhVS0V3amczWkhpdHNtR0F4VzVFeEFJSFZoUkZQSVE0ZFVEQ0JBJnVhY3Q9NSZvcT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmZ3NfbHA9RWd4bmQzTXRkMmw2TFhObGNuQWlHR05zYVdOcmFHOTFjMlVnWW1GelpUWTBJR1JsWTI5a1pUSUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUnpJS0VBQVlzQU1ZMWdRWVJ6SUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUnpJS0VBQVlzQU1ZMWdRWVJ6SUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUjBqWEJGQUFXQUJ3QVhnQmtBRUFtQUVBb0FFQXFnRUF1QUVEeUFFQW1BSUJvQUlIbUFNQWlBWUJrQVlJa2djQk1hQUhBQSZzY2xpZW50PWd3cy13aXotc2VycA +https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode&gs_lp=Egxnd3Mtd2l6LXNlcnAiGGNsaWNraG91c2UgYmFzZTY0IGRlY29kZTIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYR0jXBFAAWABwAXgBkAEAmAEAoAEAqgEAuAEDyAEAmAIBoAIHmAMAiAYBkAYIkgcBMaAHAA&sclient=gws-wiz-serp +https://clic +https://clickh +https://click +aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ +https://clickhouse.com diff --git a/tests/queries/0_stateless/03167_base64_url_functions.sql b/tests/queries/0_stateless/03167_base64_url_functions.sql new file mode 100644 index 00000000000..908ca890be2 --- /dev/null +++ b/tests/queries/0_stateless/03167_base64_url_functions.sql @@ -0,0 +1,28 @@ +SELECT base64UrlEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64UrlDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64UrlEncode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +-- test with valid inputs +SELECT base64UrlEncode('https://clickhouse.com'); +SELECT base64UrlDecode('aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ'); +-- encoding differs from base64Encode +SELECT base64UrlEncode('12?'); +SELECT base64UrlDecode('MTI_'); +-- long string +SELECT base64UrlEncode('https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode&gs_lp=Egxnd3Mtd2l6LXNlcnAiGGNsaWNraG91c2UgYmFzZTY0IGRlY29kZTIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYR0jXBFAAWABwAXgBkAEAmAEAoAEAqgEAuAEDyAEAmAIBoAIHmAMAiAYBkAYIkgcBMaAHAA&sclient=gws-wiz-serp'); +SELECT base64UrlDecode('aHR0cHM6Ly93d3cuZ29vZ2xlLmNvbS9zZWFyY2g_cT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmc2NhX2Vzdj03MzlmOGJiMzgwZTRjN2VkJmVpPVRmUmlacUNESXJtbndQQVAyS0xSa0E4JnZlZD0wYWhVS0V3amczWkhpdHNtR0F4VzVFeEFJSFZoUkZQSVE0ZFVEQ0JBJnVhY3Q9NSZvcT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmZ3NfbHA9RWd4bmQzTXRkMmw2TFhObGNuQWlHR05zYVdOcmFHOTFjMlVnWW1GelpUWTBJR1JsWTI5a1pUSUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUnpJS0VBQVlzQU1ZMWdRWVJ6SUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUnpJS0VBQVlzQU1ZMWdRWVJ6SUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUjBqWEJGQUFXQUJ3QVhnQmtBRUFtQUVBb0FFQXFnRUF1QUVEeUFFQW1BSUJvQUlIbUFNQWlBWUJrQVlJa2djQk1hQUhBQSZzY2xpZW50PWd3cy13aXotc2VycA'); +-- no padding +SELECT base64UrlDecode('aHR0cHM6Ly9jbGlj'); +-- one-byte padding +SELECT base64UrlDecode('aHR0cHM6Ly9jbGlja2g'); +-- two-bytes padding +SELECT base64UrlDecode('aHR0cHM6Ly9jbGljaw'); + +-- invalid inputs +SELECT base64UrlDecode('https://clickhouse.com'); -- { serverError INCORRECT_DATA } +SELECT base64UrlDecode('12?'); -- { serverError INCORRECT_DATA } + +-- test FixedString argument +SELECT base64UrlEncode(toFixedString('https://clickhouse.com', 22)); +SELECT base64UrlDecode(toFixedString('aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ', 30)); From 4c2783a0d8ae14b9da78815420f43e7b7eb9bab5 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 18:15:57 +0200 Subject: [PATCH 417/856] CI: Builds in CI settings --- .github/PULL_REQUEST_TEMPLATE.md | 8 +++----- tests/ci/ci_config.py | 13 +++++++++++-- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 51a1a6e2df8..73b2155482e 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -48,19 +48,17 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Allow: Stateful tests - [ ] Allow: Integration Tests - [ ] Allow: Performance tests +- [ ] Allow: Normal Builds +- [ ] Allow: Special Builds - [ ] Allow: All NOT Required Checks - [ ] Allow: batch 1, 2 for multi-batch jobs - [ ] Allow: batch 3, 4, 5, 6 for multi-batch jobs --- - [ ] Exclude: Style check - [ ] Exclude: Fast test -- [ ] Exclude: Integration Tests -- [ ] Exclude: Stateless tests -- [ ] Exclude: Stateful tests -- [ ] Exclude: Performance tests - [ ] Exclude: All with ASAN -- [ ] Exclude: All with Aarch64 - [ ] Exclude: All with TSAN, MSAN, UBSAN, Coverage +- [ ] Exclude: All with aarch64, release, debug --- - [ ] Do not test - [ ] Upload binaries for special builds diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index ca46088eb45..775e1f98b34 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -53,6 +53,8 @@ class CILabels(metaclass=WithIter): CI_SET_SYNC = "ci_set_sync" CI_SET_ARM = "ci_set_arm" CI_SET_REQUIRED = "ci_set_required" + CI_SET_NORMAL_BUILDS = "ci_set_normal_builds" + CI_SET_SPECIAL_BUILDS = "ci_set_special_builds" CI_SET_NON_REQUIRED = "ci_set_non_required" CI_SET_OLD_ANALYZER = "ci_set_old_analyzer" @@ -683,12 +685,13 @@ class CIConfig: return result def get_job_parents(self, check_name: str) -> List[str]: + if check_name in self.builds_report_config: + return self.builds_report_config[check_name].builds + res = [] check_name = normalize_string(check_name) - for config in ( self.build_config, - self.builds_report_config, self.test_configs, self.other_jobs_configs, ): @@ -899,6 +902,12 @@ CI_CONFIG = CIConfig( ] ), CILabels.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), + CILabels.CI_SET_NORMAL_BUILDS: LabelConfig( + run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK] + ), + CILabels.CI_SET_SPECIAL_BUILDS: LabelConfig( + run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK_SPECIAL] + ), CILabels.CI_SET_NON_REQUIRED: LabelConfig( run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] ), From 33cac00011b5c095f062b64eed5d6158ceb6b389 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 7 Jun 2024 19:39:48 +0200 Subject: [PATCH 418/856] Make a setting not important --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f8c7606d8b2..5436a18df93 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -334,7 +334,7 @@ class IColumn; M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ \ M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ - M(Bool, allow_experimental_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", IMPORTANT) \ + M(Bool, allow_experimental_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", 0) \ \ M(JoinStrictness, join_default_strictness, JoinStrictness::All, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ From 2d189eb1a3973ecd8cdde12507bb9bafc6878da4 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 7 Jun 2024 18:27:06 +0000 Subject: [PATCH 419/856] init --- .../AggregateFunctionGroupConcat.cpp | 265 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 2 + .../0_stateless/03156_group_concat.reference | 14 + .../0_stateless/03156_group_concat.sql | 40 +++ 4 files changed, 321 insertions(+) create mode 100644 src/AggregateFunctions/AggregateFunctionGroupConcat.cpp create mode 100644 tests/queries/0_stateless/03156_group_concat.reference create mode 100644 tests/queries/0_stateless/03156_group_concat.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp new file mode 100644 index 00000000000..767c536a8fd --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp @@ -0,0 +1,265 @@ +#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 58e657d3723..4ac25e14ee6 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -19,6 +19,7 @@ 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 &); @@ -120,6 +121,7 @@ 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 new file mode 100644 index 00000000000..75b347be0c4 --- /dev/null +++ b/tests/queries/0_stateless/03156_group_concat.reference @@ -0,0 +1,14 @@ +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 new file mode 100644 index 00000000000..c14fde8943a --- /dev/null +++ b/tests/queries/0_stateless/03156_group_concat.sql @@ -0,0 +1,40 @@ +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 54630c932e18b182c11b9366bd97bf564323550b Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Fri, 7 Jun 2024 18:28:30 +0000 Subject: [PATCH 420/856] Style fix --- 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 c35e860a5d7..c48b931dc54 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1008,6 +1008,8 @@ Updatable Uppercased Uptime Uptrace +UrlDecode +UrlEncode UserID Util VARCHAR From d993fa5cc610f7cc3c68bb9a8b8d52ff60942f8b Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 20:33:31 +0200 Subject: [PATCH 421/856] CI: Fix backports --- tests/ci/cherry_pick.py | 44 ++++++++++++++++++++--------------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index e470621e2c5..d7a78c1d143 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -127,13 +127,11 @@ close it. to_pop.append(i) elif pr.head.ref.startswith(f"backport/{self.name}"): self.backport_pr = pr + self._backported = True to_pop.append(i) else: assert False, f"BUG! Invalid PR's branch [{pr.head.ref}]" - # Cherry-pick or backport PR found, set @backported flag for current release branch - self._backported = True - for i in reversed(to_pop): # Going from the tail to keep the order and pop greater index first prs.pop(i) @@ -218,6 +216,7 @@ close it. self.name, self.pr.number, ) + self._backported = True return except CalledProcessError: # There are most probably conflicts, they'll be resolved in PR @@ -247,7 +246,6 @@ close it. self.cherrypick_pr.add_to_labels(Labels.PR_CRITICAL_BUGFIX) elif Labels.PR_BUGFIX in [label.name for label in self.pr.labels]: self.cherrypick_pr.add_to_labels(Labels.PR_BUGFIX) - self._backported = True self._assign_new_pr(self.cherrypick_pr) # update cherrypick PR to get the state for PR.mergable self.cherrypick_pr.update() @@ -359,10 +357,10 @@ class Backport: self._fetch_from = fetch_from self.dry_run = dry_run - self.must_create_backport_label = ( - Labels.MUST_BACKPORT + self.must_create_backport_labels = ( + [Labels.MUST_BACKPORT] if self._repo_name == self._fetch_from - else Labels.MUST_BACKPORT_CLOUD + else [Labels.MUST_BACKPORT_CLOUD, Labels.MUST_BACKPORT] ) self.backport_created_label = ( Labels.PR_BACKPORTS_CREATED @@ -468,7 +466,7 @@ class Backport: query_args = { "query": f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}", "label": ",".join( - self.labels_to_backport + [self.must_create_backport_label] + self.labels_to_backport + self.must_create_backport_labels ), "merged": [since_date, tomorrow], } @@ -492,20 +490,22 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - if self.must_create_backport_label in pr_labels: - branches = [ - ReleaseBranch(br, pr, self.repo, self.backport_created_label) - for br in self.release_branches - ] # type: List[ReleaseBranch] - else: - branches = [ - ReleaseBranch(br, pr, self.repo, self.backport_created_label) - for br in [ - label.split("-", 1)[0][1:] # v21.8-must-backport - for label in pr_labels - if label in self.labels_to_backport - ] - ] + # FIXME: currently backport to all branches, for branch-specified backports too + # Handle different branch name formats in cloud + # if self.must_create_backport_label in pr_labels: + branches = [ + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in self.release_branches + ] # type: List[ReleaseBranch] + # else: + # branches = [ + # ReleaseBranch(br, pr, self.repo, self.backport_created_label) + # for br in [ + # label.split("-", 1)[0][1:] # v21.8-must-backport + # for label in pr_labels + # if label in self.labels_to_backport + # ] + # ] assert branches, "BUG!" logging.info( From ca756b4d519f796d2b6a4f6f4a1cf62703d34f4b Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 20:42:07 +0200 Subject: [PATCH 422/856] CI: Fix backports --- tests/ci/cherry_pick.py | 30 ++++++++++++++---------------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index d7a78c1d143..375a5f6214a 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -490,22 +490,20 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - # FIXME: currently backport to all branches, for branch-specified backports too - # Handle different branch name formats in cloud - # if self.must_create_backport_label in pr_labels: - branches = [ - ReleaseBranch(br, pr, self.repo, self.backport_created_label) - for br in self.release_branches - ] # type: List[ReleaseBranch] - # else: - # branches = [ - # ReleaseBranch(br, pr, self.repo, self.backport_created_label) - # for br in [ - # label.split("-", 1)[0][1:] # v21.8-must-backport - # for label in pr_labels - # if label in self.labels_to_backport - # ] - # ] + if self.must_create_backport_labels in pr_labels or self._repo_name != self._fetch_from: + branches = [ + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in self.release_branches + ] # type: List[ReleaseBranch] + else: + branches = [ + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in [ + label.split("-", 1)[0][1:] # v21.8-must-backport + for label in pr_labels + if label in self.labels_to_backport + ] + ] assert branches, "BUG!" logging.info( From 18e12c48a464712f406a6654938a45c403adabaf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 7 Jun 2024 18:49:22 +0000 Subject: [PATCH 423/856] Automatic style fix --- tests/ci/cherry_pick.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 375a5f6214a..47e255b1882 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -490,7 +490,10 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - if self.must_create_backport_labels in pr_labels or self._repo_name != self._fetch_from: + if ( + any(label in pr_labels for label in self.must_create_backport_labels) + or self._repo_name != self._fetch_from + ): branches = [ ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in self.release_branches From 9a7f5d1e2071255451224e16ea80eb69e8ee4658 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 7 Jun 2024 20:17:48 +0000 Subject: [PATCH 424/856] 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 c7ec61e8e6044eee5b4823147fd3147e67ee6858 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 7 Jun 2024 17:37:29 -0300 Subject: [PATCH 425/856] Update settings.md --- docs/en/operations/settings/settings.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 65b0fa3f385..79d0ca4f151 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3878,6 +3878,10 @@ Possible values: Default value: 30. +:::note +It's applicable only to the default profile. A server reboot is required for the changes to take effect. +::: + ## http_receive_timeout {#http_receive_timeout} HTTP receive timeout (in seconds). From 9453a159355e4073ec05cb1bd4a7f319849152b2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 8 Jun 2024 00:12:41 +0200 Subject: [PATCH 426/856] Try fix 03143_prewhere_profile_events --- tests/queries/0_stateless/03143_prewhere_profile_events.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03143_prewhere_profile_events.sh b/tests/queries/0_stateless/03143_prewhere_profile_events.sh index 863fcc1fe01..00daa0fe7cc 100755 --- a/tests/queries/0_stateless/03143_prewhere_profile_events.sh +++ b/tests/queries/0_stateless/03143_prewhere_profile_events.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-random-merge-tree-settings +# Tags: no-random-settings, no-random-merge-tree-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -nq " DROP TABLE IF EXISTS t; - CREATE TABLE t(a UInt32, b UInt32, c UInt32, d UInt32) ENGINE=MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part=1, min_rows_for_wide_part=1; + CREATE TABLE t(a UInt32, b UInt32, c UInt32, d UInt32) ENGINE=MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=0; INSERT INTO t SELECT number, number, number, number FROM numbers_mt(1e7); From 3238bcf90af9dfb7d5ce86b834744f15c11cec7f Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 8 Jun 2024 09:48:56 +0200 Subject: [PATCH 427/856] Review changes --- .../functions/other-functions.md | 115 ++++++++---------- 1 file changed, 50 insertions(+), 65 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 4639f9d0991..5e63d9824b4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -228,29 +228,24 @@ Query: ```sql DROP TABLE IF EXISTS test; CREATE TABLE test (n UInt8) ENGINE = Memory; --- Insert 3 blocks: -INSERT INTO test VALUES (1); -INSERT INTO test VALUES (1),(2); -INSERT INTO test VALUES (1),(2),(3); -SELECT blockSize(), n FROM (SELECT * FROM test); +INSERT INTO test +SELECT * FROM system.numbers LIMIT 5; + +SELECT blockSize() +FROM test; ``` Result: ```response - ┌─blockSize()─┬─n─┐ -1. │ 1 │ 1 │ - └─────────────┴───┘ - ┌─blockSize()─┬─n─┐ -2. │ 3 │ 1 │ -3. │ 3 │ 2 │ -4. │ 3 │ 3 │ - └─────────────┴───┘ - ┌─blockSize()─┬─n─┐ -5. │ 2 │ 1 │ -6. │ 2 │ 2 │ - └─────────────┴───┘ + ┌─blockSize()─┐ +1. │ 5 │ +2. │ 5 │ +3. │ 5 │ +4. │ 5 │ +5. │ 5 │ + └─────────────┘ ``` ## byteSize @@ -3723,11 +3718,7 @@ Result: ## lowCardinalityIndices -For each row in the current [block](../../development/architecture.md/#block-block), returns the index of the value in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. - -:::note -The first unique value encountered in the block is enumerated from 1. -::: +Returns the position of a value in the dictionary of a [LowCardinality](../data-types/lowcardinality.md) column. Positions start at 1. Since LowCardinality have per-part dictionaries, this function may return different positions for the same value in different parts. **Syntax** @@ -3741,7 +3732,7 @@ lowCardinalityIndices(col) **Returned value** -- returns the index of the value in the dictionary of unique values, for each row in the current block. [UInt64](../data-types/int-uint.md). +- The position of the value in the dictionary of the current part. [UInt64](../data-types/int-uint.md). **Example** @@ -3751,10 +3742,10 @@ Query: DROP TABLE IF EXISTS test; CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; --- insert two blocks of data: +-- create two parts: -INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); -INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); +INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df'); +INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef'); SELECT s, lowCardinalityIndices(s) FROM test; ``` @@ -3762,29 +3753,24 @@ SELECT s, lowCardinalityIndices(s) FROM test; Result: ```response - ┌─s───┬─lowCardinalityIndices(s)─┐ -1. │ one │ 1 │ -2. │ two │ 2 │ -3. │ one │ 1 │ -4. │ one │ 1 │ -5. │ two │ 2 │ - └─────┴──────────────────────────┘ - ┌─s─────┬─lowCardinalityIndices(s)─┐ - 6. │ three │ 1 │ - 7. │ two │ 2 │ - 8. │ one │ 3 │ - 9. │ two │ 2 │ -10. │ two │ 2 │ -11. │ three │ 1 │ - └───────┴──────────────────────────┘ + ┌─s──┬─lowCardinalityIndices(s)─┐ +1. │ ab │ 1 │ +2. │ cd │ 2 │ +3. │ ab │ 1 │ +4. │ ab │ 1 │ +5. │ df │ 3 │ + └────┴──────────────────────────┘ + ┌─s──┬─lowCardinalityIndices(s)─┐ + 6. │ ef │ 1 │ + 7. │ cd │ 2 │ + 8. │ ab │ 3 │ + 9. │ cd │ 2 │ +10. │ ef │ 1 │ + └────┴──────────────────────────┘ ``` ## lowCardinalityKeys -For each row in the current [block](../../development/architecture.md/#block-block), returns the keys (unique values) in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. - -:::note -If the column size is less than the dictionary size, then values will be cut. If it is greater, then defaults will be added. -::: +Returns the dictionary values of a [LowCardinality](../data-types/lowcardinality.md) column. If the block is smaller or larger than the dictionary size, the result will be truncated or extended with default values. Since LowCardinality have per-part dictionaries, this function may return different dictionary values in different parts. **Syntax** @@ -3798,7 +3784,7 @@ lowCardinalityIndices(col) **Returned value** -- returns the keys of the dictionary, for each row in the current block. [UInt64](../data-types/int-uint.md). +- The dictionary keys. [UInt64](../data-types/int-uint.md). **Example** @@ -3808,10 +3794,10 @@ Query: DROP TABLE IF EXISTS test; CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; --- insert two blocks of data: +-- create two parts: -INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); -INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); +INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df'); +INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef'); SELECT s, lowCardinalityKeys(s) FROM test; ``` @@ -3819,19 +3805,18 @@ SELECT s, lowCardinalityKeys(s) FROM test; Result: ```response - ┌─s───┬─lowCardinalityKeys(s)─┐ -1. │ one │ │ -2. │ two │ one │ -3. │ one │ two │ -4. │ one │ │ -5. │ two │ │ - └─────┴───────────────────────┘ - ┌─s─────┬─lowCardinalityKeys(s)─┐ - 6. │ three │ │ - 7. │ two │ three │ - 8. │ one │ two │ - 9. │ two │ one │ -10. │ two │ │ -11. │ three │ │ - └───────┴───────────────────────┘ + ┌─s──┬─lowCardinalityKeys(s)─┐ +1. │ ef │ │ +2. │ cd │ ef │ +3. │ ab │ cd │ +4. │ cd │ ab │ +5. │ ef │ │ + └────┴───────────────────────┘ + ┌─s──┬─lowCardinalityKeys(s)─┐ + 6. │ ab │ │ + 7. │ cd │ ab │ + 8. │ ab │ cd │ + 9. │ ab │ df │ +10. │ df │ │ + └────┴───────────────────────┘ ``` From 9f90f2463aff93765483f65dc6c1884abb1339e6 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 8 Jun 2024 18:06:10 +0200 Subject: [PATCH 428/856] fix --- tests/ci/cherry_pick.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 47e255b1882..629464d0422 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -533,8 +533,8 @@ class Backport: br.process(self.dry_run) for br in branches: - assert br.backported, f"BUG! backport to branch [{br}] failed" - self.mark_pr_backported(pr) + if br.backported: + self.mark_pr_backported(pr) def mark_pr_backported(self, pr: PullRequest) -> None: if self.dry_run: From ab79addee492172440724df16de1c6a417ccd04f Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 9 Jun 2024 12:55:31 +0000 Subject: [PATCH 429/856] 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 430/856] 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 cdd2957a31fae18e44c2230a73e0770c545f793c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 9 Jun 2024 19:09:33 +0000 Subject: [PATCH 431/856] Move MergeTree setting docs into MergeTree settings docs page --- .../mergetree-family/mergetree.md | 136 ++++------------ .../settings/merge-tree-settings.md | 154 ++++++++++++++---- 2 files changed, 151 insertions(+), 139 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 803b753fe0a..0e53d7525b4 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -6,35 +6,26 @@ sidebar_label: MergeTree # MergeTree -The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most commonly used and most robust ClickHouse table engines. +The `MergeTree` engine and other engines of the `MergeTree` family (e.g. `ReplacingMergeTree`, `AggregatingMergeTree` ) are the most commonly used and most robust table engines in ClickHouse. -Engines in the `MergeTree` family are designed for inserting a very large amount of data into a table. The data is quickly written to the table part by part, then rules are applied for merging the parts in the background. This method is much more efficient than continually rewriting the data in storage during insert. +`MergeTree`-family table engines are designed for high data ingest rates and huge data volumes. +Insert operations create table parts which are merged by a background process with other table parts. -Main features: +Main features of `MergeTree`-family table engines. -- Stores data sorted by primary key. +- The table's primary key determines the sort order within each table part (clustered index). The primary key also does not reference individual rows but blocks of 8192 rows called granules. This makes primary keys of huge data sets small enough to remain loaded in main memory, while still providing fast access to on-disk data. - This allows you to create a small sparse index that helps find data faster. +- Tables can be partitioned using an arbitrary partition expression. Partition pruning ensures partitions are omitted from reading when the query allows it. -- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified. +- Data can be replicated across multiple cluster nodes for high availability, failover, and zero downtime upgrades. See [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - ClickHouse supports certain operations with partitions that are more efficient than general operations on the same data with the same result. ClickHouse also automatically cuts off the partition data where the partitioning key is specified in the query. +- `MergeTree` table engines support various statistics kinds and sampling methods to help query optimization. -- Data replication support. - - The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - -- Data sampling support. - - If necessary, you can set the data sampling method in the table. - -:::info -The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family. +:::note +Despite a similar name, the [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine is different from `*MergeTree` engines. ::: -If you need to update rows frequently, we recommend using the [`ReplacingMergeTree`](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md) table engine. Using `ALTER TABLE my_table UPDATE` to update rows triggers a mutation, which causes parts to be re-written and uses IO/resources. With `ReplacingMergeTree`, you can simply insert the updated rows and the old rows will be replaced according to the table sorting key. - -## Creating a Table {#table_engine-mergetree-creating-a-table} +## Creating Tables {#table_engine-mergetree-creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -59,23 +50,24 @@ ORDER BY expr [SETTINGS name = value, ...] ``` -For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md). +For a detailed description of the parameters, see the [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md) statement ### Query Clauses {#mergetree-query-clauses} #### ENGINE -`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters. +`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine has no parameters. #### ORDER_BY `ORDER BY` — The sorting key. -A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. +A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID + 1, EventDate)`. -ClickHouse uses the sorting key as a primary key if the primary key is not defined explicitly by the `PRIMARY KEY` clause. +If no primary key is defined (i.e. `PRIMARY KEY` was not specified), ClickHouse uses the the sorting key as primary key. -Use the `ORDER BY tuple()` syntax, if you do not need sorting, or set `create_table_empty_primary_key_by_default` to `true` to use the `ORDER BY tuple()` syntax by default. See [Selecting the Primary Key](#selecting-the-primary-key). +If no sorting is required, you can use syntax `ORDER BY tuple()`. +Alternatively, if setting `create_table_empty_primary_key_by_default` is enabled, `ORDER BY tuple()` is implicitly added to `CREATE TABLE` statements. See [Selecting a Primary Key](#selecting-a-primary-key). #### PARTITION BY @@ -87,100 +79,32 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da `PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional. -By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. +Specifying a sorting key (using `ORDER BY` clause) implicitly specifies a primary key. +It is usually not necessary to specify the primary key in addition to the primary key. #### SAMPLE BY -`SAMPLE BY` — An expression for sampling. Optional. +`SAMPLE BY` — A sampling expression. Optional. -If a sampling expression is used, the primary key must contain it. The result of a sampling expression must be an unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. +If specified, it must be contained in the primary key. +The sampling expression must result in an unsigned integer. + +Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. #### TTL -`TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. +`TTL` — A list of rules that specify the storage duration of rows and the logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. -Expression must have one `Date` or `DateTime` column as a result. Example: -``` -TTL date + INTERVAL 1 DAY -``` +Expression must result in a `Date` or `DateTime`, e.g. `TTL date + INTERVAL 1 DAY`. Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an action to be done with the part if the expression is satisfied (reaches current time): removal of expired rows, moving a part (if expression is satisfied for all rows in a part) to specified disk (`TO DISK 'xxx'`) or to volume (`TO VOLUME 'xxx'`), or aggregating values in expired rows. Default type of the rule is removal (`DELETE`). List of multiple rules can be specified, but there should be no more than one `DELETE` rule. + For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) -### SETTINGS -Additional parameters that control the behavior of the `MergeTree` (optional): +#### Settings -#### index_granularity - -`index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Data Storage](#mergetree-data-storage). - -#### index_granularity_bytes - -`index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Data Storage](#mergetree-data-storage). - -#### min_index_granularity_bytes - -`min_index_granularity_bytes` — Min allowed size of data granules in bytes. Default value: 1024b. To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. See [Data Storage](#mergetree-data-storage). - -#### enable_mixed_granularity_parts - -`enable_mixed_granularity_parts` — Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries. - -#### use_minimalistic_part_header_in_zookeeper - -`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”. - -#### min_merge_bytes_to_use_direct_io - -`min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes. - -#### merge_with_ttl_timeout - -`merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours). -#### merge_with_recompression_ttl_timeout - -`merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours). - -#### try_fetch_recompressed_part_timeout - -`try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours). - -#### write_final_mark - -`write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Don’t turn it off. - -#### merge_max_block_size - -`merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192. - -#### storage_policy - -`storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes). - -#### min_bytes_for_wide_part - -`min_bytes_for_wide_part`, `min_rows_for_wide_part` — Minimum number of bytes/rows in a data part that can be stored in `Wide` format. You can set one, both or none of these settings. See [Data Storage](#mergetree-data-storage). - -#### max_parts_in_total - -`max_parts_in_total` — Maximum number of parts in all partitions. - -#### max_compress_block_size - -`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - -#### min_compress_block_size - -`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - -#### max_partitions_to_read - -`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. - -#### allow_experimental_optimized_row_order - -`allow_experimental_optimized_row_order` - Experimental. Enables the optimization of the row order during inserts to improve the compressability of the data for compression codecs (e.g. LZ4). Analyzes and reorders the data, and thus increases the CPU overhead of inserts. +See [MergeTree Settings](../../../operations/settings/merge-tree-settings.md). **Example of Sections Setting** @@ -270,7 +194,7 @@ ClickHouse does not require a unique primary key. You can insert multiple rows w You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses but it is strongly discouraged. To allow this feature, turn on the [allow_nullable_key](/docs/en/operations/settings/settings.md/#allow-nullable-key) setting. The [NULLS_LAST](/docs/en/sql-reference/statements/select/order-by.md/#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. -### Selecting the Primary Key {#selecting-the-primary-key} +### Selecting a Primary Key {#selecting-a-primary-key} The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may: diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index c3f303dcd38..d791683ac2b 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -3,9 +3,126 @@ slug: /en/operations/settings/merge-tree-settings title: "MergeTree tables settings" --- -The values of `merge_tree` settings (for all MergeTree tables) can be viewed in the table `system.merge_tree_settings`, they can be overridden in `config.xml` in the `merge_tree` section, or set in the `SETTINGS` section of each table. +System table `system.merge_tree_settings` shows the globally set MergeTree settings. -These are example overrides for `max_suspicious_broken_parts`: +MergeTree settings can be set in the `merge_tree` section of the server config file, or specified for each `MergeTree` table individually in +the `SETTINGS` clause of the `CREATE TABLE` statement. + +Example for customizing setting `max_suspicious_broken_parts`: + +Configure the default for all `MergeTree` tables in the server configuration file: + +``` text + + 5 + +``` + +Set for a particular table: + +``` sql +CREATE TABLE tab +( + `A` Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS max_suspicious_broken_parts = 500; +``` + +Change the settings for a particular table using `ALTER TABLE ... MODIFY SETTING`: + +```sql +ALTER TABLE tab MODIFY SETTING max_suspicious_broken_parts = 100; + +-- reset to global default (value from system.merge_tree_settings) +ALTER TABLE tab RESET SETTING max_suspicious_broken_parts; +``` + +## index_granularity + +Maximum number of data rows between the marks of an index. + +Default value: 8192. + +## index_granularity_bytes + +Maximum size of data granules in bytes. + +Default value: 10Mb. + +To restrict the granule size only by number of rows, set to 0 (not recommended). + +## min_index_granularity_bytes + +Min allowed size of data granules in bytes. + +Default value: 1024b. + +To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. + +## enable_mixed_granularity_parts + +Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries. + +## use_minimalistic_part_header_in_zookeeper + +Storage method of the data parts headers in ZooKeeper. If enabled, ZooKeeper stores less data. For details, see [here](../server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper). + +## min_merge_bytes_to_use_direct_io + +The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. +When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. +If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). +If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. + +Default value: `10 * 1024 * 1024 * 1024` bytes. + +## merge_with_ttl_timeout + +Minimum delay in seconds before repeating a merge with delete TTL. + +Default value: `14400` seconds (4 hours). + +## merge_with_recompression_ttl_timeout + +Minimum delay in seconds before repeating a merge with recompression TTL. + +Default value: `14400` seconds (4 hours). + +## write_final_mark + +Enables or disables writing the final index mark at the end of data part (after the last byte). + +Default value: 1. + +Don’t change or bad things will happen. + +## storage_policy + +Storage policy. + +## min_bytes_for_wide_part + +Minimum number of bytes/rows in a data part that can be stored in `Wide` format. +You can set one, both or none of these settings. + +## max_compress_block_size + +Maximum size of blocks of uncompressed data before compressing for writing to a table. +You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). +The value specified when table is created overrides the global value for this setting. + +## min_compress_block_size + +Minimum size of blocks of uncompressed data required for compression when writing the next mark. +You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). +The value specified when table is created overrides the global value for this setting. + +## max_partitions_to_read + +Limits the maximum number of partitions that can be accessed in one query. +You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. ## max_suspicious_broken_parts @@ -17,37 +134,6 @@ Possible values: Default value: 100. -Override example in `config.xml`: - -``` text - - 5 - -``` - -An example to set in `SETTINGS` for a particular table: - -``` sql -CREATE TABLE foo -( - `A` Int64 -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS max_suspicious_broken_parts = 500; -``` - -An example of changing the settings for a specific table with the `ALTER TABLE ... MODIFY SETTING` command: - -``` sql -ALTER TABLE foo - MODIFY SETTING max_suspicious_broken_parts = 100; - --- reset to default (use value from system.merge_tree_settings) -ALTER TABLE foo - RESET SETTING max_suspicious_broken_parts; -``` - ## parts_to_throw_insert {#parts-to-throw-insert} If the number of active parts in a single partition exceeds the `parts_to_throw_insert` value, `INSERT` is interrupted with the `Too many parts (N). Merges are processing significantly slower than inserts` exception. @@ -301,6 +387,8 @@ Default value: 10800 ## try_fetch_recompressed_part_timeout +Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. + Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression. Possible values: From dfc4184d7ff76dace72caeed4c5effbbdc3aa906 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 9 Jun 2024 19:43:23 +0000 Subject: [PATCH 432/856] Fix ALTER MODIFY COMMENT in parameterized VIEWs --- src/Storages/AlterCommands.cpp | 5 ++++- .../03142_alter_comment_parameterized_view.reference | 0 .../0_stateless/03142_alter_comment_parameterized_view.sql | 4 ++++ 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference create mode 100644 tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 281fc72dfc4..f6d1bda422a 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1583,7 +1583,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const } } - if (all_columns.empty()) + /// Parameterized views do not have 'columns' in their metadata + bool is_parameterized_view = table->as() && table->as()->isParameterizedView(); + + if (!is_parameterized_view && all_columns.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot DROP or CLEAR all columns"); validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context); diff --git a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql new file mode 100644 index 00000000000..665bbbbc963 --- /dev/null +++ b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS test_table_comment; +CREATE VIEW test_table_comment AS SELECT toString({date_from:String}); +ALTER TABLE test_table_comment MODIFY COMMENT 'test comment'; +DROP TABLE test_table_comment; From c4605b690b1c757d72113352f0b8cc150b4686eb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 9 Jun 2024 22:15:38 +0000 Subject: [PATCH 433/856] fix tests --- .../0_stateless/03166_skip_indexes_vertical_merge_1.sql | 3 ++- .../0_stateless/03166_skip_indexes_vertical_merge_2.sql | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql index ac987c9c75c..d3e3b38a3cb 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql @@ -9,7 +9,8 @@ ORDER BY a SETTINGS merge_max_block_size = 8192, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1, - min_bytes_for_wide_part = 0; + min_bytes_for_wide_part = 0, + min_bytes_for_full_part_storage = 0; INSERT INTO t_ind_merge_1 SELECT number, number, rand(), rand() FROM numbers(1000); INSERT INTO t_ind_merge_1 SELECT number, number, rand(), rand() FROM numbers(1000); diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql index 2805059d918..b749e0c84b0 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql @@ -18,7 +18,8 @@ ORDER BY a SETTINGS index_granularity = 64, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1, - min_bytes_for_wide_part = 0; + min_bytes_for_wide_part = 0, + min_bytes_for_full_part_storage = 0; INSERT INTO t_ind_merge_2 SELECT number, number, rand(), rand(), rand(), rand() FROM numbers(1000); INSERT INTO t_ind_merge_2 SELECT number, number, rand(), rand(), rand(), rand() FROM numbers(1000); From 7997ce850c376d063303edf9ef5071f1ad47147f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 10 Jun 2024 01:12:44 +0200 Subject: [PATCH 434/856] include storageview --- src/Storages/AlterCommands.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index f6d1bda422a..c29deda6fc5 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include From 5e679a33e18e3782feabe3f848610c1baa5b2654 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 10 Jun 2024 01:43:27 +0000 Subject: [PATCH 435/856] Fix --- src/Databases/DatabasesCommon.cpp | 5 +++-- .../03142_alter_comment_parameterized_view.reference | 1 + .../0_stateless/03142_alter_comment_parameterized_view.sql | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index fc75f8e44b9..b9d182c9c9f 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -8,6 +8,7 @@ #include #include #include +#include "Common/logger_useful.h" #include #include #include @@ -44,11 +45,11 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function" " and doesn't have structure in metadata", backQuote(ast_create_query.getTable())); - if (!has_structure && !ast_create_query.is_dictionary) + if (!has_structure && !ast_create_query.is_dictionary && !ast_create_query.isParameterizedView()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot alter table {} metadata doesn't have structure", backQuote(ast_create_query.getTable())); - if (!ast_create_query.is_dictionary) + if (!ast_create_query.is_dictionary && !ast_create_query.isParameterizedView()) { ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); diff --git a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference index e69de29bb2d..9b93c75ea56 100644 --- a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference +++ b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference @@ -0,0 +1 @@ +CREATE VIEW default.test_table_comment AS (SELECT toString({date_from:String})) COMMENT \'test comment\' diff --git a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql index 665bbbbc963..14af304f98c 100644 --- a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql +++ b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS test_table_comment; CREATE VIEW test_table_comment AS SELECT toString({date_from:String}); ALTER TABLE test_table_comment MODIFY COMMENT 'test comment'; +SELECT create_table_query FROM system.tables WHERE name = 'test_table_comment'; DROP TABLE test_table_comment; From e6f88126964f3018f07d4137e89541f8b45806a3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 10 Jun 2024 06:53:25 +0000 Subject: [PATCH 436/856] fix mistake Signed-off-by: Duc Canh Le --- src/Common/filesystemHelpers.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index f8d209bc11f..09c4508b7b2 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -1,5 +1,5 @@ #include "filesystemHelpers.h" -#include +#include #include #if defined(OS_LINUX) @@ -371,11 +371,10 @@ Poco::Timestamp getModificationTimestamp(const std::string & path) void setModificationTime(const std::string & path, time_t time) { - struct timeval times[2]; - times[0].tv_usec = times[1].tv_usec = 0; - times[0].tv_sec = ::time(nullptr); - times[1].tv_sec = time; - if (utimes(path.c_str(), times) != 0) + struct utimbuf tb; + tb.actime = time; + tb.modtime = time; + if (utime(path.c_str(), &tb) != 0) DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot set modification time to file: {}", path); } From d59a17014461048ec215a45ce742f813c17f95c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 07:05:36 +0000 Subject: [PATCH 437/856] Docs for MergeTree: Capitalized SETTINGS --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 0e53d7525b4..f0c4e1b0e34 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -102,7 +102,7 @@ Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an ac For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) -#### Settings +#### SETTINGS See [MergeTree Settings](../../../operations/settings/merge-tree-settings.md). From 0f6f86314faec7aea15b75210cf632032ccd2884 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 10 Jun 2024 09:37:52 +0200 Subject: [PATCH 438/856] Better code --- src/Common/GWPAsan.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 4c6c8c7e9cc..8e9cbf8e842 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -81,12 +81,6 @@ struct ScopedEndOfReportDecorator Poco::LoggerPtr log; }; -constexpr std::string_view unknown_crash_text = - "GWP-ASan cannot provide any more information about this error. This may " - "occur due to a wild memory access into the GWP-ASan pool, or an " - "overflow/underflow that is > 512B in length.\n"; - - // Prints the provided error and metadata information. void printHeader(gwp_asan::Error error, uintptr_t fault_address, const gwp_asan::AllocationMetadata * allocation_meta, Poco::LoggerPtr log) { @@ -164,6 +158,11 @@ void printReport([[maybe_unused]] uintptr_t fault_address) const gwp_asan::AllocationMetadata * allocation_meta = __gwp_asan_get_metadata(state, GuardedAlloc.getMetadataRegion(), fault_address); + static constexpr std::string_view unknown_crash_text = + "GWP-ASan cannot provide any more information about this error. This may " + "occur due to a wild memory access into the GWP-ASan pool, or an " + "overflow/underflow that is > 512B in length.\n"; + if (allocation_meta == nullptr) { LOG_FATAL(logger, "*** GWP-ASan detected a memory error ***"); From ee94d68cb963172dc6c783f5e69b6f00a5732d2b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 5 Jun 2024 08:38:12 +0000 Subject: [PATCH 439/856] Less aggressive logging --- src/Storages/MergeTree/RowOrderOptimizer.cpp | 25 ++++++++++---------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/RowOrderOptimizer.cpp b/src/Storages/MergeTree/RowOrderOptimizer.cpp index 34f9fed4500..76b0d6452ad 100644 --- a/src/Storages/MergeTree/RowOrderOptimizer.cpp +++ b/src/Storages/MergeTree/RowOrderOptimizer.cpp @@ -78,9 +78,8 @@ std::vector getOtherColumnIndexes(const Block & block, const SortDescrip /// -------- /// 2 1 a 3 /// ---------------------- -EqualRanges getEqualRanges(const Block & block, const SortDescription & sort_description, const IColumn::Permutation & permutation, const LoggerPtr & log) +EqualRanges getEqualRanges(const Block & block, const SortDescription & sort_description, const IColumn::Permutation & permutation) { - LOG_TRACE(log, "Finding equal ranges"); EqualRanges ranges; const size_t rows = block.rows(); if (sort_description.empty()) @@ -122,11 +121,10 @@ void updatePermutationInEqualRange( const std::vector & other_column_indexes, IColumn::Permutation & permutation, const EqualRange & equal_range, - const std::vector & cardinalities) + const std::vector & cardinalities, + const LoggerPtr & log) { - LoggerPtr log = getLogger("RowOrderOptimizer"); - - LOG_TRACE(log, "Starting optimization in equal range"); + LOG_TEST(log, "Starting optimization in equal range"); std::vector column_order(other_column_indexes.size()); iota(column_order.begin(), column_order.end(), 0); @@ -134,17 +132,17 @@ void updatePermutationInEqualRange( stable_sort(column_order.begin(), column_order.end(), cmp); std::vector ranges = {equal_range}; - LOG_TRACE(log, "equal_range: .from: {}, .to: {}", equal_range.from, equal_range.to); + LOG_TEST(log, "equal_range: .from: {}, .to: {}", equal_range.from, equal_range.to); for (size_t i : column_order) { const size_t column_id = other_column_indexes[i]; const ColumnPtr & column = block.getByPosition(column_id).column; - LOG_TRACE(log, "i: {}, column_id: {}, column->getName(): {}, cardinality: {}", i, column_id, column->getName(), cardinalities[i]); + LOG_TEST(log, "i: {}, column_id: {}, column type: {}, cardinality: {}", i, column_id, column->getName(), cardinalities[i]); column->updatePermutation( IColumn::PermutationSortDirection::Ascending, IColumn::PermutationSortStability::Stable, 0, 1, permutation, ranges); } - LOG_TRACE(log, "Finish optimization in equal range"); + LOG_TEST(log, "Finish optimization in equal range"); } } @@ -156,7 +154,10 @@ void RowOrderOptimizer::optimize(const Block & block, const SortDescription & so LOG_TRACE(log, "Starting optimization"); if (block.columns() == 0) + { + LOG_TRACE(log, "Finished optimization (block has no columns)"); return; /// a table without columns, this should not happen in the first place ... + } if (permutation.empty()) { @@ -165,17 +166,17 @@ void RowOrderOptimizer::optimize(const Block & block, const SortDescription & so iota(permutation.data(), rows, IColumn::Permutation::value_type(0)); } - const EqualRanges equal_ranges = getEqualRanges(block, sort_description, permutation, log); + const EqualRanges equal_ranges = getEqualRanges(block, sort_description, permutation); const std::vector other_columns_indexes = getOtherColumnIndexes(block, sort_description); - LOG_TRACE(log, "block.columns(): {}, block.rows(): {}, sort_description.size(): {}, equal_ranges.size(): {}", block.columns(), block.rows(), sort_description.size(), equal_ranges.size()); + LOG_TRACE(log, "columns: {}, sorting key columns: {}, rows: {}, equal ranges: {}", block.columns(), sort_description.size(), block.rows(), equal_ranges.size()); for (const auto & equal_range : equal_ranges) { if (equal_range.size() <= 1) continue; const std::vector cardinalities = getCardinalitiesInPermutedRange(block, other_columns_indexes, permutation, equal_range); - updatePermutationInEqualRange(block, other_columns_indexes, permutation, equal_range, cardinalities); + updatePermutationInEqualRange(block, other_columns_indexes, permutation, equal_range, cardinalities, log); } LOG_TRACE(log, "Finished optimization"); From ccdaf6f5a42fd5be5120addbd72d8f19a956cbb0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 9 Jun 2024 19:12:47 +0000 Subject: [PATCH 440/856] Restrict to MergeTree --- docs/en/operations/settings/merge-tree-settings.md | 2 ++ src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ++++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index d791683ac2b..39d9cd69e8f 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -978,6 +978,8 @@ Default value: false Controls if the row order should be optimized during inserts to improve the compressability of the newly inserted table part. +Only has an effect for ordinary MergeTree-engine tables. Does nothing for specialized MergeTree engine tables (e.g. CollapsingMergeTree). + MergeTree tables are (optionally) compressed using [compression codecs](../../sql-reference/statements/create/table.md#column_compression_codec). Generic compression codecs such as LZ4 and ZSTD achieve maximum compression rates if the data exposes patterns. Long runs of the same value typically compress very well. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 8e304936747..bb8f104c2bb 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -503,7 +503,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted); } - if (data.getSettings()->allow_experimental_optimized_row_order) + if (data.getSettings()->allow_experimental_optimized_row_order + && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if it the optimization messes up specialized MergeTree engines. { RowOrderOptimizer::optimize(block, sort_description, perm); perm_ptr = &perm; @@ -730,7 +731,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocksAlreadySorted); } - if (data.getSettings()->allow_experimental_optimized_row_order) + if (data.getSettings()->allow_experimental_optimized_row_order + && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if it the optimization messes up specialized MergeTree engines. { RowOrderOptimizer::optimize(block, sort_description, perm); perm_ptr = &perm; From 25fd51e5bc9b3849451037f71f89fb2d88f0bb1b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 9 Jun 2024 19:18:21 +0000 Subject: [PATCH 441/856] Mark row order optimization non-experimental --- docs/en/operations/settings/merge-tree-settings.md | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- .../03166_optimize_row_order_during_insert.sql | 8 ++++---- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 39d9cd69e8f..b45dc290797 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -974,7 +974,7 @@ Default value: false - [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge) setting -### allow_experimental_optimized_row_order +### optimize_row_order Controls if the row order should be optimized during inserts to improve the compressability of the newly inserted table part. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index bb8f104c2bb..7aa9c12a24b 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -503,7 +503,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted); } - if (data.getSettings()->allow_experimental_optimized_row_order + if (data.getSettings()->optimize_row_order && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if it the optimization messes up specialized MergeTree engines. { RowOrderOptimizer::optimize(block, sort_description, perm); @@ -731,7 +731,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocksAlreadySorted); } - if (data.getSettings()->allow_experimental_optimized_row_order + if (data.getSettings()->optimize_row_order) && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if it the optimization messes up specialized MergeTree engines. { RowOrderOptimizer::optimize(block, sort_description, perm); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 026a1da7196..6ababefa530 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -94,6 +94,7 @@ struct Settings; M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background.", 0) \ M(Bool, add_implicit_sign_column_constraint_for_collapsing_engine, false, "If true, add implicit constraint for sign column for CollapsingMergeTree engine.", 0) \ M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ + M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ \ /* Part removal settings. */ \ M(UInt64, simultaneous_parts_removal_limit, 0, "Maximum number of parts to remove during one CleanupThread iteration (0 means unlimited).", 0) \ @@ -199,7 +200,6 @@ struct Settings; M(Bool, cache_populated_by_fetch, false, "Only available in ClickHouse Cloud", 0) \ M(Bool, force_read_through_cache_for_merges, false, "Force read-through filesystem cache for merges", 0) \ M(Bool, allow_experimental_replacing_merge_with_cleanup, false, "Allow experimental CLEANUP merges for ReplacingMergeTree with is_deleted column.", 0) \ - M(Bool, allow_experimental_optimized_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ \ /** Compress marks and primary key. */ \ M(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ diff --git a/tests/queries/0_stateless/03166_optimize_row_order_during_insert.sql b/tests/queries/0_stateless/03166_optimize_row_order_during_insert.sql index bb2f5e94d05..5fc71598e47 100644 --- a/tests/queries/0_stateless/03166_optimize_row_order_during_insert.sql +++ b/tests/queries/0_stateless/03166_optimize_row_order_during_insert.sql @@ -14,7 +14,7 @@ CREATE TABLE tab ( event Int8 ) ENGINE = MergeTree ORDER BY name -SETTINGS allow_experimental_optimized_row_order = true; +SETTINGS optimize_row_order = true; INSERT INTO tab VALUES ('Igor', 3), ('Egor', 1), ('Egor', 2), ('Igor', 2), ('Igor', 1); SELECT * FROM tab ORDER BY name SETTINGS max_threads=1; @@ -34,7 +34,7 @@ CREATE TABLE tab ( flag String ) ENGINE = MergeTree ORDER BY () -SETTINGS allow_experimental_optimized_row_order = True; +SETTINGS optimize_row_order = True; INSERT INTO tab VALUES ('Bob', 4, 100, '1'), ('Nikita', 2, 54, '1'), ('Nikita', 1, 228, '1'), ('Alex', 4, 83, '1'), ('Alex', 4, 134, '1'), ('Alex', 1, 65, '0'), ('Alex', 4, 134, '1'), ('Bob', 2, 53, '0'), ('Alex', 4, 83, '0'), ('Alex', 1, 63, '1'), ('Bob', 2, 53, '1'), ('Alex', 4, 192, '1'), ('Alex', 2, 128, '1'), ('Nikita', 2, 148, '0'), ('Bob', 4, 177, '0'), ('Nikita', 1, 173, '0'), ('Alex', 1, 239, '0'), ('Alex', 1, 63, '0'), ('Alex', 2, 224, '1'), ('Bob', 4, 177, '0'), ('Alex', 2, 128, '1'), ('Alex', 4, 134, '0'), ('Alex', 4, 83, '1'), ('Bob', 4, 100, '0'), ('Nikita', 2, 54, '1'), ('Alex', 1, 239, '1'), ('Bob', 2, 187, '1'), ('Alex', 1, 65, '1'), ('Bob', 2, 53, '1'), ('Alex', 2, 224, '0'), ('Alex', 4, 192, '0'), ('Nikita', 1, 173, '1'), ('Nikita', 2, 148, '1'), ('Bob', 2, 187, '1'), ('Nikita', 2, 208, '1'), ('Nikita', 2, 208, '0'), ('Nikita', 1, 228, '0'), ('Nikita', 2, 148, '0'); SELECT * FROM tab SETTINGS max_threads=1; @@ -58,7 +58,7 @@ CREATE TABLE tab ( flag Nullable(Int32) ) ENGINE = MergeTree ORDER BY (flag, money) -SETTINGS allow_experimental_optimized_row_order = True, allow_nullable_key = True; +SETTINGS optimize_row_order = True, allow_nullable_key = True; INSERT INTO tab VALUES ('AB', 0, 42, Null), ('AB', 0, 42, Null), ('A', 1, 42, Null), ('AB', 1, 9.81, 0), ('B', 0, 42, Null), ('B', -1, 3.14, Null), ('B', 1, 2.7, 1), ('B', 0, 42, 1), ('A', 1, 42, 1), ('B', 1, 42, Null), ('B', 0, 2.7, 1), ('A', 0, 2.7, 1), ('B', 2, 3.14, Null), ('A', 0, 3.14, Null), ('A', 1, 2.7, 1), ('A', 1, 42, Null); SELECT * FROM tab ORDER BY (flag, money) SETTINGS max_threads=1; @@ -89,7 +89,7 @@ CREATE TABLE tab ( tuple_column Tuple(UInt256) ) ENGINE = MergeTree() ORDER BY (fixed_str, event_date) -SETTINGS allow_experimental_optimized_row_order = True; +SETTINGS optimize_row_order = True; INSERT INTO tab VALUES ('A', '2020-01-01', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('A', '2020-01-01', [0.0, 1.1], NULL, 'example', {}, (26)), ('A', '2020-01-01', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5)), ('A', '2020-01-02', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('A', '2020-01-02', [0.0, 2.2], 10, 'example', {}, (26)), ('A', '2020-01-02', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5)), ('B', '2020-01-04', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('B', '2020-01-04', [0.0, 2.2], Null, 'example', {}, (26)), ('B', '2020-01-04', [2.2, 1.1], 1, 'some string', {'key2':'value2'}, (5)), ('B', '2020-01-05', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('B', '2020-01-05', [0.0, 2.2], Null, 'example', {}, (26)), ('B', '2020-01-05', [2.2, 1.1], 1, 'some other string', {'key':'value'}, (5)), ('C', '2020-01-04', [0.0, 1.1], 10, 'some string', {'key':'value'}, (5)), ('C', '2020-01-04', [0.0, 2.2], Null, 'example', {}, (26)), ('C', '2020-01-04', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5)); From 1cd61b804dd01e3b56b3f2714d7c9adffe248900 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 07:52:14 +0000 Subject: [PATCH 442/856] Fix build --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 7aa9c12a24b..a5fbca111f3 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -731,7 +731,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocksAlreadySorted); } - if (data.getSettings()->optimize_row_order) + if (data.getSettings()->optimize_row_order && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if it the optimization messes up specialized MergeTree engines. { RowOrderOptimizer::optimize(block, sort_description, perm); From 163cacf701361e04f3832cbeb91b770fbcd070c9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 07:53:29 +0000 Subject: [PATCH 443/856] Bump googletest --- contrib/googletest | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/googletest b/contrib/googletest index e47544ad31c..a7f443b80b1 160000 --- a/contrib/googletest +++ b/contrib/googletest @@ -1 +1 @@ -Subproject commit e47544ad31cb3ceecd04cc13e8fe556f8df9fe0b +Subproject commit a7f443b80b105f940225332ed3c31f2790092f47 From c6e43f7a7b74a8928c1a9bf0a572aadcb56e8c54 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 08:17:01 +0000 Subject: [PATCH 444/856] Bump absl to 2023-11-28 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 3bd86026c93..0c09fd0ff0d 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 3bd86026c93da5a40006fd53403dff9d5f5e30e3 +Subproject commit 0c09fd0ff0d502c30831ff2ccf59894e36d2b60a From 4d3d18cee71ecadf520868623130538c1d3179e3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 08:18:22 +0000 Subject: [PATCH 445/856] Bump absl to 2023-12-06 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 0c09fd0ff0d..8588e7d14dc 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 0c09fd0ff0d502c30831ff2ccf59894e36d2b60a +Subproject commit 8588e7d14dca32eb2c695a9cd49d272aa23cc483 From 70c0589675d3c3c7f9a17d805818601fc0bd698e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 08:19:20 +0000 Subject: [PATCH 446/856] Bump absl to 2023-12-12 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 8588e7d14dc..ad0a6d2faf8 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 8588e7d14dca32eb2c695a9cd49d272aa23cc483 +Subproject commit ad0a6d2faf803645c8126f0b67eee2eaad98bc3f From 1bca6b900bb55aaadecdb24f07a60b18b6677eb8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 08:20:25 +0000 Subject: [PATCH 447/856] Bump absl to 2023-12-20 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index ad0a6d2faf8..794352a92f0 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit ad0a6d2faf803645c8126f0b67eee2eaad98bc3f +Subproject commit 794352a92f09425714b9116974b29e58ce8f9ba9 From 8fe272f210c7d214cedfaffa8eb72f73cb7756be Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 08:22:27 +0000 Subject: [PATCH 448/856] Bump absl to 2024-01-02 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 794352a92f0..925a5e681ea 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 794352a92f09425714b9116974b29e58ce8f9ba9 +Subproject commit 925a5e681ea1958171ba580c4402e5ce76473cb5 From a0d8d5a37ca944f6cb135444112f4906deb03371 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 08:31:33 +0000 Subject: [PATCH 449/856] Bump absl to 2024-01-02 --- contrib/abseil-cpp | 2 +- contrib/abseil-cpp-cmake/CMakeLists.txt | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 925a5e681ea..4038192a57c 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 925a5e681ea1958171ba580c4402e5ce76473cb5 +Subproject commit 4038192a57cb75f7ee671f81a3378ff4c74c4f8e diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index 7372195bb0d..be42d98345e 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -1283,12 +1283,9 @@ absl_cc_library( absl_cc_library( NAME flags - SRCS - "${DIR}/flag.cc" HDRS "${DIR}/declare.h" "${DIR}/flag.h" - "${DIR}/internal/flag_msvc.inc" COPTS ${ABSL_DEFAULT_COPTS} LINKOPTS From d4a453aad5eaaceb993570bb369e9321bc17bcf8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 08:32:22 +0000 Subject: [PATCH 450/856] Bump absl to 2024-03-06 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 4038192a57c..6f0bb2747d0 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 4038192a57cb75f7ee671f81a3378ff4c74c4f8e +Subproject commit 6f0bb2747d0a910de4a958eeeab2b9d615156382 From ae7d8821a78b5ff7a2228ca463323ca90c385a96 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 08:46:40 +0000 Subject: [PATCH 451/856] Bump absl to 2024-04-04 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 6f0bb2747d0..1ec4a27e399 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 6f0bb2747d0a910de4a958eeeab2b9d615156382 +Subproject commit 1ec4a27e39944462a574abbfa040498ed2831cc8 From d80cba90fb467b40d098c028c163fa1deba07dd9 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 10 Jun 2024 11:22:06 +0200 Subject: [PATCH 452/856] CI: Remove fuzzer build from normal CI run (bugfix) --- tests/ci/ci_config.py | 35 ++++++++++++++++++++++++++++++----- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 775e1f98b34..412a55b8534 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -685,9 +685,6 @@ class CIConfig: return result def get_job_parents(self, check_name: str) -> List[str]: - if check_name in self.builds_report_config: - return self.builds_report_config[check_name].builds - res = [] check_name = normalize_string(check_name) for config in ( @@ -903,10 +900,38 @@ CI_CONFIG = CIConfig( ), CILabels.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), CILabels.CI_SET_NORMAL_BUILDS: LabelConfig( - run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK] + run_jobs=[ + JobNames.STYLE_CHECK, + JobNames.BUILD_CHECK, + Build.PACKAGE_RELEASE, + Build.PACKAGE_AARCH64, + Build.PACKAGE_ASAN, + Build.PACKAGE_UBSAN, + Build.PACKAGE_TSAN, + Build.PACKAGE_MSAN, + Build.PACKAGE_DEBUG, + Build.BINARY_RELEASE, + Build.PACKAGE_RELEASE_COVERAGE, + Build.FUZZERS, + ] ), CILabels.CI_SET_SPECIAL_BUILDS: LabelConfig( - run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK_SPECIAL] + run_jobs=[ + JobNames.STYLE_CHECK, + JobNames.BUILD_CHECK_SPECIAL, + Build.BINARY_TIDY, + Build.BINARY_DARWIN, + Build.BINARY_AARCH64, + Build.BINARY_AARCH64_V80COMPAT, + Build.BINARY_FREEBSD, + Build.BINARY_DARWIN_AARCH64, + Build.BINARY_PPC64LE, + Build.BINARY_RISCV64, + Build.BINARY_S390X, + Build.BINARY_LOONGARCH64, + Build.BINARY_AMD64_COMPAT, + Build.BINARY_AMD64_MUSL, + ] ), CILabels.CI_SET_NON_REQUIRED: LabelConfig( run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] From 0da0d8dfb1fc73fb70926338f2e946ad5737d880 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Mon, 10 Jun 2024 10:14:45 +0000 Subject: [PATCH 453/856] PR post-review fixes --- .../functions/string-functions.md | 4 + .../functions/string-functions.md | 4 + src/Functions/FunctionBase64Conversion.cpp | 93 +++++++++++++ src/Functions/FunctionBase64Conversion.h | 127 ++++++------------ src/Functions/base64Decode.cpp | 2 +- src/Functions/base64Encode.cpp | 2 +- src/Functions/base64UrlDecode.cpp | 2 +- src/Functions/base64UrlEncode.cpp | 2 +- src/Functions/tryBase64Decode.cpp | 2 +- src/Functions/tryBase64UrlDecode.cpp | 14 ++ .../03167_base64_url_functions.reference | 21 ++- .../03167_base64_url_functions.sql | 28 ++-- 12 files changed, 188 insertions(+), 113 deletions(-) create mode 100644 src/Functions/FunctionBase64Conversion.cpp create mode 100644 src/Functions/tryBase64UrlDecode.cpp diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 7365e0f4d27..6c8f09e74ce 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1158,6 +1158,10 @@ Decodes a base64-encoded URL (String or FixedString) according to [RFC 4648](htt Like `base64Decode` but returns an empty string in case of error. +## tryBase64UrlDecode + +Like `base64UrlDecode` but returns an empty string in case of error. + **Syntax** ```sql diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index c44cf94876f..fa76e84f130 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -556,6 +556,10 @@ SELECT base58Decode('3dc8KtHrwM'); Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку. +## tryBase64UrlDecode(s) + +Функционал аналогичен base64UrlDecode, но при невозможности декодирования возвращает пустую строку. + ## endsWith(s, suffix) {#endswith} Возвращает 1, если строка завершается указанным суффиксом, и 0 в противном случае. diff --git a/src/Functions/FunctionBase64Conversion.cpp b/src/Functions/FunctionBase64Conversion.cpp new file mode 100644 index 00000000000..a87ce31f478 --- /dev/null +++ b/src/Functions/FunctionBase64Conversion.cpp @@ -0,0 +1,93 @@ +#include "config.h" +#if USE_BASE64 +# include +# include +# include + +namespace DB +{ + +std::vector preprocessBase64Url(const std::span src) +{ + std::vector padded_src{}; + // insert padding to please aklomp library + size_t padded_size = src.size(); + size_t remainder = padded_size % 4; + switch (remainder) + { + case 0: + break; // no padding needed + case 1: + padded_size += 3; // this case is impossible to occur, however, we'll insert padding anyway + break; + case 2: + padded_size += 2; // two bytes padding + break; + default: // remainder == 3 + padded_size += 1; // one byte padding + break; + } + padded_src.resize(padded_size); + + // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 + size_t i = 0; + for (; i < src.size(); ++i) + { + switch (src[i]) + { + case '_': + padded_src[i] = '/'; + break; + case '-': + padded_src[i] = '+'; + break; + default: + padded_src[i] = src[i]; + break; + } + } + if (remainder == 1) + { + padded_src[i] = '='; + ++i; + padded_src[i] = '='; + ++i; + padded_src[i] = '='; + } + else if (remainder == 2) + { + padded_src[i] = '='; + ++i; + padded_src[i] = '='; + } + else if (remainder == 3) + padded_src[i] = '='; + + return padded_src; +} + +size_t postprocessBase64Url(UInt8 * dst, size_t out_len) +{ + // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 + for (size_t i = 0; i < out_len; ++i) + { + switch (dst[i]) + { + case '/': + dst[i] = '_'; + break; + case '+': + dst[i] = '-'; + break; + case '=': // stop when padding is detected + return i; + default: + break; + } + } + return out_len; +} + +} + +#endif diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 008ce0b7338..51ca28aa670 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -22,9 +22,19 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } +enum class Base64Variant : uint8_t +{ + Normal, + Url +}; + +extern std::vector preprocessBase64Url(const std::span src); +extern size_t postprocessBase64Url(UInt8 * dst, size_t out_len); + +template struct Base64Encode { - static constexpr auto name = "base64Encode"; + static constexpr auto name = (variant == Base64Variant::Normal) ? "base64Encode" : "base64UrlEncode"; static size_t getBufferSize(size_t string_length, size_t string_count) { @@ -35,13 +45,18 @@ struct Base64Encode { size_t outlen = 0; base64_encode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); - return outlen; + + if constexpr (variant == Base64Variant::Url) + return postprocessBase64Url(dst, outlen); + else + return outlen; } }; +template struct Base64Decode { - static constexpr auto name = "base64Decode"; + static constexpr auto name = (variant == Base64Variant::Normal) ? "base64Decode" : "base64UrlDecode"; static size_t getBufferSize(size_t string_length, size_t string_count) { @@ -50,8 +65,17 @@ struct Base64Decode static size_t perform(const std::span src, UInt8 * dst) { + int rc; size_t outlen = 0; - int rc = base64_decode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); + if constexpr (variant == Base64Variant::Url) + { + auto src_padded = preprocessBase64Url(src); + rc = base64_decode(reinterpret_cast(src_padded.data()), src_padded.size(), reinterpret_cast(dst), &outlen, 0); + } + else + { + rc = base64_decode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); + } if (rc != 1) throw Exception( @@ -64,19 +88,29 @@ struct Base64Decode } }; +template struct TryBase64Decode { - static constexpr auto name = "tryBase64Decode"; + static constexpr auto name = (variant == Base64Variant::Normal) ? "tryBase64Decode" : "tryBase64UrlDecode"; static size_t getBufferSize(size_t string_length, size_t string_count) { - return Base64Decode::getBufferSize(string_length, string_count); + return Base64Decode::getBufferSize(string_length, string_count); } static size_t perform(const std::span src, UInt8 * dst) { + int rc; size_t outlen = 0; - int rc = base64_decode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); + if constexpr (variant == Base64Variant::Url) + { + auto src_padded = preprocessBase64Url(src); + rc = base64_decode(reinterpret_cast(src_padded.data()), src_padded.size(), reinterpret_cast(dst), &outlen, 0); + } + else + { + rc = base64_decode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); + } if (rc != 1) outlen = 0; @@ -85,85 +119,6 @@ struct TryBase64Decode } }; -struct Base64UrlEncode : Base64Encode -{ - static constexpr auto name = "base64UrlEncode"; - - static size_t perform(const std::span src, UInt8 * dst) - { - auto out_len = Base64Encode::perform(src, dst); - - // Do postprocessing as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 - for (size_t i = 0; i < out_len; ++i) - { - switch (dst[i]) - { - case '/': - dst[i] = '_'; - break; - case '+': - dst[i] = '-'; - break; - case '=': // stop when padding is detected - return i; - default: - break; - } - } - return out_len; - } -}; - -struct Base64UrlDecode : Base64Decode -{ - static constexpr auto name = "base64UrlDecode"; - - static size_t perform(const std::span src, UInt8 * dst) - { - std::vector tmp{}; - // insert padding to please alcomp library - auto size = src.size(); - auto remainder = size % 4; - switch (remainder) - { - case 0: - break; // no padding needed - case 1: - break; // invalid input, let it be detected by alcomp library - case 2: - size += 2; // two bytes padding - break; - default: // remainder == 3 - ++size; // one byte padding - } - tmp.resize(size); - - size_t i = 0; - for (; i < src.size(); ++i) - { - switch (src[i]) - { - case '_': - tmp[i] = '/'; - break; - case '-': - tmp[i] = '+'; - break; - default: - tmp[i] = src[i]; - break; - } - } - if (remainder == 2 || remainder == 3) - tmp[i++] = '='; - if (remainder == 2) - tmp[i++] = '='; - - return Base64Decode::perform(tmp, dst); - } -}; - - template class FunctionBase64Conversion : public IFunction { diff --git a/src/Functions/base64Decode.cpp b/src/Functions/base64Decode.cpp index 5f7a3406c62..a7a243b6d7d 100644 --- a/src/Functions/base64Decode.cpp +++ b/src/Functions/base64Decode.cpp @@ -7,7 +7,7 @@ namespace DB { REGISTER_FUNCTION(Base64Decode) { - factory.registerFunction>(); + factory.registerFunction>>(); /// MySQL compatibility alias. factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive); diff --git a/src/Functions/base64Encode.cpp b/src/Functions/base64Encode.cpp index 69268f5a25d..1599505a413 100644 --- a/src/Functions/base64Encode.cpp +++ b/src/Functions/base64Encode.cpp @@ -7,7 +7,7 @@ namespace DB { REGISTER_FUNCTION(Base64Encode) { - factory.registerFunction>(); + factory.registerFunction>>(); /// MySQL compatibility alias. factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive); diff --git a/src/Functions/base64UrlDecode.cpp b/src/Functions/base64UrlDecode.cpp index fa2adac3c5f..1ed836768b8 100644 --- a/src/Functions/base64UrlDecode.cpp +++ b/src/Functions/base64UrlDecode.cpp @@ -7,7 +7,7 @@ namespace DB { REGISTER_FUNCTION(Base64UrlDecode) { - factory.registerFunction>(); + factory.registerFunction>>(); } } diff --git a/src/Functions/base64UrlEncode.cpp b/src/Functions/base64UrlEncode.cpp index a3775324145..9d959c6bbc6 100644 --- a/src/Functions/base64UrlEncode.cpp +++ b/src/Functions/base64UrlEncode.cpp @@ -7,7 +7,7 @@ namespace DB { REGISTER_FUNCTION(Base64UrlEncode) { - factory.registerFunction>(); + factory.registerFunction>>(); } } diff --git a/src/Functions/tryBase64Decode.cpp b/src/Functions/tryBase64Decode.cpp index bd452b8357b..da1a24fd776 100644 --- a/src/Functions/tryBase64Decode.cpp +++ b/src/Functions/tryBase64Decode.cpp @@ -7,7 +7,7 @@ namespace DB { REGISTER_FUNCTION(TryBase64Decode) { - factory.registerFunction>(); + factory.registerFunction>>(); } } diff --git a/src/Functions/tryBase64UrlDecode.cpp b/src/Functions/tryBase64UrlDecode.cpp new file mode 100644 index 00000000000..528018b26f9 --- /dev/null +++ b/src/Functions/tryBase64UrlDecode.cpp @@ -0,0 +1,14 @@ +#include + +#if USE_BASE64 +#include + +namespace DB +{ +REGISTER_FUNCTION(TryBase64UrlDecode) +{ + factory.registerFunction>>(); +} +} + +#endif diff --git a/tests/queries/0_stateless/03167_base64_url_functions.reference b/tests/queries/0_stateless/03167_base64_url_functions.reference index 075d1729cef..2a0d0013609 100644 --- a/tests/queries/0_stateless/03167_base64_url_functions.reference +++ b/tests/queries/0_stateless/03167_base64_url_functions.reference @@ -1,11 +1,10 @@ -aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ -https://clickhouse.com -MTI_ -12? -aHR0cHM6Ly93d3cuZ29vZ2xlLmNvbS9zZWFyY2g_cT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmc2NhX2Vzdj03MzlmOGJiMzgwZTRjN2VkJmVpPVRmUmlacUNESXJtbndQQVAyS0xSa0E4JnZlZD0wYWhVS0V3amczWkhpdHNtR0F4VzVFeEFJSFZoUkZQSVE0ZFVEQ0JBJnVhY3Q9NSZvcT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmZ3NfbHA9RWd4bmQzTXRkMmw2TFhObGNuQWlHR05zYVdOcmFHOTFjMlVnWW1GelpUWTBJR1JsWTI5a1pUSUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUnpJS0VBQVlzQU1ZMWdRWVJ6SUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUnpJS0VBQVlzQU1ZMWdRWVJ6SUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUjBqWEJGQUFXQUJ3QVhnQmtBRUFtQUVBb0FFQXFnRUF1QUVEeUFFQW1BSUJvQUlIbUFNQWlBWUJrQVlJa2djQk1hQUhBQSZzY2xpZW50PWd3cy13aXotc2VycA -https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode&gs_lp=Egxnd3Mtd2l6LXNlcnAiGGNsaWNraG91c2UgYmFzZTY0IGRlY29kZTIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYR0jXBFAAWABwAXgBkAEAmAEAoAEAqgEAuAEDyAEAmAIBoAIHmAMAiAYBkAYIkgcBMaAHAA&sclient=gws-wiz-serp -https://clic -https://clickh -https://click -aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ -https://clickhouse.com +https://clickhouse.com aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ https://clickhouse.com https://clickhouse.com +12? MTI_ 12? 12? +https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode aHR0cHM6Ly93d3cuZ29vZ2xlLmNvbS9zZWFyY2g_cT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmc2NhX2Vzdj03MzlmOGJiMzgwZTRjN2VkJmVpPVRmUmlacUNESXJtbndQQVAyS0xSa0E4JnZlZD0wYWhVS0V3amczWkhpdHNtR0F4VzVFeEFJSFZoUkZQSVE0ZFVEQ0JBJnVhY3Q9NSZvcT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGU https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode +aHR0cHM6Ly9jbGlj https://clic https://clic +aHR0cHM6Ly9jbGlja2g https://clickh https://clickh +aHR0cHM6Ly9jbGljaw https://click https://click + + + +https://clickhouse.com aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ https://clickhouse.com https://clickhouse.com diff --git a/tests/queries/0_stateless/03167_base64_url_functions.sql b/tests/queries/0_stateless/03167_base64_url_functions.sql index 908ca890be2..60bb1746e90 100644 --- a/tests/queries/0_stateless/03167_base64_url_functions.sql +++ b/tests/queries/0_stateless/03167_base64_url_functions.sql @@ -1,28 +1,34 @@ +-- incorrect number of arguments SELECT base64UrlEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64UrlDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryBase64UrlDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64UrlEncode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryBase64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- test with valid inputs -SELECT base64UrlEncode('https://clickhouse.com'); -SELECT base64UrlDecode('aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ'); +SELECT 'https://clickhouse.com' as original, base64UrlEncode(original) as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; + -- encoding differs from base64Encode -SELECT base64UrlEncode('12?'); -SELECT base64UrlDecode('MTI_'); +SELECT '12?' as original, base64UrlEncode(original) as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; + -- long string -SELECT base64UrlEncode('https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode&gs_lp=Egxnd3Mtd2l6LXNlcnAiGGNsaWNraG91c2UgYmFzZTY0IGRlY29kZTIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYRzIKEAAYsAMY1gQYR0jXBFAAWABwAXgBkAEAmAEAoAEAqgEAuAEDyAEAmAIBoAIHmAMAiAYBkAYIkgcBMaAHAA&sclient=gws-wiz-serp'); -SELECT base64UrlDecode('aHR0cHM6Ly93d3cuZ29vZ2xlLmNvbS9zZWFyY2g_cT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmc2NhX2Vzdj03MzlmOGJiMzgwZTRjN2VkJmVpPVRmUmlacUNESXJtbndQQVAyS0xSa0E4JnZlZD0wYWhVS0V3amczWkhpdHNtR0F4VzVFeEFJSFZoUkZQSVE0ZFVEQ0JBJnVhY3Q9NSZvcT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmZ3NfbHA9RWd4bmQzTXRkMmw2TFhObGNuQWlHR05zYVdOcmFHOTFjMlVnWW1GelpUWTBJR1JsWTI5a1pUSUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUnpJS0VBQVlzQU1ZMWdRWVJ6SUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUnpJS0VBQVlzQU1ZMWdRWVJ6SUtFQUFZc0FNWTFnUVlSeklLRUFBWXNBTVkxZ1FZUjBqWEJGQUFXQUJ3QVhnQmtBRUFtQUVBb0FFQXFnRUF1QUVEeUFFQW1BSUJvQUlIbUFNQWlBWUJrQVlJa2djQk1hQUhBQSZzY2xpZW50PWd3cy13aXotc2VycA'); +SELECT 'https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode' as original, base64UrlEncode(original) as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; + -- no padding -SELECT base64UrlDecode('aHR0cHM6Ly9jbGlj'); +SELECT 'aHR0cHM6Ly9jbGlj' as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; -- one-byte padding -SELECT base64UrlDecode('aHR0cHM6Ly9jbGlja2g'); +SELECT 'aHR0cHM6Ly9jbGlja2g' as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; -- two-bytes padding -SELECT base64UrlDecode('aHR0cHM6Ly9jbGljaw'); +SELECT 'aHR0cHM6Ly9jbGljaw' as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; -- invalid inputs SELECT base64UrlDecode('https://clickhouse.com'); -- { serverError INCORRECT_DATA } +SELECT tryBase64UrlDecode('https://clickhouse.com'); SELECT base64UrlDecode('12?'); -- { serverError INCORRECT_DATA } +SELECT tryBase64UrlDecode('12?'); +SELECT base64UrlDecode('aHR0cHM6Ly9jbGlja'); -- { serverError INCORRECT_DATA } +SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja'); -- test FixedString argument -SELECT base64UrlEncode(toFixedString('https://clickhouse.com', 22)); -SELECT base64UrlDecode(toFixedString('aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ', 30)); +SELECT toFixedString('https://clickhouse.com', 22) as original, base64UrlEncode(original) as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; 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 454/856] empty commit From 61b464321759e8edf0fad69aa80c8b0daef1818c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 10:39:35 +0000 Subject: [PATCH 455/856] Bump absl to 2024-04-24 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 1ec4a27e399..08b21bd0379 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 1ec4a27e39944462a574abbfa040498ed2831cc8 +Subproject commit 08b21bd037990c18d44fda1691211e73835bf214 From 643444eb1134c9e3767efeb1698e1553b1c686af Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 10:41:47 +0000 Subject: [PATCH 456/856] Bump absl to 2024-05-03 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 08b21bd0379..c1e1b47d989 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 08b21bd037990c18d44fda1691211e73835bf214 +Subproject commit c1e1b47d989978cde8c5a2a219df425b785a0c47 From fdfc5471635c8d6675add87796a35260ea1d966a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 10 Jun 2024 10:34:14 +0200 Subject: [PATCH 457/856] Add useful comment --- src/Common/AsynchronousMetrics.h | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index b62529a08e7..10a972d2458 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -45,14 +45,17 @@ struct ProtocolServerMetrics }; /** Periodically (by default, each second) - * calculates and updates some metrics, - * that are not updated automatically (so, need to be asynchronously calculated). + * calculates and updates some metrics, + * that are not updated automatically (so, need to be asynchronously calculated). * - * This includes both ClickHouse-related metrics (like memory usage of ClickHouse process) - * and common OS-related metrics (like total memory usage on the server). + * This includes both general process metrics (like memory usage) + * and common OS-related metrics (like total memory usage on the server). * * All the values are either gauge type (like the total number of tables, the current memory usage). * Or delta-counters representing some accumulation during the interval of time. + * + * Server and Keeper specific metrics are contained inside + * ServerAsynchronousMetrics and KeeperAsynchronousMetrics respectively. */ class AsynchronousMetrics { From af08c2bc13c06f369dcdb1a19e36663fcd9e5531 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 10 Jun 2024 12:50:02 +0200 Subject: [PATCH 458/856] Increase default sample rate --- src/Common/GWPAsan.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 8e9cbf8e842..088f34fa6ae 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -56,6 +56,9 @@ static bool guarded_alloc_initialized = [] if (!env_options_raw || !std::string_view{env_options_raw}.contains("MaxSimultaneousAllocations")) opts.MaxSimultaneousAllocations = 1024; + if (!env_options_raw || !std::string_view{env_options_raw}.contains("SampleRate")) + opts.SampleRate = 10000; + opts.Backtrace = getBackTrace; GuardedAlloc.init(opts); From c95ed40d3eb7db5fcef1a5a51c3964e11cb77f56 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 11:32:29 +0000 Subject: [PATCH 459/856] Bump absl to 2024-05-06 --- contrib/abseil-cpp | 2 +- contrib/abseil-cpp-cmake/CMakeLists.txt | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index c1e1b47d989..a28ee5b51c9 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit c1e1b47d989978cde8c5a2a219df425b785a0c47 +Subproject commit a28ee5b51c9ea41707d9a5d2d358ad77850264c4 diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index be42d98345e..d026a7c78bc 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -1058,8 +1058,10 @@ absl_cc_library( demangle_internal HDRS "${DIR}/internal/demangle.h" + "${DIR}/internal/demangle_rust.h" SRCS "${DIR}/internal/demangle.cc" + "${DIR}/internal/demangle_rust.cc" COPTS ${ABSL_DEFAULT_COPTS} DEPS From da91dd64283de30172fca6cd30df4c711d291b44 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 11:35:07 +0000 Subject: [PATCH 460/856] Bump absl to 2024-06-07 --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index a28ee5b51c9..696b32788ca 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit a28ee5b51c9ea41707d9a5d2d358ad77850264c4 +Subproject commit 696b32788ca887881547380530926314c521ea7d From 4c629bcfb32dc0fa6e0dd1c8ee237461143da066 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 13:47:22 +0200 Subject: [PATCH 461/856] Review fixes --- .../NamedCollectionsFactory.cpp | 49 +++++++++---------- .../NamedCollectionsMetadataStorage.cpp | 42 +++++++++++++--- .../NamedCollectionsMetadataStorage.h | 5 +- 3 files changed, 60 insertions(+), 36 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.cpp b/src/Common/NamedCollections/NamedCollectionsFactory.cpp index 4edeb7251b8..14105a8651d 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.cpp +++ b/src/Common/NamedCollections/NamedCollectionsFactory.cpp @@ -237,7 +237,7 @@ bool NamedCollectionFactory::loadIfNot(std::lock_guard & lock) if (metadata_storage->supportsPeriodicUpdate()) { - update_task = context->getMessageBrokerSchedulePool().createTask("NamedCollectionsMetadataStorage", [this]{ updateFunc(); }); + update_task = context->getSchedulePool().createTask("NamedCollectionsMetadataStorage", [this]{ updateFunc(); }); update_task->activate(); update_task->schedule(); } @@ -363,38 +363,35 @@ void NamedCollectionFactory::updateFunc() while (!shutdown_called.load()) { - NamedCollectionsMap collections; - try + if (metadata_storage->waitUpdate()) { - reloadFromSQL(); - } - catch (const Coordination::Exception & e) - { - if (Coordination::isHardwareError(e.code)) + try { - LOG_INFO(log, "Lost ZooKeeper connection, will try to connect again: {}", - DB::getCurrentExceptionMessage(true)); - - sleepForSeconds(1); + reloadFromSQL(); } - else + catch (const Coordination::Exception & e) { - tryLogCurrentException(__PRETTY_FUNCTION__); + if (Coordination::isHardwareError(e.code)) + { + LOG_INFO(log, "Lost ZooKeeper connection, will try to connect again: {}", + DB::getCurrentExceptionMessage(true)); + + sleepForSeconds(1); + } + else + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); + } + continue; + } + catch (...) + { + DB::tryLogCurrentException(__PRETTY_FUNCTION__); chassert(false); + continue; } - continue; } - catch (...) - { - DB::tryLogCurrentException(__PRETTY_FUNCTION__); - chassert(false); - continue; - } - - if (shutdown_called.load()) - break; - - metadata_storage->waitUpdate(); } LOG_TRACE(log, "Named collections background updating thread finished"); diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index 3c0a62e3b0b..32fdb25abd3 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -69,7 +69,7 @@ public: virtual bool supportsPeriodicUpdate() const = 0; - virtual void waitUpdate(size_t /* timeout */) {} + virtual bool waitUpdate(size_t /* timeout */) { return false; } }; @@ -196,6 +196,7 @@ private: std::string root_path; mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; mutable zkutil::EventPtr wait_event; + mutable Int32 collections_node_cversion = 0; public: ZooKeeperStorage(ContextPtr context_, const std::string & path_) @@ -222,17 +223,44 @@ public: bool supportsPeriodicUpdate() const override { return true; } - void waitUpdate(size_t timeout) override + /// Return true if children changed. + bool waitUpdate(size_t timeout) override { - if (wait_event) - wait_event->tryWait(timeout); + if (!wait_event) + { + /// We did not yet made any list() attempt, so do that. + return true; + } + + if (wait_event->tryWait(timeout)) + { + /// Children changed before timeout. + return true; + } + + std::string res; + Coordination::Stat stat; + + if (!getClient()->tryGet(root_path, res, &stat)) + { + /// We do create root_path in constructor of this class, + /// so this case is not really possible. + chassert(false); + return false; + } + + return stat.cversion != collections_node_cversion; } std::vector list() const override { if (!wait_event) wait_event = std::make_shared(); - return getClient()->getChildren(root_path, nullptr, wait_event); + + Coordination::Stat stat; + auto children = getClient()->getChildren(root_path, &stat, wait_event); + collections_node_cversion = stat.cversion; + return children; } bool exists(const std::string & path) const override @@ -442,7 +470,7 @@ bool NamedCollectionsMetadataStorage::supportsPeriodicUpdate() const return storage->supportsPeriodicUpdate(); } -void NamedCollectionsMetadataStorage::waitUpdate() +bool NamedCollectionsMetadataStorage::waitUpdate() { if (!storage->supportsPeriodicUpdate()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Periodic updates are not supported"); @@ -450,7 +478,7 @@ void NamedCollectionsMetadataStorage::waitUpdate() const auto & config = Context::getGlobalContextInstance()->getConfigRef(); const size_t timeout = config.getUInt(named_collections_storage_config_path + ".update_timeout_ms", 5000); - storage->waitUpdate(timeout); + return storage->waitUpdate(timeout); } std::unique_ptr NamedCollectionsMetadataStorage::create(const ContextPtr & context_) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h index 4762d55bf99..3c089fe2fa2 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h @@ -7,8 +7,6 @@ namespace DB { -class NamedCollectionsMetadataStorage; -std::unique_ptr checkKek(const ContextPtr & context); class NamedCollectionsMetadataStorage : private WithContext { @@ -29,7 +27,8 @@ public: void shutdown(); - void waitUpdate(); + /// Return true if update was made + bool waitUpdate(); bool supportsPeriodicUpdate() const; From 132aa996a3f25fa98f3edc1ad92bbc22725d4c8d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 10 Jun 2024 13:49:39 +0200 Subject: [PATCH 462/856] Revert "Fix duplicating Delete events in blob_storage_log" --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 ++--- src/Interpreters/SystemLog.cpp | 7 +------ 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index afc13251f5b..ae719f5cde4 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -382,7 +382,6 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { std::vector current_chunk; String keys; - size_t first_position = current_position; for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position) { Aws::S3::Model::ObjectIdentifier obj; @@ -408,9 +407,9 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError(); auto time_now = std::chrono::system_clock::now(); - for (size_t i = first_position; i < current_position; ++i) + for (const auto & object : objects) blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, - uri.bucket, objects[i].remote_path, objects[i].local_path, objects[i].bytes_size, + uri.bucket, object.remote_path, object.local_path, object.bytes_size, outcome_error, time_now); } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3b25deeb59d..5e0ce2cb0de 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -504,10 +504,6 @@ void SystemLog::flushImpl(const std::vector & to_flush, Block block(std::move(log_element_columns)); MutableColumns columns = block.mutateColumns(); - - for (auto & column : columns) - column->reserve(to_flush.size()); - for (const auto & elem : to_flush) elem.appendToBlock(columns); @@ -536,8 +532,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to flush system log {} with {} entries up to offset {}", - table_id.getNameForLogs(), to_flush.size(), to_flush_end)); + tryLogCurrentException(__PRETTY_FUNCTION__); } queue->confirm(to_flush_end); From a50019c6ca2f4a20b59866fac68387a63787ab0e Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 11:49:41 +0000 Subject: [PATCH 463/856] Add metrics for PutObject and GetObject throttlers of ObjectStorage --- src/Common/Throttler.cpp | 39 ++++++++++++------- src/Common/Throttler.h | 6 +++ src/IO/S3/Client.h | 3 ++ .../ServerAsynchronousMetrics.cpp | 25 ++++++++++++ 4 files changed, 60 insertions(+), 13 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 4c1320db27a..61d120e11ea 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -43,19 +43,7 @@ UInt64 Throttler::add(size_t amount) // Values obtained under lock to be checked after release size_t count_value; double tokens_value; - { - std::lock_guard lock(mutex); - auto now = clock_gettime_ns_adjusted(prev_ns); - if (max_speed) - { - double delta_seconds = prev_ns ? static_cast(now - prev_ns) / NS : 0; - tokens = std::min(tokens + max_speed * delta_seconds - amount, max_burst); - } - count += amount; - count_value = count; - tokens_value = tokens; - prev_ns = now; - } + addImpl(amount, count_value, tokens_value); if (limit && count_value > limit) throw Exception::createDeprecated(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED); @@ -77,6 +65,21 @@ UInt64 Throttler::add(size_t amount) return static_cast(sleep_time_ns); } +void Throttler::addImpl(size_t amount, size_t & count_value, double & tokens_value) +{ + std::lock_guard lock(mutex); + auto now = clock_gettime_ns_adjusted(prev_ns); + if (max_speed) + { + double delta_seconds = prev_ns ? static_cast(now - prev_ns) / NS : 0; + tokens = std::min(tokens + max_speed * delta_seconds - amount, max_burst); + } + count += amount; + count_value = count; + tokens_value = tokens; + prev_ns = now; +} + void Throttler::reset() { std::lock_guard lock(mutex); @@ -98,4 +101,14 @@ bool Throttler::isThrottling() const return false; } +Int64 Throttler::getAvailable() +{ + // To update bucket state and receive current number of token in a thread-safe way + size_t count_value; + double tokens_value; + addImpl(0, count_value, tokens_value); + + return static_cast(tokens_value); +} + } diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 7508065096b..32293d7400f 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -57,7 +57,13 @@ public: /// Is throttler already accumulated some sleep time and throttling. bool isThrottling() const; + Int64 getAvailable(); + UInt64 getMaxSpeed() const { return static_cast(max_speed); } + UInt64 getMaxBurst() const { return static_cast(max_burst); } + private: + void addImpl(size_t amount, size_t & count_value, double & tokens_value); + size_t count{0}; const size_t max_speed{0}; /// in tokens per second. const size_t max_burst{0}; /// in tokens. diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index bd281846343..2fd0a9cd2d1 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -219,6 +219,9 @@ public: return client_configuration.for_disk_s3; } + ThrottlerPtr getPutRequestThrottler() const { return client_configuration.put_request_throttler; } + ThrottlerPtr getGetRequestThrottler() const { return client_configuration.get_request_throttler; } + private: friend struct ::MockS3::Client; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 315202cc01d..20409577332 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -231,6 +231,31 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr if (unreserved) new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved, "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." }; + + try + { + if (auto s3_client = disk->getS3StorageClient()) + { + if (auto put_throttler = s3_client->getPutRequestThrottler()) + { + new_values[fmt::format("DiskPutObjectThrottlerRPS_{}", name)] = { put_throttler->getMaxSpeed(), + "PutObject Request throttling limit on the disk in requests per second (virtual filesystem). Local filesystems may not provide this information." }; + new_values[fmt::format("DiskPutObjectThrottlerAvailable_{}", name)] = { put_throttler->getAvailable(), + "Number of PutObject requests that can be currently issued without hitting throttling limit on the disk (virtual filesystem). Local filesystems may not provide this information." }; + } + if (auto get_throttler = s3_client->getGetRequestThrottler()) + { + new_values[fmt::format("DiskGetObjectThrottlerRPS_{}", name)] = { get_throttler->getMaxSpeed(), + "GetObject Request throttling limit on the disk in requests per second (virtual filesystem). Local filesystems may not provide this information." }; + new_values[fmt::format("DiskGetObjectThrottlerAvailable_{}", name)] = { get_throttler->getAvailable(), + "Number of GetObject requests that can be currently issued without hitting throttling limit on the disk (virtual filesystem). Local filesystems may not provide this information." }; + } + } + } + catch(...) + { + // Skip disk than do not have s3 throttlers + } } } From 2056ed8ee83ce8bca37492d521d5a3d2f7d19242 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 10 Jun 2024 13:54:46 +0200 Subject: [PATCH 464/856] Capture weak_ptr for safety --- src/Access/ContextAccess.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 2a658d7aaa2..28a825de6cf 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -360,10 +360,13 @@ void ContextAccess::setUser(const UserPtr & user_) const subscription_for_roles_changes.reset(); enabled_roles = access_control->getEnabledRoles(current_roles, current_roles_with_admin_option); - subscription_for_roles_changes = enabled_roles->subscribeForChanges([this](const std::shared_ptr & roles_info_) + subscription_for_roles_changes = enabled_roles->subscribeForChanges([weak_ptr = weak_from_this()](const std::shared_ptr & roles_info_) { - std::lock_guard lock{mutex}; - setRolesInfo(roles_info_); + auto ptr = weak_ptr.lock(); + if (!ptr) + return; + std::lock_guard lock{ptr->mutex}; + ptr->setRolesInfo(roles_info_); }); setRolesInfo(enabled_roles->getRolesInfo()); From 41cca9df1a251c36c2b4ff7f8469ceb7c9ca0251 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 11:59:52 +0000 Subject: [PATCH 465/856] add space --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 20409577332..2892025da2d 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -252,7 +252,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr } } } - catch(...) + catch (...) { // Skip disk than do not have s3 throttlers } From 9e538b70bb452022d5a3dff0509a901032432c49 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Mon, 10 Jun 2024 12:16:03 +0000 Subject: [PATCH 466/856] Try to fix base64Url functions registration --- src/Functions/base64Decode.cpp | 1 + src/Functions/base64Encode.cpp | 1 + src/Functions/base64UrlDecode.cpp | 14 -------------- src/Functions/base64UrlEncode.cpp | 14 -------------- src/Functions/tryBase64Decode.cpp | 1 + src/Functions/tryBase64UrlDecode.cpp | 14 -------------- 6 files changed, 3 insertions(+), 42 deletions(-) delete mode 100644 src/Functions/base64UrlDecode.cpp delete mode 100644 src/Functions/base64UrlEncode.cpp delete mode 100644 src/Functions/tryBase64UrlDecode.cpp diff --git a/src/Functions/base64Decode.cpp b/src/Functions/base64Decode.cpp index a7a243b6d7d..2c0cf27c592 100644 --- a/src/Functions/base64Decode.cpp +++ b/src/Functions/base64Decode.cpp @@ -8,6 +8,7 @@ namespace DB REGISTER_FUNCTION(Base64Decode) { factory.registerFunction>>(); + factory.registerFunction>>(); /// MySQL compatibility alias. factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive); diff --git a/src/Functions/base64Encode.cpp b/src/Functions/base64Encode.cpp index 1599505a413..07ca28d6a87 100644 --- a/src/Functions/base64Encode.cpp +++ b/src/Functions/base64Encode.cpp @@ -8,6 +8,7 @@ namespace DB REGISTER_FUNCTION(Base64Encode) { factory.registerFunction>>(); + factory.registerFunction>>(); /// MySQL compatibility alias. factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive); diff --git a/src/Functions/base64UrlDecode.cpp b/src/Functions/base64UrlDecode.cpp deleted file mode 100644 index 1ed836768b8..00000000000 --- a/src/Functions/base64UrlDecode.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include - -#if USE_BASE64 -#include - -namespace DB -{ -REGISTER_FUNCTION(Base64UrlDecode) -{ - factory.registerFunction>>(); -} -} - -#endif diff --git a/src/Functions/base64UrlEncode.cpp b/src/Functions/base64UrlEncode.cpp deleted file mode 100644 index 9d959c6bbc6..00000000000 --- a/src/Functions/base64UrlEncode.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include - -#if USE_BASE64 -#include - -namespace DB -{ -REGISTER_FUNCTION(Base64UrlEncode) -{ - factory.registerFunction>>(); -} -} - -#endif diff --git a/src/Functions/tryBase64Decode.cpp b/src/Functions/tryBase64Decode.cpp index da1a24fd776..25da111492d 100644 --- a/src/Functions/tryBase64Decode.cpp +++ b/src/Functions/tryBase64Decode.cpp @@ -8,6 +8,7 @@ namespace DB REGISTER_FUNCTION(TryBase64Decode) { factory.registerFunction>>(); + factory.registerFunction>>(); } } diff --git a/src/Functions/tryBase64UrlDecode.cpp b/src/Functions/tryBase64UrlDecode.cpp deleted file mode 100644 index 528018b26f9..00000000000 --- a/src/Functions/tryBase64UrlDecode.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include - -#if USE_BASE64 -#include - -namespace DB -{ -REGISTER_FUNCTION(TryBase64UrlDecode) -{ - factory.registerFunction>>(); -} -} - -#endif From 05ad1e1b18af346b813f13f2879fec99a6532333 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 3 Jun 2024 13:22:57 +0200 Subject: [PATCH 467/856] Remove unused `Git.new_branch` --- tests/ci/git_helper.py | 1 - tests/ci/test_git.py | 5 ++--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/ci/git_helper.py b/tests/ci/git_helper.py index 8ec90dd7b2d..3316ae7dfdf 100644 --- a/tests/ci/git_helper.py +++ b/tests/ci/git_helper.py @@ -127,7 +127,6 @@ class Git: self.run = git_runner.run self.latest_tag = "" self.new_tag = "" - self.new_branch = "" self.branch = "" self.sha = "" self.sha_short = "" diff --git a/tests/ci/test_git.py b/tests/ci/test_git.py index 3aedd8a8dea..930edde368a 100644 --- a/tests/ci/test_git.py +++ b/tests/ci/test_git.py @@ -1,10 +1,10 @@ #!/usr/bin/env python -from unittest.mock import patch import os.path as p import unittest +from unittest.mock import patch -from git_helper import Git, Runner, CWD +from git_helper import CWD, Git, Runner class TestRunner(unittest.TestCase): @@ -45,7 +45,6 @@ class TestGit(unittest.TestCase): update_mock.assert_called_once() self.git.run("test") self.run_mock.assert_called_once() - self.git.new_branch = "NEW_BRANCH_NAME" self.git.new_tag = "v21.12.333.22222-stable" self.git.branch = "old_branch" self.git.sha = "" From 61f547b08d16ed19ae2efac32b68fff8d175876d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 3 Jun 2024 16:20:57 +0200 Subject: [PATCH 468/856] Support `-new` tags in git_helper, clean helper, add tests --- tests/ci/git_helper.py | 84 ++++++++++++++++++++++++++++++++---------- tests/ci/test_git.py | 49 +++++++++++++++++------- 2 files changed, 100 insertions(+), 33 deletions(-) diff --git a/tests/ci/git_helper.py b/tests/ci/git_helper.py index 3316ae7dfdf..6b66bc44d10 100644 --- a/tests/ci/git_helper.py +++ b/tests/ci/git_helper.py @@ -7,7 +7,7 @@ import os.path as p import re import subprocess import tempfile -from typing import Any, List, Optional +from typing import Any, List, Literal, Optional logger = logging.getLogger(__name__) @@ -15,7 +15,9 @@ logger = logging.getLogger(__name__) # \A and \Z match only start and end of the whole string RELEASE_BRANCH_REGEXP = r"\A\d+[.]\d+\Z" TAG_REGEXP = ( - r"\Av\d{2}[.][1-9]\d*[.][1-9]\d*[.][1-9]\d*-(testing|prestable|stable|lts)\Z" + r"\Av\d{2}" # First two digits of major part + r"([.][1-9]\d*){3}" # minor.patch.tweak parts + r"-(new|testing|prestable|stable|lts)\Z" # suffix with a version type ) SHA_REGEXP = re.compile(r"\A([0-9]|[a-f]){40}\Z") @@ -122,6 +124,25 @@ class Git: _tag_pattern = re.compile(TAG_REGEXP) def __init__(self, ignore_no_tags: bool = False): + """ + new_tag is used for special v24.1.1.1-new tags where the previous version is moved to the release branch + * 66666666666 Some another commit with version 24.8.1.xxxxx-testing, tweak is counted from new_tag = v24.8.1.1-new + | * 55555555555 (tag: v24.7.1.123123123-stable, branch: 24.7) tweak counted from new_tag = v24.7.1.1-new + |/ + * 44444444444 (tag: v24.8.1.1-new) + | * 33333333333 (tag: v24.6.1.123123123-stable, branch: 24.6) tweak counted from new_tag = v24.6.1.1-new + |/ + * 22222222222 (tag: v24.7.1.1-new) + | * 11111111111 (tag: v24.5.1.123123123-stable, branch: 24.5) tweak counted from new_tag = v24.4.1.2088-stable + |/ + * 00000000000 (tag: v24.6.1.1-new) + * 6d4b31322d1 (tag: v24.4.1.2088-stable) + * 2c5c589a882 (tag: v24.3.1.2672-lts) + * 891689a4150 (tag: v24.2.1.2248-stable) + * 5a024dfc093 (tag: v24.1.1.2048-stable) + * a2faa65b080 (tag: v23.12.1.1368-stable) + * 05bc8ef1e02 (tag: v23.11.1.2711-stable) + """ self.root = git_runner.cwd self._ignore_no_tags = ignore_no_tags self.run = git_runner.run @@ -130,8 +151,8 @@ class Git: self.branch = "" self.sha = "" self.sha_short = "" - self.description = "shallow-checkout" - self.commits_since_tag = 0 + self.commits_since_latest = 0 + self.commits_since_new = 0 self.update() def update(self): @@ -154,10 +175,20 @@ class Git: stderr = subprocess.DEVNULL if suppress_stderr else None self.latest_tag = self.run("git describe --tags --abbrev=0", stderr=stderr) # Format should be: {latest_tag}-{commits_since_tag}-g{sha_short} - self.description = self.run("git describe --tags --long") - self.commits_since_tag = int( + self.commits_since_latest = int( self.run(f"git rev-list {self.latest_tag}..HEAD --count") ) + if self.latest_tag.endswith("-new"): + # We won't change the behaviour of the the "latest_tag" + # So here we set "new_tag" to the previous tag in the graph, that will allow + # getting alternative "tweak" + self.new_tag = self.run( + f"git describe --tags --abbrev=0 --exclude='{self.latest_tag}'", + stderr=stderr, + ) + self.commits_since_new = int( + self.run(f"git rev-list {self.new_tag}..HEAD --count") + ) @staticmethod def check_tag(value: str) -> None: @@ -186,19 +217,34 @@ class Git: @property def tweak(self) -> int: - if not self.latest_tag.endswith("-testing"): + return self._tweak("latest") + + @property + def tweak_to_new(self) -> int: + return self._tweak("new") + + def _tweak(self, tag_type: Literal["latest", "new"]) -> int: + """Accepts latest or new as a tag_type and returns the tweak number to it""" + if tag_type == "latest": + commits = self.commits_since_latest + tag = self.latest_tag + else: + commits = self.commits_since_new + tag = self.new_tag + + if not tag.endswith("-testing"): # When we are on the tag, we still need to have tweak=1 to not # break cmake with versions like 12.13.14.0 - if not self.commits_since_tag: - # We are in a tagged commit. The tweak should match the - # current version's value - version = self.latest_tag.split("-", maxsplit=1)[0] - try: - return int(version.split(".")[-1]) - except ValueError: - # There are no tags, or a wrong tag. Return default - return TWEAK - return self.commits_since_tag + if commits: + return commits + # We are in a tagged commit or shallow checkout. The tweak should match the + # current version's value + version = tag.split("-", maxsplit=1)[0] + try: + return int(version.split(".")[-1]) + except ValueError: + # There are no tags (shallow checkout), or a wrong tag. Return default + return TWEAK - version = self.latest_tag.split("-", maxsplit=1)[0] - return int(version.split(".")[-1]) + self.commits_since_tag + version = tag.split("-", maxsplit=1)[0] + return int(version.split(".")[-1]) + commits diff --git a/tests/ci/test_git.py b/tests/ci/test_git.py index 930edde368a..60cd95b6869 100644 --- a/tests/ci/test_git.py +++ b/tests/ci/test_git.py @@ -2,9 +2,10 @@ import os.path as p import unittest +from dataclasses import dataclass from unittest.mock import patch -from git_helper import CWD, Git, Runner +from git_helper import CWD, Git, Runner, git_runner class TestRunner(unittest.TestCase): @@ -35,8 +36,10 @@ class TestRunner(unittest.TestCase): class TestGit(unittest.TestCase): def setUp(self): """we use dummy git object""" + # get the git_runner's cwd to set it properly before the Runner is patched + _ = git_runner.cwd run_patcher = patch("git_helper.Runner.run", return_value="") - self.run_mock = run_patcher.start() + run_mock = run_patcher.start() self.addCleanup(run_patcher.stop) update_patcher = patch("git_helper.Git.update") update_mock = update_patcher.start() @@ -44,14 +47,13 @@ class TestGit(unittest.TestCase): self.git = Git() update_mock.assert_called_once() self.git.run("test") - self.run_mock.assert_called_once() - self.git.new_tag = "v21.12.333.22222-stable" + run_mock.assert_called_once() self.git.branch = "old_branch" self.git.sha = "" self.git.sha_short = "" self.git.latest_tag = "" - self.git.description = "" - self.git.commits_since_tag = 0 + self.git.commits_since_latest = 0 + self.git.commits_since_new = 0 def test_tags(self): self.git.new_tag = "v21.12.333.22222-stable" @@ -70,11 +72,30 @@ class TestGit(unittest.TestCase): setattr(self.git, tag_attr, tag) def test_tweak(self): - self.git.commits_since_tag = 0 - self.assertEqual(self.git.tweak, 1) - self.git.commits_since_tag = 2 - self.assertEqual(self.git.tweak, 2) - self.git.latest_tag = "v21.12.333.22222-testing" - self.assertEqual(self.git.tweak, 22224) - self.git.commits_since_tag = 0 - self.assertEqual(self.git.tweak, 22222) + # tweak for the latest tag + @dataclass + class TestCase: + tag: str + commits: int + tweak: int + + cases = ( + TestCase("", 0, 1), + TestCase("", 2, 2), + TestCase("v21.12.333.22222-stable", 0, 22222), + TestCase("v21.12.333.22222-stable", 2, 2), + TestCase("v21.12.333.22222-testing", 0, 22222), + TestCase("v21.12.333.22222-testing", 2, 22224), + ) + for tag, commits, tweak in ( + ("latest_tag", "commits_since_latest", "tweak"), + ("new_tag", "commits_since_new", "tweak_to_new"), + ): + for tc in cases: + setattr(self.git, tag, tc.tag) + setattr(self.git, commits, tc.commits) + self.assertEqual( + getattr(self.git, tweak), + tc.tweak, + f"Wrong tweak for tag {tc.tag} and commits {tc.commits} of {tag}", + ) From aa2a08c1573618842764136e8d59f3e72b0724a1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 3 Jun 2024 16:47:52 +0200 Subject: [PATCH 469/856] Some improvements for CHVersion.tweak type --- tests/ci/version_helper.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 800bfcf52c3..825feb161ad 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -22,7 +22,7 @@ VERSIONS = Dict[str, Union[int, str]] VERSIONS_TEMPLATE = """# 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 {revision}) SET(VERSION_MAJOR {major}) @@ -47,7 +47,7 @@ class ClickHouseVersion: patch: Union[int, str], revision: Union[int, str], git: Optional[Git], - tweak: Optional[str] = None, + tweak: Optional[Union[int, str]] = None, ): self._major = int(major) self._minor = int(minor) @@ -95,7 +95,7 @@ class ClickHouseVersion: if self._git is not None: self._git.update() return ClickHouseVersion( - self.major, self.minor, self.patch, self.revision, self._git, "1" + self.major, self.minor, self.patch, self.revision, self._git, 1 ) @property @@ -172,7 +172,7 @@ class ClickHouseVersion: self.patch, self.revision, self._git, - str(self.tweak), + self.tweak, ) try: copy.with_description(self.description) From da2c3b7332fd859e5beeba2d700fbb008f1f20ee Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 3 Jun 2024 17:12:12 +0200 Subject: [PATCH 470/856] Support `*-new` tags in version_helper.py --- tests/ci/test_version.py | 56 +++++++++++++++++++++++ tests/ci/tests/autogenerated_versions.txt | 12 +++++ tests/ci/version_helper.py | 28 ++++++++++-- 3 files changed, 93 insertions(+), 3 deletions(-) create mode 100644 tests/ci/tests/autogenerated_versions.txt diff --git a/tests/ci/test_version.py b/tests/ci/test_version.py index 978edcc093e..011018be648 100644 --- a/tests/ci/test_version.py +++ b/tests/ci/test_version.py @@ -2,8 +2,12 @@ import unittest from argparse import ArgumentTypeError +from dataclasses import dataclass import version_helper as vh +from git_helper import Git + +CHV = vh.ClickHouseVersion class TestFunctions(unittest.TestCase): @@ -32,3 +36,55 @@ class TestFunctions(unittest.TestCase): for error_case in error_cases: with self.assertRaises(ArgumentTypeError): version = vh.version_arg(error_case[0]) + + def test_get_version_from_repo(self): + @dataclass + class TestCase: + latest_tag: str + commits_since_latest: int + new_tag: str + commits_since_new: int + expected: CHV + + cases = ( + TestCase( + "v24.6.1.1-new", + 15, + "v24.4.1.2088-stable", + 415, + CHV(24, 5, 1, 54487, None, 415), + ), + TestCase( + "v24.6.1.1-testing", + 15, + "v24.4.1.2088-stable", + 415, + CHV(24, 5, 1, 54487, None, 16), + ), + TestCase( + "v24.6.1.1-stable", + 15, + "v24.4.1.2088-stable", + 415, + CHV(24, 5, 1, 54487, None, 15), + ), + TestCase( + "v24.5.1.1-stable", + 15, + "v24.4.1.2088-stable", + 415, + CHV(24, 5, 1, 54487, None, 15), + ), + ) + git = Git(True) + for tc in cases: + git.latest_tag = tc.latest_tag + git.commits_since_latest = tc.commits_since_latest + git.new_tag = tc.new_tag + git.commits_since_new = tc.commits_since_new + self.assertEqual( + vh.get_version_from_repo( + "tests/ci/tests/autogenerated_versions.txt", git + ), + tc.expected, + ) diff --git a/tests/ci/tests/autogenerated_versions.txt b/tests/ci/tests/autogenerated_versions.txt new file mode 100644 index 00000000000..10028bf50c8 --- /dev/null +++ b/tests/ci/tests/autogenerated_versions.txt @@ -0,0 +1,12 @@ +# This variables autochanged by tests/ci/version_helper.py: + +# NOTE: 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_MAJOR 24) +SET(VERSION_MINOR 5) +SET(VERSION_PATCH 1) +SET(VERSION_GITHASH 70a1d3a63d47f0be077d67b8deb907230fc7cfb0) +SET(VERSION_DESCRIBE v24.5.1.1-testing) +SET(VERSION_STRING 24.5.1.1) +# end of autochange diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 825feb161ad..0543bf3de5d 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -114,6 +114,10 @@ class ClickHouseVersion: def tweak(self) -> int: return self._tweak + @tweak.setter + def tweak(self, tweak: int) -> None: + self._tweak = tweak + @property def revision(self) -> int: return self._revision @@ -190,7 +194,9 @@ class ClickHouseVersion: and self.tweak == other.tweak ) - def __lt__(self, other: "ClickHouseVersion") -> bool: + def __lt__(self, other: Any) -> bool: + if not isinstance(self, type(other)): + return NotImplemented for part in ("major", "minor", "patch", "tweak"): if getattr(self, part) < getattr(other, part): return True @@ -220,10 +226,11 @@ ClickHouseVersions = List[ClickHouseVersion] class VersionType: LTS = "lts" + NEW = "new" PRESTABLE = "prestable" STABLE = "stable" TESTING = "testing" - VALID = (TESTING, PRESTABLE, STABLE, LTS) + VALID = (NEW, TESTING, PRESTABLE, STABLE, LTS) def validate_version(version: str) -> None: @@ -263,14 +270,29 @@ def get_version_from_repo( versions_path: str = FILE_WITH_VERSION_PATH, git: Optional[Git] = None, ) -> ClickHouseVersion: + """Get a ClickHouseVersion from FILE_WITH_VERSION_PATH. When the `git` parameter is + present, a proper `tweak` version part is calculated for case if the latest tag has + a `new` type and greater than version in `FILE_WITH_VERSION_PATH`""" versions = read_versions(versions_path) - return ClickHouseVersion( + cmake_version = ClickHouseVersion( versions["major"], versions["minor"], versions["patch"], versions["revision"], git, ) + # Since 24.5 we have tags like v24.6.1.1-new, and we must check if the release + # branch already has it's own commit. It's necessary for a proper tweak version + if git is not None and git.latest_tag: + version_from_tag = get_version_from_tag(git.latest_tag) + if ( + version_from_tag.description == VersionType.NEW + and cmake_version < version_from_tag + ): + # We are in a new release branch without existing release. + # We should change the tweak version to a `tweak_to_new` + cmake_version.tweak = git.tweak_to_new + return cmake_version def get_version_from_string( From 569a03d30daa9e34cd6b5a1901c94714d4823e81 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jun 2024 19:27:23 +0200 Subject: [PATCH 471/856] Ignore files in tests/ci/tmp --- tests/ci/tmp/.gitignore | 1 + 1 file changed, 1 insertion(+) create mode 100644 tests/ci/tmp/.gitignore diff --git a/tests/ci/tmp/.gitignore b/tests/ci/tmp/.gitignore new file mode 100644 index 00000000000..72e8ffc0db8 --- /dev/null +++ b/tests/ci/tmp/.gitignore @@ -0,0 +1 @@ +* From 76842d4e1be2084e8090de7911f1884b16e58ee1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jun 2024 19:55:11 +0200 Subject: [PATCH 472/856] Use pathlib.Path in version_helper --- tests/ci/test_version.py | 3 ++- tests/ci/version_helper.py | 52 ++++++++++++++++++-------------------- 2 files changed, 26 insertions(+), 29 deletions(-) diff --git a/tests/ci/test_version.py b/tests/ci/test_version.py index 011018be648..c4f12091ec0 100644 --- a/tests/ci/test_version.py +++ b/tests/ci/test_version.py @@ -3,6 +3,7 @@ import unittest from argparse import ArgumentTypeError from dataclasses import dataclass +from pathlib import Path import version_helper as vh from git_helper import Git @@ -84,7 +85,7 @@ class TestFunctions(unittest.TestCase): git.commits_since_new = tc.commits_since_new self.assertEqual( vh.get_version_from_repo( - "tests/ci/tests/autogenerated_versions.txt", git + Path("tests/ci/tests/autogenerated_versions.txt"), git ), tc.expected, ) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 0543bf3de5d..50263f6ebb6 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -1,7 +1,7 @@ #!/usr/bin/env python3 import logging -import os.path as p from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser, ArgumentTypeError +from pathlib import Path from typing import Any, Dict, Iterable, List, Literal, Optional, Set, Tuple, Union from git_helper import TWEAK, Git, get_tags, git_runner, removeprefix @@ -241,33 +241,31 @@ def validate_version(version: str) -> None: int(part) -def get_abs_path(path: str) -> str: - return p.abspath(p.join(git_runner.cwd, path)) +def get_abs_path(path: Union[Path, str]) -> Path: + return (Path(git_runner.cwd) / path).absolute() -def read_versions(versions_path: str = FILE_WITH_VERSION_PATH) -> VERSIONS: +def read_versions(versions_path: Union[Path, str] = FILE_WITH_VERSION_PATH) -> VERSIONS: versions = {} - path_to_file = get_abs_path(versions_path) - with open(path_to_file, "r", encoding="utf-8") as f: - for line in f: - line = line.strip() - if not line.startswith("SET("): - continue + for line in get_abs_path(versions_path).read_text(encoding="utf-8").splitlines(): + line = line.strip() + if not line.startswith("SET("): + continue - value = 0 # type: Union[int, str] - name, value = line[4:-1].split(maxsplit=1) - name = removeprefix(name, "VERSION_").lower() - try: - value = int(value) - except ValueError: - pass - versions[name] = value + value = 0 # type: Union[int, str] + name, value = line[4:-1].split(maxsplit=1) + name = removeprefix(name, "VERSION_").lower() + try: + value = int(value) + except ValueError: + pass + versions[name] = value return versions def get_version_from_repo( - versions_path: str = FILE_WITH_VERSION_PATH, + versions_path: Union[Path, str] = FILE_WITH_VERSION_PATH, git: Optional[Git] = None, ) -> ClickHouseVersion: """Get a ClickHouseVersion from FILE_WITH_VERSION_PATH. When the `git` parameter is @@ -372,15 +370,15 @@ def get_supported_versions( def update_cmake_version( version: ClickHouseVersion, - versions_path: str = FILE_WITH_VERSION_PATH, + versions_path: Union[Path, str] = FILE_WITH_VERSION_PATH, ) -> None: - path_to_file = get_abs_path(versions_path) - with open(path_to_file, "w", encoding="utf-8") as f: - f.write(VERSIONS_TEMPLATE.format_map(version.as_dict())) + get_abs_path(versions_path).write_text( + VERSIONS_TEMPLATE.format_map(version.as_dict()), encoding="utf-8" + ) def update_contributors( - relative_contributors_path: str = GENERATED_CONTRIBUTORS, + relative_contributors_path: Union[Path, str] = GENERATED_CONTRIBUTORS, force: bool = False, raise_error: bool = False, ) -> None: @@ -400,13 +398,11 @@ def update_contributors( ) contributors = [f' "{c}",' for c in contributors] - executer = p.relpath(p.realpath(__file__), git_runner.cwd) + executer = Path(__file__).relative_to(git_runner.cwd) content = CONTRIBUTORS_TEMPLATE.format( executer=executer, contributors="\n".join(contributors) ) - contributors_path = get_abs_path(relative_contributors_path) - with open(contributors_path, "w", encoding="utf-8") as cfd: - cfd.write(content) + get_abs_path(relative_contributors_path).write_text(content, encoding="utf-8") def update_version_local(version, version_type="testing"): From c8e00d987e1c27746980d00c4b15a4dc78859856 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jun 2024 00:40:24 +0200 Subject: [PATCH 473/856] Changes in version_helper.py should change the builds --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 412a55b8534..6ab1eb8bac4 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -280,6 +280,7 @@ builds_job_config = JobConfig( "./packages", "./docker/packager/packager", "./rust", + "./tests/ci/version_helper.py", # FIXME: This is a WA to rebuild the CH and recreate the Performance.tar.zst artifact # when there are changes in performance test scripts. # Due to the current design of the perf test we need to rebuild CH when the performance test changes, From 0cd1cf6533ad9e9571e0f64e4f783a6176eeff23 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jun 2024 13:36:34 +0200 Subject: [PATCH 474/856] Do not require "Ready for release" status for new release --- tests/ci/release.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 2ae82177c67..1f426f9ab32 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -180,7 +180,8 @@ class Release: ) raise - self.check_commit_release_ready() + if self.release_type == self.PATCH: + self.check_commit_release_ready() def do( self, check_dirty: bool, check_run_from_master: bool, check_branch: bool From d506220f8ceb3bc0e4530495c88082ffde9558b3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jun 2024 13:37:37 +0200 Subject: [PATCH 475/856] Fix possible issues in Release._checkout --- tests/ci/release.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 1f426f9ab32..cc8c41d7442 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -459,10 +459,10 @@ class Release: @contextmanager def _checkout(self, ref: str, with_checkout_back: bool = False) -> Iterator[None]: + self._git.update() orig_ref = self._git.branch or self._git.sha - need_rollback = False + rollback_cmd = "" if ref not in (self._git.branch, self._git.sha): - need_rollback = True self.run(f"git checkout {ref}") # checkout is not put into rollback_stack intentionally rollback_cmd = f"git checkout {orig_ref}" @@ -475,7 +475,7 @@ class Release: self.run(f"git reset --hard; git checkout -f {orig_ref}") raise # Normal flow when we need to checkout back - if with_checkout_back and need_rollback: + if with_checkout_back and rollback_cmd: self.run(rollback_cmd) @contextmanager From 43b0866ed623b9f6e92afceeb6e6f71b6e523ae4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jun 2024 13:38:57 +0200 Subject: [PATCH 476/856] Make Release._create_tag more flexible --- tests/ci/release.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index cc8c41d7442..d8599c66310 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -511,9 +511,9 @@ class Release: @contextmanager def _create_gh_release(self, as_prerelease: bool) -> Iterator[None]: - with self._create_tag(): + tag = self.release_version.describe + with self._create_tag(tag, self.release_commit): # Preserve tag if version is changed - tag = self.release_version.describe prerelease = "" if as_prerelease: prerelease = "--prerelease" @@ -535,11 +535,12 @@ class Release: raise @contextmanager - def _create_tag(self): - tag = self.release_version.describe + def _create_tag( + self, tag: str, commit: str, tag_message: str = "" + ) -> Iterator[None]: + tag_message = tag_message or "Release {tag}" self.run( - f"git tag -a -m 'Release {tag}' '{tag}' {self.release_commit}", - dry_run=self.dry_run, + f"git tag -a -m '{tag_message}' '{tag}' {commit}", dry_run=self.dry_run ) rollback_cmd = f"{self.dry_run_prefix}git tag -d '{tag}'" self._rollback_stack.append(rollback_cmd) From 3d07f729276dda4a6b66b855c0a621efbcc3d44b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jun 2024 13:39:42 +0200 Subject: [PATCH 477/856] Create tags v{version}-new for new releases --- tests/ci/release.py | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index d8599c66310..f18050e37d4 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -94,6 +94,7 @@ class Release: self._version = get_version_from_repo(git=self._git) self.release_version = self.version self._release_branch = "" + self._version_new_tag = None # type: Optional[ClickHouseVersion] self._rollback_stack = [] # type: List[str] def run( @@ -329,10 +330,16 @@ class Release: self.check_no_tags_after() # Create release branch self.read_version() - with self._create_branch(self.release_branch, self.release_commit): - with self._checkout(self.release_branch, True): - with self._bump_release_branch(): - yield + assert self._version_new_tag is not None + with self._create_tag( + self._version_new_tag.describe, + self.release_commit, + f"Initial commit for release {self._version_new_tag.major}.{self._version_new_tag.minor}", + ): + with self._create_branch(self.release_branch, self.release_commit): + with self._checkout(self.release_branch, True): + with self._bump_release_branch(): + yield @contextmanager def patch_release(self): @@ -445,6 +452,11 @@ class Release: self.version.with_description(VersionType.TESTING) self._update_cmake_contributors(self.version) self._commit_cmake_contributors(self.version) + # Create a version-new tag + self._version_new_tag = self.version.copy() + self._version_new_tag.tweak = 1 + self._version_new_tag.with_description(VersionType.NEW) + with self._push(helper_branch): body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") # The following command is rolled back by deleting branch in self._push From d984a80b57a6fc54c85742d52b6f82fb93b1d320 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 7 Jun 2024 12:44:08 +0200 Subject: [PATCH 478/856] Create tag in dry-run to check the order of commands --- tests/ci/release.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index f18050e37d4..7490ce0a373 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -551,10 +551,9 @@ class Release: self, tag: str, commit: str, tag_message: str = "" ) -> Iterator[None]: tag_message = tag_message or "Release {tag}" - self.run( - f"git tag -a -m '{tag_message}' '{tag}' {commit}", dry_run=self.dry_run - ) - rollback_cmd = f"{self.dry_run_prefix}git tag -d '{tag}'" + # Create tag even in dry-run + self.run(f"git tag -a -m '{tag_message}' '{tag}' {commit}") + rollback_cmd = f"git tag -d '{tag}'" self._rollback_stack.append(rollback_cmd) try: with self._push(tag): From 4cb53e951369f24a6b2fa0c21e7c940f0733f82e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 10 Jun 2024 14:46:39 +0200 Subject: [PATCH 479/856] Revert "Revert "Fix duplicating Delete events in blob_storage_log"" --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 +++-- src/Interpreters/SystemLog.cpp | 7 ++++++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index ae719f5cde4..afc13251f5b 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -382,6 +382,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { std::vector current_chunk; String keys; + size_t first_position = current_position; for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position) { Aws::S3::Model::ObjectIdentifier obj; @@ -407,9 +408,9 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError(); auto time_now = std::chrono::system_clock::now(); - for (const auto & object : objects) + for (size_t i = first_position; i < current_position; ++i) blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, - uri.bucket, object.remote_path, object.local_path, object.bytes_size, + uri.bucket, objects[i].remote_path, objects[i].local_path, objects[i].bytes_size, outcome_error, time_now); } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 5e0ce2cb0de..3b25deeb59d 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -504,6 +504,10 @@ void SystemLog::flushImpl(const std::vector & to_flush, Block block(std::move(log_element_columns)); MutableColumns columns = block.mutateColumns(); + + for (auto & column : columns) + column->reserve(to_flush.size()); + for (const auto & elem : to_flush) elem.appendToBlock(columns); @@ -532,7 +536,8 @@ void SystemLog::flushImpl(const std::vector & to_flush, } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to flush system log {} with {} entries up to offset {}", + table_id.getNameForLogs(), to_flush.size(), to_flush_end)); } queue->confirm(to_flush_end); From 8af077f3d3cd891768b310b59b42696691578245 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 12:53:06 +0000 Subject: [PATCH 480/856] Update build descriptions --- contrib/abseil-cpp-cmake/CMakeLists.txt | 258 ++++++++++++++++++++---- 1 file changed, 218 insertions(+), 40 deletions(-) diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index d026a7c78bc..a9e79be4f09 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -1,6 +1,8 @@ set(ABSL_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") set(ABSL_COMMON_INCLUDE_DIRS "${ABSL_ROOT_DIR}") +# This is a minimized version of the function definition in CMake/AbseilHelpers.cmake + # # Copyright 2017 The Abseil Authors. # @@ -16,7 +18,6 @@ set(ABSL_COMMON_INCLUDE_DIRS "${ABSL_ROOT_DIR}") # See the License for the specific language governing permissions and # limitations under the License. # - function(absl_cc_library) cmake_parse_arguments(ABSL_CC_LIB "DISABLE_INSTALL;PUBLIC;TESTONLY" @@ -76,6 +77,12 @@ function(absl_cc_library) add_library(absl::${ABSL_CC_LIB_NAME} ALIAS ${_NAME}) endfunction() +# The following definitions are an amalgamation of the CMakeLists.txt files in absl/*/ +# To refresh them when upgrading to a new version: +# - copy them over from upstream +# - remove calls of 'absl_cc_test' +# - remove calls of `absl_cc_library` that contain `TESTONLY` +# - append '${DIR}' to the file definitions set(DIR ${ABSL_ROOT_DIR}/absl/algorithm) @@ -102,12 +109,12 @@ absl_cc_library( absl::algorithm absl::core_headers absl::meta + absl::nullability PUBLIC ) set(DIR ${ABSL_ROOT_DIR}/absl/base) -# Internal-only target, do not depend on directly. absl_cc_library( NAME atomic_hook @@ -146,6 +153,18 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} ) +absl_cc_library( + NAME + no_destructor + HDRS + "${DIR}/no_destructor.h" + DEPS + absl::config + absl::nullability + COPTS + ${ABSL_DEFAULT_COPTS} +) + absl_cc_library( NAME nullability @@ -305,6 +324,8 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} LINKOPTS ${ABSL_DEFAULT_LINKOPTS} + $<$:-lrt> + $<$:-ladvapi32> DEPS absl::atomic_hook absl::base_internal @@ -312,6 +333,7 @@ absl_cc_library( absl::core_headers absl::dynamic_annotations absl::log_severity + absl::nullability absl::raw_logging_internal absl::spinlock_wait absl::type_traits @@ -357,6 +379,7 @@ absl_cc_library( absl::base absl::config absl::core_headers + absl::nullability PUBLIC ) @@ -467,10 +490,11 @@ absl_cc_library( LINKOPTS ${ABSL_DEFAULT_LINKOPTS} DEPS - absl::container_common absl::common_policy_traits absl::compare absl::compressed_tuple + absl::config + absl::container_common absl::container_memory absl::cord absl::core_headers @@ -480,7 +504,6 @@ absl_cc_library( absl::strings absl::throw_delegate absl::type_traits - absl::utility ) # Internal-only target, do not depend on directly. @@ -523,7 +546,9 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::base_internal absl::compressed_tuple + absl::config absl::core_headers absl::memory absl::span @@ -548,18 +573,6 @@ absl_cc_library( PUBLIC ) -# Internal-only target, do not depend on directly. -absl_cc_library( - NAME - counting_allocator - HDRS - "${DIR}/internal/counting_allocator.h" - COPTS - ${ABSL_DEFAULT_COPTS} - DEPS - absl::config -) - absl_cc_library( NAME flat_hash_map @@ -570,7 +583,7 @@ absl_cc_library( DEPS absl::container_memory absl::core_headers - absl::hash_function_defaults + absl::hash_container_defaults absl::raw_hash_map absl::algorithm_container absl::memory @@ -586,7 +599,7 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} DEPS absl::container_memory - absl::hash_function_defaults + absl::hash_container_defaults absl::raw_hash_set absl::algorithm_container absl::core_headers @@ -604,7 +617,7 @@ absl_cc_library( DEPS absl::container_memory absl::core_headers - absl::hash_function_defaults + absl::hash_container_defaults absl::node_slot_policy absl::raw_hash_map absl::algorithm_container @@ -620,8 +633,9 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::container_memory absl::core_headers - absl::hash_function_defaults + absl::hash_container_defaults absl::node_slot_policy absl::raw_hash_set absl::algorithm_container @@ -629,6 +643,19 @@ absl_cc_library( PUBLIC ) +absl_cc_library( + NAME + hash_container_defaults + HDRS + "${DIR}/hash_container_defaults.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::hash_function_defaults + PUBLIC +) + # Internal-only target, do not depend on directly. absl_cc_library( NAME @@ -655,9 +682,11 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} DEPS absl::config + absl::container_common absl::cord absl::hash absl::strings + absl::type_traits PUBLIC ) @@ -703,6 +732,7 @@ absl_cc_library( absl::base absl::config absl::exponential_biased + absl::no_destructor absl::raw_logging_internal absl::sample_recorder absl::synchronization @@ -756,7 +786,9 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::config absl::container_memory + absl::core_headers absl::raw_hash_set absl::throw_delegate PUBLIC @@ -817,6 +849,7 @@ absl_cc_library( DEPS absl::config absl::core_headers + absl::debugging_internal absl::meta absl::strings absl::span @@ -931,6 +964,7 @@ absl_cc_library( absl::crc32c absl::config absl::strings + absl::no_destructor ) set(DIR ${ABSL_ROOT_DIR}/absl/debugging) @@ -954,6 +988,8 @@ absl_cc_library( "${DIR}/stacktrace.cc" COPTS ${ABSL_DEFAULT_COPTS} + LINKOPTS + $<$:${EXECINFO_LIBRARY}> DEPS absl::debugging_internal absl::config @@ -980,6 +1016,7 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} LINKOPTS ${ABSL_DEFAULT_LINKOPTS} + $<$:-ldbghelp> DEPS absl::debugging_internal absl::demangle_internal @@ -1254,6 +1291,7 @@ absl_cc_library( absl::strings absl::synchronization absl::flat_hash_map + absl::no_destructor ) # Internal-only target, do not depend on directly. @@ -1298,7 +1336,6 @@ absl_cc_library( absl::flags_config absl::flags_internal absl::flags_reflection - absl::base absl::core_headers absl::strings ) @@ -1378,6 +1415,9 @@ absl_cc_library( absl::synchronization ) +############################################################################ +# Unit tests in alphabetical order. + set(DIR ${ABSL_ROOT_DIR}/absl/functional) absl_cc_library( @@ -1430,6 +1470,18 @@ absl_cc_library( PUBLIC ) +absl_cc_library( + NAME + overload + HDRS + "${DIR}/overload.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::meta + PUBLIC +) + set(DIR ${ABSL_ROOT_DIR}/absl/hash) absl_cc_library( @@ -1639,6 +1691,7 @@ absl_cc_library( absl::log_internal_conditions absl::log_internal_message absl::log_internal_strip + absl::absl_vlog_is_on ) absl_cc_library( @@ -1720,6 +1773,7 @@ absl_cc_library( absl::log_entry absl::log_severity absl::log_sink + absl::no_destructor absl::raw_logging_internal absl::synchronization absl::span @@ -1770,6 +1824,7 @@ absl_cc_library( LINKOPTS ${ABSL_DEFAULT_LINKOPTS} DEPS + absl::core_headers absl::log_internal_message absl::log_internal_nullstream absl::log_severity @@ -1875,6 +1930,11 @@ absl_cc_library( PUBLIC ) +# Warning: Many linkers will strip the contents of this library because its +# symbols are only used in a global constructor. A workaround is for clients +# to link this using $ instead of +# the plain absl::log_flags. +# TODO(b/320467376): Implement the equivalent of Bazel's alwayslink=True. absl_cc_library( NAME log_flags @@ -1896,6 +1956,7 @@ absl_cc_library( absl::flags absl::flags_marshalling absl::strings + absl::vlog_config_internal PUBLIC ) @@ -1918,6 +1979,7 @@ absl_cc_library( absl::log_severity absl::raw_logging_internal absl::strings + absl::vlog_config_internal ) absl_cc_library( @@ -1951,6 +2013,7 @@ absl_cc_library( ${ABSL_DEFAULT_LINKOPTS} DEPS absl::log_internal_log_impl + absl::vlog_is_on PUBLIC ) @@ -2063,21 +2126,75 @@ absl_cc_library( ) absl_cc_library( - NAME - log_internal_fnmatch - SRCS - "${DIR}/internal/fnmatch.cc" - HDRS - "${DIR}/internal/fnmatch.h" - COPTS - ${ABSL_DEFAULT_COPTS} - LINKOPTS - ${ABSL_DEFAULT_LINKOPTS} - DEPS - absl::config - absl::strings + NAME + vlog_config_internal + SRCS + "${DIR}/internal/vlog_config.cc" + HDRS + "${DIR}/internal/vlog_config.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::log_internal_fnmatch + absl::memory + absl::no_destructor + absl::strings + absl::synchronization + absl::optional ) +absl_cc_library( + NAME + absl_vlog_is_on + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + HDRS + "${DIR}/absl_vlog_is_on.h" + DEPS + absl::vlog_config_internal + absl::config + absl::core_headers + absl::strings +) + +absl_cc_library( + NAME + vlog_is_on + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + HDRS + "${DIR}/vlog_is_on.h" + DEPS + absl::absl_vlog_is_on +) + +absl_cc_library( + NAME + log_internal_fnmatch + SRCS + "${DIR}/internal/fnmatch.cc" + HDRS + "${DIR}/internal/fnmatch.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::strings +) + +# Test targets + set(DIR ${ABSL_ROOT_DIR}/absl/memory) absl_cc_library( @@ -2146,6 +2263,7 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::compare absl::config absl::core_headers absl::bits @@ -2175,6 +2293,8 @@ absl_cc_library( PUBLIC ) +set(DIR ${ABSL_ROOT_DIR}/absl/profiling) + absl_cc_library( NAME sample_recorder @@ -2187,8 +2307,6 @@ absl_cc_library( absl::synchronization ) -set(DIR ${ABSL_ROOT_DIR}/absl/profiling) - absl_cc_library( NAME exponential_biased @@ -2264,6 +2382,7 @@ absl_cc_library( LINKOPTS ${ABSL_DEFAULT_LINKOPTS} DEPS + absl::config absl::fast_type_id absl::optional ) @@ -2335,11 +2454,13 @@ absl_cc_library( DEPS absl::config absl::inlined_vector + absl::nullability absl::random_internal_pool_urbg absl::random_internal_salted_seed_seq absl::random_internal_seed_material absl::random_seed_gen_exception absl::span + absl::string_view ) # Internal-only target, do not depend on directly. @@ -2398,6 +2519,7 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} LINKOPTS ${ABSL_DEFAULT_LINKOPTS} + $<$:-lbcrypt> DEPS absl::core_headers absl::optional @@ -2657,6 +2779,29 @@ absl_cc_library( absl::config ) +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_distribution_test_util + SRCS + "${DIR}/internal/chi_square.cc" + "${DIR}/internal/distribution_test_util.cc" + HDRS + "${DIR}/internal/chi_square.h" + "${DIR}/internal/distribution_test_util.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::raw_logging_internal + absl::strings + absl::str_format + absl::span +) + # Internal-only target, do not depend on directly. absl_cc_library( NAME @@ -2698,6 +2843,8 @@ absl_cc_library( absl::function_ref absl::inlined_vector absl::memory + absl::no_destructor + absl::nullability absl::optional absl::raw_logging_internal absl::span @@ -2723,8 +2870,11 @@ absl_cc_library( absl::base absl::config absl::core_headers + absl::has_ostream_operator + absl::nullability absl::raw_logging_internal absl::status + absl::str_format absl::strings absl::type_traits absl::utility @@ -2747,6 +2897,7 @@ absl_cc_library( absl::base absl::config absl::core_headers + absl::nullability absl::throw_delegate PUBLIC ) @@ -2761,6 +2912,7 @@ absl_cc_library( "${DIR}/has_absl_stringify.h" "${DIR}/internal/damerau_levenshtein_distance.h" "${DIR}/internal/string_constant.h" + "${DIR}/internal/has_absl_stringify.h" "${DIR}/match.h" "${DIR}/numbers.h" "${DIR}/str_cat.h" @@ -2804,6 +2956,7 @@ absl_cc_library( absl::endian absl::int128 absl::memory + absl::nullability absl::raw_logging_internal absl::throw_delegate absl::type_traits @@ -2823,6 +2976,18 @@ absl_cc_library( PUBLIC ) +absl_cc_library( + NAME + has_ostream_operator + HDRS + "${DIR}/has_ostream_operator.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) + # Internal-only target, do not depend on directly. absl_cc_library( NAME @@ -2850,11 +3015,16 @@ absl_cc_library( NAME str_format HDRS - "${DIR}/str_format.h" + "str_format.h" COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::config + absl::core_headers + absl::nullability + absl::span absl::str_format_internal + absl::string_view PUBLIC ) @@ -2885,6 +3055,7 @@ absl_cc_library( absl::strings absl::config absl::core_headers + absl::fixed_array absl::inlined_vector absl::numeric_representation absl::type_traits @@ -2988,6 +3159,7 @@ absl_cc_library( DEPS absl::base absl::config + absl::no_destructor absl::raw_logging_internal absl::synchronization ) @@ -3078,6 +3250,7 @@ absl_cc_library( absl::endian absl::function_ref absl::inlined_vector + absl::nullability absl::optional absl::raw_logging_internal absl::span @@ -3245,6 +3418,8 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} DEPS Threads::Threads + # TODO(#1495): Use $ once our + # minimum CMake version >= 3.24 $<$:-Wl,-framework,CoreFoundation> ) @@ -3254,7 +3429,7 @@ absl_cc_library( NAME any HDRS - "${DIR}/any.h" + "any.h" COPTS ${ABSL_DEFAULT_COPTS} DEPS @@ -3285,8 +3460,8 @@ absl_cc_library( NAME bad_any_cast_impl SRCS - "${DIR}/bad_any_cast.h" - "${DIR}/bad_any_cast.cc" + "${DIR}/bad_any_cast.h" + "${DIR}/bad_any_cast.cc" COPTS ${ABSL_DEFAULT_COPTS} DEPS @@ -3306,6 +3481,7 @@ absl_cc_library( DEPS absl::algorithm absl::core_headers + absl::nullability absl::throw_delegate absl::type_traits PUBLIC @@ -3326,6 +3502,7 @@ absl_cc_library( absl::config absl::core_headers absl::memory + absl::nullability absl::type_traits absl::utility PUBLIC @@ -3388,6 +3565,7 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::config absl::core_headers absl::type_traits PUBLIC From 598219c57dac54d9000d2fe338b523007e13be21 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 12:56:21 +0000 Subject: [PATCH 481/856] Minor update --- contrib/abseil-cpp-cmake/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index a9e79be4f09..4137547b736 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -3015,7 +3015,7 @@ absl_cc_library( NAME str_format HDRS - "str_format.h" + "${DIR}/str_format.h" COPTS ${ABSL_DEFAULT_COPTS} DEPS @@ -3429,7 +3429,7 @@ absl_cc_library( NAME any HDRS - "any.h" + "${DIR}/any.h" COPTS ${ABSL_DEFAULT_COPTS} DEPS From 1f17ddc6fe35be95736b448ebb3b73123c034196 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 13:06:44 +0000 Subject: [PATCH 482/856] Update .clang-tidy --- .clang-tidy | 1 - 1 file changed, 1 deletion(-) diff --git a/.clang-tidy b/.clang-tidy index 896052915f7..de19059d09e 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -37,7 +37,6 @@ Checks: [ '-cert-oop54-cpp', '-cert-oop57-cpp', - '-clang-analyzer-optin.core.EnumCastOutOfRange', # https://github.com/abseil/abseil-cpp/issues/1667 '-clang-analyzer-optin.performance.Padding', '-clang-analyzer-unix.Malloc', From e849b21cbaafa5171daad76c6b05d606d81fb0d4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 10 Jun 2024 15:15:44 +0200 Subject: [PATCH 483/856] Increase even more --- src/Common/GWPAsan.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 088f34fa6ae..488f8e2c5dc 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -57,7 +57,7 @@ static bool guarded_alloc_initialized = [] opts.MaxSimultaneousAllocations = 1024; if (!env_options_raw || !std::string_view{env_options_raw}.contains("SampleRate")) - opts.SampleRate = 10000; + opts.SampleRate = 50000; opts.Backtrace = getBackTrace; GuardedAlloc.init(opts); From 1fb5b35dd9a926de4eca1d83b7b1bfc29347d253 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 10 Jun 2024 13:53:25 +0000 Subject: [PATCH 484/856] explicitly define formatter for StringRef Signed-off-by: Duc Canh Le --- base/base/StringRef.h | 4 ++++ src/Coordination/KeeperSnapshotManager.cpp | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/base/base/StringRef.h b/base/base/StringRef.h index 24af84626de..fc0674b8440 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include @@ -376,3 +378,5 @@ namespace PackedZeroTraits std::ostream & operator<<(std::ostream & os, const StringRef & str); + +template<> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 23ff714a929..f25ccab86b1 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -466,7 +466,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial #ifdef NDEBUG /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key.toView()); + " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key); #else throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" " is different from actual children size {} for node {}", From b3ca9cbaf23b4e5686eff8e39d15073eb4466cd5 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Mon, 10 Jun 2024 14:00:36 +0000 Subject: [PATCH 485/856] no-fasttest tag for base64 functions --- tests/queries/0_stateless/03167_base64_url_functions.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03167_base64_url_functions.sql b/tests/queries/0_stateless/03167_base64_url_functions.sql index 60bb1746e90..2152002e412 100644 --- a/tests/queries/0_stateless/03167_base64_url_functions.sql +++ b/tests/queries/0_stateless/03167_base64_url_functions.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + -- incorrect number of arguments SELECT base64UrlEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64UrlDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From f026cc43cd2572894f1e86fcc75650b97c1bbcd9 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 10 Jun 2024 11:17:10 -0300 Subject: [PATCH 486/856] safe guard around no proxy strings --- .../EnvironmentProxyConfigurationResolver.cpp | 17 +++++------------ .../proxyConfigurationToPocoProxyConfig.cpp | 5 +++++ 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index fff2d354e3a..b7b1f1ecfde 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -37,16 +37,9 @@ namespace } } - std::string getNoProxyHostsString() + const char * getNoProxyHosts() { - const char * no_proxy = std::getenv(NO_PROXY_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) - - if (!no_proxy) - { - return ""; - } - - return no_proxy; + return std::getenv(NO_PROXY_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) } ProxyConfiguration buildProxyConfiguration( @@ -86,7 +79,8 @@ ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() { static const auto * http_proxy_host = getProxyHost(Protocol::HTTP); static const auto * https_proxy_host = getProxyHost(Protocol::HTTPS); - static const auto no_proxy_hosts_string = buildPocoNonProxyHosts(getNoProxyHostsString()); + static const auto * no_proxy = getNoProxyHosts(); + static const auto poco_no_proxy_hosts = no_proxy ? buildPocoNonProxyHosts(no_proxy) : ""; static const Poco::URI http_proxy_uri(http_proxy_host ? http_proxy_host : ""); static const Poco::URI https_proxy_uri(https_proxy_host ? https_proxy_host : ""); @@ -94,9 +88,8 @@ ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() return buildProxyConfiguration( request_protocol, request_protocol == Protocol::HTTP ? http_proxy_uri : https_proxy_uri, - no_proxy_hosts_string, + poco_no_proxy_hosts, disable_tunneling_for_https_requests_over_http_proxy); - } } diff --git a/src/Common/proxyConfigurationToPocoProxyConfig.cpp b/src/Common/proxyConfigurationToPocoProxyConfig.cpp index f64dbc3bc02..c06014ac2dc 100644 --- a/src/Common/proxyConfigurationToPocoProxyConfig.cpp +++ b/src/Common/proxyConfigurationToPocoProxyConfig.cpp @@ -53,6 +53,11 @@ std::string buildPocoRegexpEntryWithoutLeadingDot(const std::string & host) * */ std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts_string) { + if (no_proxy_hosts_string.empty()) + { + return ""; + } + static constexpr auto OR_SEPARATOR = "|"; static constexpr auto MATCH_ANYTHING = R"(.*)"; static constexpr auto MATCH_SUBDOMAINS_REGEX = R"((?:.*\.)?)"; From 6fb4514293a83ea92575deee4ae0809f4c97ec8d Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 10 Jun 2024 14:27:45 +0000 Subject: [PATCH 487/856] 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 4ed0eaa3b064fef20a5c6bcfa55167b0033b58dc Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Jun 2024 16:42:09 +0200 Subject: [PATCH 488/856] Update 03142_alter_comment_parameterized_view.sql --- .../0_stateless/03142_alter_comment_parameterized_view.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql index 14af304f98c..98318e99e4a 100644 --- a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql +++ b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test_table_comment; CREATE VIEW test_table_comment AS SELECT toString({date_from:String}); ALTER TABLE test_table_comment MODIFY COMMENT 'test comment'; -SELECT create_table_query FROM system.tables WHERE name = 'test_table_comment'; +SELECT create_table_query FROM system.tables WHERE name = 'test_table_comment' AND database = currentDatabase(); DROP TABLE test_table_comment; From 6b459ee2d2bddb1a11b499880ff1b296de4f7a7d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 10 Jun 2024 14:46:12 +0000 Subject: [PATCH 489/856] remove unused ErrorCodes Signed-off-by: Duc Canh Le --- src/Processors/Transforms/FilterTransform.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index f3b3d8127d4..cd87019a8e0 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; - extern const int LOGICAL_ERROR; } static void replaceFilterToConstant(Block & block, const String & filter_column_name) From dcbcfc9ee84fe6eabdcd721b4d3fceace94ae371 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 15:21:18 +0000 Subject: [PATCH 490/856] fix skipping of remote disks --- .../ServerAsynchronousMetrics.cpp | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 2892025da2d..5633b9b9350 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -210,28 +210,28 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr auto total = disk->getTotalSpace(); /// Some disks don't support information about the space. - if (!total) - continue; - - auto available = disk->getAvailableSpace(); - auto unreserved = disk->getUnreservedSpace(); - - new_values[fmt::format("DiskTotal_{}", name)] = { *total, - "The total size in bytes of the disk (virtual filesystem). Remote filesystems may not provide this information." }; - - if (available) + if (total) { - new_values[fmt::format("DiskUsed_{}", name)] = { *total - *available, - "Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." }; + auto available = disk->getAvailableSpace(); + auto unreserved = disk->getUnreservedSpace(); - new_values[fmt::format("DiskAvailable_{}", name)] = { *available, - "Available bytes on the disk (virtual filesystem). Remote filesystems may not provide this information." }; + new_values[fmt::format("DiskTotal_{}", name)] = { *total, + "The total size in bytes of the disk (virtual filesystem). Remote filesystems may not provide this information." }; + + if (available) + { + new_values[fmt::format("DiskUsed_{}", name)] = { *total - *available, + "Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." }; + + new_values[fmt::format("DiskAvailable_{}", name)] = { *available, + "Available bytes on the disk (virtual filesystem). Remote filesystems may not provide this information." }; + } + + if (unreserved) + new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved, + "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." }; } - if (unreserved) - new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved, - "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." }; - try { if (auto s3_client = disk->getS3StorageClient()) From ec5fa482e40d5602134f4ead167b63efd08d1cc4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 10 Jun 2024 15:26:36 +0000 Subject: [PATCH 491/856] 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 33766797c1de8523dc3e66418777733debe7396d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 10 Jun 2024 15:37:40 +0000 Subject: [PATCH 492/856] Refactor query plan prewhere optimization for Merge --- .../Optimizations/filterPushDown.cpp | 9 + .../Optimizations/optimizePrewhere.cpp | 7 +- .../QueryPlan/SourceStepWithFilter.h | 5 - src/Storages/StorageMerge.cpp | 313 +++++++++--------- src/Storages/StorageMerge.h | 26 +- .../02156_storage_merge_prewhere.reference | 18 +- 6 files changed, 205 insertions(+), 173 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 8ca240b3e8b..263598bdca7 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -608,6 +609,14 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return 3; } + if (auto * read_from_merge = typeid_cast(child.get())) + { + FilterDAGInfo info{filter->getExpression(), filter->getFilterColumnName(), filter->removesFilterColumn()}; + read_from_merge->addFilter(std::move(info)); + std::swap(*parent_node, *child_node); + return 1; + } + return 0; } diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index fbd9b451ddc..1ce9d1482c9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -4,10 +4,10 @@ #include #include #include +#include #include #include #include - namespace DB { @@ -30,7 +30,7 @@ static void removeFromOutput(ActionsDAG & dag, const std::string name) void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) { - if (stack.size() < 3) + if (stack.size() < 2) return; auto & frame = stack.back(); @@ -45,6 +45,9 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) if (!source_step_with_filter) return; + if (typeid_cast(frame.node->step.get())) + return; + const auto & storage_snapshot = source_step_with_filter->getStorageSnapshot(); const auto & storage = storage_snapshot->storage; if (!storage.canMoveConditionsToPrewhere()) diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index 0971b99d828..126d4824fff 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -49,11 +49,6 @@ public: filter_dags.push_back(std::move(filter_dag)); } - void addFilterFromParentStep(const ActionsDAG::Node * filter_node) - { - filter_nodes.nodes.push_back(filter_node); - } - /// Apply filters that can optimize reading from storage. void applyFilters() { diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 4c678a1228b..b42a8ed90ed 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -37,6 +37,8 @@ #include #include #include +#include +#include #include #include #include @@ -402,10 +404,14 @@ ReadFromMerge::ReadFromMerge( { } -void ReadFromMerge::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) +void ReadFromMerge::addFilter(FilterDAGInfo filter) { - SourceStepWithFilter::updatePrewhereInfo(prewhere_info_value); - common_header = applyPrewhereActions(common_header, prewhere_info); + output_stream->header = FilterTransform::transformHeader( + output_stream->header, + filter.actions.get(), + filter.column_name, + filter.do_remove_column); + pushed_down_filters.push_back(std::move(filter)); } void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) @@ -441,14 +447,11 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu auto modified_query_info = getModifiedQueryInfo(modified_context, table, nested_storage_snaphsot, real_column_names, column_names_as_aliases, aliases); - auto source_pipeline = createSources( + auto source_pipeline = buildPipeline( child_plan.plan, nested_storage_snaphsot, modified_query_info, common_processed_stage, - common_header, - child_plan.table_aliases, - child_plan.row_policy_data_opt, table); if (source_pipeline && source_pipeline->initialized()) @@ -651,6 +654,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ res.back().plan = createPlanForTable( nested_storage_snaphsot, + aliases, modified_query_info, common_processed_stage, required_max_block_size, @@ -660,8 +664,17 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ modified_context, current_streams); res.back().plan.addInterpreterContext(modified_context); - } + // createSources1( + // res.back().plan, + // nested_storage_snaphsot, + // modified_query_info, + // common_processed_stage, + // common_header, + // res.back().table_aliases, + // row_policy_data_opt, + // table); + } return res; } @@ -1019,7 +1032,7 @@ bool recursivelyApplyToReadingSteps(QueryPlan::Node * node, const std::function< return ok; } -QueryPipelineBuilderPtr ReadFromMerge::createSources( +void ReadFromMerge::updatePlan( QueryPlan & plan, const StorageSnapshotPtr & storage_snapshot_, SelectQueryInfo & modified_query_info, @@ -1027,22 +1040,124 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( const Block & header, const Aliases & aliases, const RowPolicyDataOpt & row_policy_data_opt, - const StorageWithLockAndName & storage_with_lock, - bool concat_streams) const + const StorageWithLockAndName & storage_with_lock) const { if (!plan.isInitialized()) - return std::make_unique(); - - QueryPipelineBuilderPtr builder; + return; const auto & [database_name, storage, _, table_name] = storage_with_lock; bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; auto storage_stage = storage->getQueryProcessingStage(context, processed_stage, storage_snapshot_, modified_query_info); - builder = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + /// Add virtual columns if we don't already have them. + Block plan_header = plan.getCurrentDataStream().header; + + if (allow_experimental_analyzer) + { + String table_alias = modified_query_info.query_tree->as()->getJoinTree()->as()->getAlias(); + + String database_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_database" : table_alias + "._database"; + String table_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_table" : table_alias + "._table"; + + if (has_database_virtual_column && common_header.has(database_column) + && storage_stage == QueryProcessingStage::FetchColumns && !plan_header.has(database_column)) + { + ColumnWithTypeAndName column; + column.name = database_column; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(database_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto expression_step = std::make_unique(plan.getCurrentDataStream(), adding_column_dag); + plan.addStep(std::move(expression_step)); + plan_header = plan.getCurrentDataStream().header; + } + + if (has_table_virtual_column && common_header.has(table_column) + && storage_stage == QueryProcessingStage::FetchColumns && !plan_header.has(table_column)) + { + ColumnWithTypeAndName column; + column.name = table_column; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(table_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto expression_step = std::make_unique(plan.getCurrentDataStream(), adding_column_dag); + plan.addStep(std::move(expression_step)); + plan_header = plan.getCurrentDataStream().header; + } + } + else + { + if (has_database_virtual_column && common_header.has("_database") && !plan_header.has("_database")) + { + ColumnWithTypeAndName column; + column.name = "_database"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(database_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto expression_step = std::make_unique(plan.getCurrentDataStream(), adding_column_dag); + plan.addStep(std::move(expression_step)); + plan_header = plan.getCurrentDataStream().header; + } + + if (has_table_virtual_column && common_header.has("_table") && !plan_header.has("_table")) + { + ColumnWithTypeAndName column; + column.name = "_table"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(table_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto expression_step = std::make_unique(plan.getCurrentDataStream(), adding_column_dag); + plan.addStep(std::move(expression_step)); + plan_header = plan.getCurrentDataStream().header; + } + } + + /// Subordinary tables could have different but convertible types, like numeric types of different width. + /// We must return streams with structure equals to structure of Merge table. + convertAndFilterSourceStream( + header, modified_query_info, storage_snapshot_, aliases, row_policy_data_opt, context, plan, storage_stage); + + for (const auto & filter_info : pushed_down_filters) + { + auto filter_step = std::make_unique( + plan.getCurrentDataStream(), + filter_info.actions->clone(), + filter_info.column_name, + filter_info.do_remove_column); + + plan.addStep(std::move(filter_step)); + } +} + +QueryPipelineBuilderPtr ReadFromMerge::buildPipeline( + QueryPlan & plan, + const StorageSnapshotPtr & storage_snapshot_, + SelectQueryInfo & modified_query_info, + QueryProcessingStage::Enum processed_stage, + const StorageWithLockAndName & storage_with_lock) const +{ + if (!plan.isInitialized()) + return nullptr; + + const auto & [database_name, storage, _, table_name] = storage_with_lock; + auto storage_stage + = storage->getQueryProcessingStage(context, processed_stage, storage_snapshot_, modified_query_info); + + auto optimisation_settings = QueryPlanOptimizationSettings::fromContext(context); + /// All optimisations will be done at plans creation + optimisation_settings.optimize_plan = false; + auto builder = plan.buildQueryPipeline(optimisation_settings, BuildQueryPipelineSettings::fromContext(context)); + + if (!builder->initialized()) + return builder; + + bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { /** Materialization is needed, since from distributed storage the constants come materialized. @@ -1052,93 +1167,11 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( builder->addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); } - if (builder->initialized()) + if (builder->getNumStreams() > 1) { - if (concat_streams && builder->getNumStreams() > 1) - { - // It's possible to have many tables read from merge, resize(1) might open too many files at the same time. - // Using concat instead. - builder->addTransform(std::make_shared(builder->getHeader(), builder->getNumStreams())); - } - - /// Add virtual columns if we don't already have them. - - Block pipe_header = builder->getHeader(); - - if (allow_experimental_analyzer) - { - String table_alias = modified_query_info.query_tree->as()->getJoinTree()->as()->getAlias(); - - String database_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_database" : table_alias + "._database"; - String table_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_table" : table_alias + "._table"; - - if (has_database_virtual_column && common_header.has(database_column) - && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(database_column)) - { - ColumnWithTypeAndName column; - column.name = database_column; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(database_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); - - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } - - if (has_table_virtual_column && common_header.has(table_column) - && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(table_column)) - { - ColumnWithTypeAndName column; - column.name = table_column; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(table_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); - - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } - } - else - { - if (has_database_virtual_column && common_header.has("_database") && !pipe_header.has("_database")) - { - ColumnWithTypeAndName column; - column.name = "_database"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(database_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } - - if (has_table_virtual_column && common_header.has("_table") && !pipe_header.has("_table")) - { - ColumnWithTypeAndName column; - column.name = "_table"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(table_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } - } - - /// Subordinary tables could have different but convertible types, like numeric types of different width. - /// We must return streams with structure equals to structure of Merge table. - convertAndFilterSourceStream( - header, modified_query_info, storage_snapshot_, aliases, row_policy_data_opt, context, *builder, storage_stage); + // It's possible to have many tables read from merge, resize(1) might open too many files at the same time. + // Using concat instead. + builder->addTransform(std::make_shared(builder->getHeader(), builder->getNumStreams())); } return builder; @@ -1146,6 +1179,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryPlan ReadFromMerge::createPlanForTable( const StorageSnapshotPtr & storage_snapshot_, + const Aliases & table_aliases, SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, @@ -1248,6 +1282,19 @@ QueryPlan ReadFromMerge::createPlanForTable( } } + updatePlan( + plan, + storage_snapshot_, + modified_query_info, + common_processed_stage, + common_header, + table_aliases, + row_policy_data_opt, + storage_with_lock); + + if (plan.isInitialized()) + plan.optimize(QueryPlanOptimizationSettings::fromContext(modified_context)); + return plan; } @@ -1306,12 +1353,10 @@ void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) step->addFilter(actions_dag, filter_column_name); } -void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & builder) const +void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove filter column */); - }); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag, filter_column_name, true /* remove filter column */); + plan.addStep(std::move(filter_step)); } StorageMerge::StorageListWithLocks ReadFromMerge::getSelectedTables( @@ -1490,13 +1535,13 @@ void ReadFromMerge::convertAndFilterSourceStream( const Aliases & aliases, const RowPolicyDataOpt & row_policy_data_opt, ContextPtr local_context, - QueryPipelineBuilder & builder, + QueryPlan & plan, QueryProcessingStage::Enum processed_stage) { - Block before_block_header = builder.getHeader(); + Block before_block_header = plan.getCurrentDataStream().header; auto storage_sample_block = snapshot->metadata->getSampleBlock(); - auto pipe_columns = builder.getHeader().getNamesAndTypesList(); + auto pipe_columns = before_block_header.getNamesAndTypesList(); if (local_context->getSettingsRef().allow_experimental_analyzer) { @@ -1519,13 +1564,8 @@ void ReadFromMerge::convertAndFilterSourceStream( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); actions_dag->addOrReplaceInOutputs(actions_dag->addAlias(*nodes.front(), alias.name)); - - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, actions); - }); + auto expression_step = std::make_unique(plan.getCurrentDataStream(), actions_dag); + plan.addStep(std::move(expression_step)); } } else @@ -1539,12 +1579,8 @@ void ReadFromMerge::convertAndFilterSourceStream( auto dag = std::make_shared(pipe_columns); auto actions_dag = expression_analyzer.getActionsDAG(true, false); - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, actions); - }); + auto expression_step = std::make_unique(plan.getCurrentDataStream(), actions_dag); + plan.addStep(std::move(expression_step)); } } @@ -1556,20 +1592,15 @@ void ReadFromMerge::convertAndFilterSourceStream( if (row_policy_data_opt) { - row_policy_data_opt->addFilterTransform(builder); + row_policy_data_opt->addFilterTransform(plan); } - auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), + auto convert_actions_dag = ActionsDAG::makeConvertingActions(plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); - auto actions = std::make_shared( - std::move(convert_actions_dag), - ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, actions); - }); + auto expression_step = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); + plan.addStep(std::move(expression_step)); } const ReadFromMerge::StorageListWithLocks & ReadFromMerge::getSelectedTables() @@ -1606,29 +1637,11 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) return true; } -void ReadFromMerge::applyFilters(const QueryPlan & plan, const ActionDAGNodes & added_filter_nodes) const -{ - auto apply_filters = [&added_filter_nodes](ReadFromMergeTree & read_from_merge_tree) - { - for (const auto & node : added_filter_nodes.nodes) - read_from_merge_tree.addFilterFromParentStep(node); - - read_from_merge_tree.SourceStepWithFilter::applyFilters(); - return true; - }; - - recursivelyApplyToReadingSteps(plan.getRootNode(), apply_filters); -} - void ReadFromMerge::applyFilters(ActionDAGNodes added_filter_nodes) { SourceStepWithFilter::applyFilters(added_filter_nodes); filterTablesAndCreateChildrenPlans(); - - for (const auto & child_plan : *child_plans) - if (child_plan.plan.isInitialized()) - applyFilters(child_plan.plan, added_filter_nodes); } QueryPlanRawPtrs ReadFromMerge::getChildPlans() diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 735c8711a63..42544676bd8 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -165,7 +165,7 @@ public: QueryPlanRawPtrs getChildPlans() override; - void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) override; + void addFilter(FilterDAGInfo filter); private: const size_t required_max_block_size; @@ -221,7 +221,7 @@ private: /// Create explicit filter transform to exclude /// rows that are not conform to row level policy - void addFilterTransform(QueryPipelineBuilder &) const; + void addFilterTransform(QueryPlan &) const; private: std::string filter_column_name; // complex filter, may contain logic operations @@ -243,14 +243,16 @@ private: /// It's needed to guarantee lifetime for child steps to be the same as for this step (mainly for EXPLAIN PIPELINE). std::optional> child_plans; + /// Store filters pushed down from query plan optimization. Filters are added on top of child plans. + std::vector pushed_down_filters; + std::vector createChildrenPlans(SelectQueryInfo & query_info_) const; void filterTablesAndCreateChildrenPlans(); - void applyFilters(const QueryPlan & plan, const ActionDAGNodes & added_filter_nodes) const; - QueryPlan createPlanForTable( const StorageSnapshotPtr & storage_snapshot, + const Aliases & table_aliases, SelectQueryInfo & query_info, QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, @@ -260,7 +262,7 @@ private: ContextMutablePtr modified_context, size_t streams_num) const; - QueryPipelineBuilderPtr createSources( + void updatePlan( QueryPlan & plan, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & modified_query_info, @@ -268,8 +270,14 @@ private: const Block & header, const Aliases & aliases, const RowPolicyDataOpt & row_policy_data_opt, - const StorageWithLockAndName & storage_with_lock, - bool concat_streams = false) const; + const StorageWithLockAndName & storage_with_lock) const; + + QueryPipelineBuilderPtr buildPipeline( + QueryPlan & plan, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & modified_query_info, + QueryProcessingStage::Enum processed_stage, + const StorageWithLockAndName & storage_with_lock) const; static void convertAndFilterSourceStream( const Block & header, @@ -278,15 +286,13 @@ private: const Aliases & aliases, const RowPolicyDataOpt & row_policy_data_opt, ContextPtr context, - QueryPipelineBuilder & builder, + QueryPlan & plan, QueryProcessingStage::Enum processed_stage); StorageMerge::StorageListWithLocks getSelectedTables( ContextPtr query_context, bool filter_by_database_virtual_column, bool filter_by_table_virtual_column) const; - - // static VirtualColumnsDescription createVirtuals(StoragePtr first_table); }; } diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 86a36a9392c..5632b333c5e 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,15 +1,21 @@ - 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) + Filter column: and(equals(k, 3), notEmpty(v)) (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) + Filter column: and(equals(k, 3), notEmpty(v)) (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) - 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 From fdb32e825c6053c1bf4fecbb5293ddb8f6e747b9 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Mon, 10 Jun 2024 15:39:38 +0000 Subject: [PATCH 493/856] PR post-review fixes --- src/Functions/FunctionBase64Conversion.cpp | 93 ---------------------- src/Functions/FunctionBase64Conversion.h | 83 ++++++++++++++++--- 2 files changed, 71 insertions(+), 105 deletions(-) delete mode 100644 src/Functions/FunctionBase64Conversion.cpp diff --git a/src/Functions/FunctionBase64Conversion.cpp b/src/Functions/FunctionBase64Conversion.cpp deleted file mode 100644 index a87ce31f478..00000000000 --- a/src/Functions/FunctionBase64Conversion.cpp +++ /dev/null @@ -1,93 +0,0 @@ -#include "config.h" -#if USE_BASE64 -# include -# include -# include - -namespace DB -{ - -std::vector preprocessBase64Url(const std::span src) -{ - std::vector padded_src{}; - // insert padding to please aklomp library - size_t padded_size = src.size(); - size_t remainder = padded_size % 4; - switch (remainder) - { - case 0: - break; // no padding needed - case 1: - padded_size += 3; // this case is impossible to occur, however, we'll insert padding anyway - break; - case 2: - padded_size += 2; // two bytes padding - break; - default: // remainder == 3 - padded_size += 1; // one byte padding - break; - } - padded_src.resize(padded_size); - - // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 - size_t i = 0; - for (; i < src.size(); ++i) - { - switch (src[i]) - { - case '_': - padded_src[i] = '/'; - break; - case '-': - padded_src[i] = '+'; - break; - default: - padded_src[i] = src[i]; - break; - } - } - if (remainder == 1) - { - padded_src[i] = '='; - ++i; - padded_src[i] = '='; - ++i; - padded_src[i] = '='; - } - else if (remainder == 2) - { - padded_src[i] = '='; - ++i; - padded_src[i] = '='; - } - else if (remainder == 3) - padded_src[i] = '='; - - return padded_src; -} - -size_t postprocessBase64Url(UInt8 * dst, size_t out_len) -{ - // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 - for (size_t i = 0; i < out_len; ++i) - { - switch (dst[i]) - { - case '/': - dst[i] = '_'; - break; - case '+': - dst[i] = '-'; - break; - case '=': // stop when padding is detected - return i; - default: - break; - } - } - return out_len; -} - -} - -#endif diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 51ca28aa670..8d73d0c70df 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -28,8 +28,67 @@ enum class Base64Variant : uint8_t Url }; -extern std::vector preprocessBase64Url(const std::span src); -extern size_t postprocessBase64Url(UInt8 * dst, size_t out_len); +inline std::string preprocessBase64Url(std::span src) +{ + std::string padded_src; + // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 + for (size_t i = 0; i < src.size(); ++i) + { + switch (src[i]) + { + case '_': + padded_src += '/'; + break; + case '-': + padded_src += '+'; + break; + default: + padded_src += src[i]; + break; + } + } + + // insert padding to please aklomp library + size_t remainder = src.size() % 4; + switch (remainder) + { + case 0: + break; // no padding needed + case 1: + padded_src.append("==="); // this case is impossible to occur, however, we'll insert padding anyway + break; + case 2: + padded_src.append("=="); // two bytes padding + break; + default: // remainder == 3 + padded_src.append("="); // one byte padding + break; + } + + return padded_src; +} + +inline size_t postprocessBase64Url(UInt8 * dst, size_t out_len) +{ + // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 + for (size_t i = 0; i < out_len; ++i) + { + switch (dst[i]) + { + case '/': + dst[i] = '_'; + break; + case '+': + dst[i] = '-'; + break; + case '=': // stop when padding is detected + return i; + default: + break; + } + } + return out_len; +} template struct Base64Encode @@ -41,15 +100,15 @@ struct Base64Encode return ((string_length - string_count) / 3 + string_count) * 4 + string_count; } - static size_t perform(const std::span src, UInt8 * dst) + static size_t perform(std::span src, UInt8 * dst) { size_t outlen = 0; base64_encode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); if constexpr (variant == Base64Variant::Url) - return postprocessBase64Url(dst, outlen); - else - return outlen; + outlen = postprocessBase64Url(dst, outlen); + + return outlen; } }; @@ -63,14 +122,14 @@ struct Base64Decode return ((string_length - string_count) / 4 + string_count) * 3 + string_count; } - static size_t perform(const std::span src, UInt8 * dst) + static size_t perform(std::span src, UInt8 * dst) { int rc; size_t outlen = 0; if constexpr (variant == Base64Variant::Url) { - auto src_padded = preprocessBase64Url(src); - rc = base64_decode(reinterpret_cast(src_padded.data()), src_padded.size(), reinterpret_cast(dst), &outlen, 0); + std::string src_padded = preprocessBase64Url(src); + rc = base64_decode(src_padded.data(), src_padded.size(), reinterpret_cast(dst), &outlen, 0); } else { @@ -98,14 +157,14 @@ struct TryBase64Decode return Base64Decode::getBufferSize(string_length, string_count); } - static size_t perform(const std::span src, UInt8 * dst) + static size_t perform(std::span src, UInt8 * dst) { int rc; size_t outlen = 0; if constexpr (variant == Base64Variant::Url) { - auto src_padded = preprocessBase64Url(src); - rc = base64_decode(reinterpret_cast(src_padded.data()), src_padded.size(), reinterpret_cast(dst), &outlen, 0); + std::string src_padded = preprocessBase64Url(src); + rc = base64_decode(src_padded.data(), src_padded.size(), reinterpret_cast(dst), &outlen, 0); } else { From a8b1b11ee640d458804e4088c4d75c767d83c204 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 10 Jun 2024 15:55:00 +0000 Subject: [PATCH 494/856] Fixing test. --- src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- src/Planner/Planner.cpp | 2 +- .../0_stateless/02156_storage_merge_prewhere.reference | 2 -- ...2156_storage_merge_prewhere_not_ready_set_bug.reference | 1 + .../02156_storage_merge_prewhere_not_ready_set_bug.sql | 7 +++++++ 5 files changed, 12 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.reference create mode 100644 tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e72cf670f69..f8f3867dfd4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2544,9 +2544,9 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc } else if (storage) { - if (shouldMoveToPrewhere() && settings.query_plan_optimize_prewhere && settings.query_plan_enable_optimizations - && typeid_cast(storage.get())) - collectFiltersForAnalysis(query_ptr, context, storage_snapshot, options, query_info); + // if (shouldMoveToPrewhere() && settings.query_plan_optimize_prewhere && settings.query_plan_enable_optimizations + // && typeid_cast(storage.get())) + // collectFiltersForAnalysis(query_ptr, context, storage_snapshot, options, query_info); /// Table. if (max_streams == 0) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b40e23a9553..15b92ed12da 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -166,7 +166,7 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & continue; const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); - if (typeid_cast(storage.get()) || typeid_cast(storage.get()) + if (typeid_cast(storage.get()) || (parallel_replicas_estimation_enabled && std::dynamic_pointer_cast(storage))) { collect_filters = true; diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 5632b333c5e..8a18c609ede 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,8 +1,6 @@ - Filter column: and(equals(k, 3), notEmpty(v)) (removed) 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) diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.reference new file mode 100644 index 00000000000..20c58c33770 --- /dev/null +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.reference @@ -0,0 +1 @@ +59900 1000 1396 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.sql new file mode 100644 index 00000000000..fc18c97cb6e --- /dev/null +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.sql @@ -0,0 +1,7 @@ +create table merge_kek_1 (x UInt32, y UInt32) engine = MergeTree order by x; +create table merge_kek_2 (x UInt32, y UInt32) engine = MergeTree order by x; + +insert into merge_kek_1 select number, number from numbers(100); +insert into merge_kek_2 select number + 500, number + 500 from numbers(1e6); + +select sum(x), min(x + x), max(x + x) from merge(currentDatabase(), '^merge_kek_.$') where x > 200 and y in (select 500 + number * 2 from numbers(100)) settings max_threads=2; From 8904880480a314e456c05e9721a19c202e606382 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 18:03:26 +0200 Subject: [PATCH 495/856] Try fix test --- src/IO/S3Settings.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp index 396eb8f5ec7..e88d8133c32 100644 --- a/src/IO/S3Settings.cpp +++ b/src/IO/S3Settings.cpp @@ -20,6 +20,8 @@ void S3SettingsByEndpoint::loadFromConfig( Poco::Util::AbstractConfiguration::Keys config_keys; config.keys(config_prefix, config_keys); + auto default_auth_settings = S3::AuthSettings(config, settings, config_prefix); + auto default_request_settings = S3::RequestSettings(config, settings, config_prefix); for (const String & key : config_keys) { @@ -27,8 +29,11 @@ void S3SettingsByEndpoint::loadFromConfig( const auto endpoint_path = key_path + ".endpoint"; if (config.has(endpoint_path)) { - auto auth_settings = S3::AuthSettings(config, settings, key_path); - auto request_settings = S3::RequestSettings(config, settings, key_path, "", settings.s3_validate_request_settings); + auto auth_settings{default_auth_settings}; + auth_settings.updateIfChanged(S3::AuthSettings(config, settings, key_path)); + + auto request_settings{default_request_settings}; + request_settings.updateIfChanged(S3::RequestSettings(config, settings, key_path, "", settings.s3_validate_request_settings)); s3_settings.emplace( config.getString(endpoint_path), From 374854a972616bb42c0d2c960511e377fb271b8b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 10 Jun 2024 18:20:55 +0200 Subject: [PATCH 496/856] Increase timeout in wait_for_all_mutations --- tests/queries/0_stateless/mergetree_mutations.lib | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/mergetree_mutations.lib b/tests/queries/0_stateless/mergetree_mutations.lib index b11b2e6b852..9eeea87b52d 100644 --- a/tests/queries/0_stateless/mergetree_mutations.lib +++ b/tests/queries/0_stateless/mergetree_mutations.lib @@ -37,7 +37,7 @@ function wait_for_all_mutations() echo "Timed out while waiting for mutation to execute!" fi - sleep 0.1 + sleep 0.3 done } From 32431b9c7d6569914724595cd69ba4178d26ccca Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 10 Jun 2024 18:40:57 +0200 Subject: [PATCH 497/856] Update src/Common/Throttler.cpp Co-authored-by: Antonio Andelic --- src/Common/Throttler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 61d120e11ea..e7e38cdd6ad 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -104,8 +104,8 @@ bool Throttler::isThrottling() const Int64 Throttler::getAvailable() { // To update bucket state and receive current number of token in a thread-safe way - size_t count_value; - double tokens_value; + size_t count_value = 0; + double tokens_value = 0.0; addImpl(0, count_value, tokens_value); return static_cast(tokens_value); From eb839de79576f3bf554658009b33122d0fc27e98 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 10 Jun 2024 18:41:08 +0200 Subject: [PATCH 498/856] Update src/Interpreters/ServerAsynchronousMetrics.cpp Co-authored-by: Antonio Andelic --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 5633b9b9350..7defe2fc721 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -254,7 +254,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr } catch (...) { - // Skip disk than do not have s3 throttlers + // Skip disk that do not have s3 throttlers } } } From ed19e7c2bfb37fda74ab838bd530ba4513867f68 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 18:44:07 +0200 Subject: [PATCH 499/856] Ping CI From 0c8a9f099c9897b84b72197eb34b1bbc72a7bd78 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 16:45:48 +0000 Subject: [PATCH 500/856] fix build --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 5633b9b9350..9a0041eff05 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -232,6 +232,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." }; } +#if USE_AWS_S3 try { if (auto s3_client = disk->getS3StorageClient()) @@ -256,6 +257,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr { // Skip disk than do not have s3 throttlers } +#endif } } From 65ee0254fa26a4bda05015d62a8250fa4d6ac1f1 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 16:48:24 +0000 Subject: [PATCH 501/856] safer --- src/Common/Throttler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index e7e38cdd6ad..a581ff1766f 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -41,8 +41,8 @@ Throttler::Throttler(size_t max_speed_, size_t limit_, const char * limit_exceed UInt64 Throttler::add(size_t amount) { // Values obtained under lock to be checked after release - size_t count_value; - double tokens_value; + size_t count_value = 0; + double tokens_value = 0.0; addImpl(amount, count_value, tokens_value); if (limit && count_value > limit) From 20d673a20645740d7b805e03521c300c829557a2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 10 Jun 2024 17:02:26 +0000 Subject: [PATCH 502/856] Fix more tests. --- src/Storages/StorageMerge.cpp | 37 ++++--------------- ..._optimize_count_for_merge_tables.reference | 9 +++-- 2 files changed, 14 insertions(+), 32 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index b42a8ed90ed..f05618f8488 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1215,35 +1215,14 @@ QueryPlan ReadFromMerge::createPlanForTable( if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(storage_snapshot_->metadata->getColumns().getAllPhysical()).name); - StorageView * view = dynamic_cast(storage.get()); - if (!view || allow_experimental_analyzer) - { - storage->read(plan, - real_column_names, - storage_snapshot_, - modified_query_info, - modified_context, - processed_stage, - max_block_size, - UInt32(streams_num)); - } - else - { - /// For view storage, we need to rewrite the `modified_query_info.view_query` to optimize read. - /// The most intuitive way is to use InterpreterSelectQuery. - - /// Intercept the settings - modified_context->setSetting("max_threads", streams_num); - modified_context->setSetting("max_streams_to_max_threads_ratio", 1); - modified_context->setSetting("max_block_size", max_block_size); - - InterpreterSelectQuery interpreter(modified_query_info.query, - modified_context, - storage, - view->getInMemoryMetadataPtr(), - SelectQueryOptions(processed_stage)); - interpreter.buildQueryPlan(plan); - } + storage->read(plan, + real_column_names, + storage_snapshot_, + modified_query_info, + modified_context, + processed_stage, + max_block_size, + UInt32(streams_num)); if (!plan.isInitialized()) return {}; diff --git a/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.reference b/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.reference index 786a6b3bf25..7278018f1d6 100644 --- a/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.reference +++ b/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.reference @@ -7,6 +7,9 @@ Expression ((Projection + Before ORDER BY)) Aggregating Expression (Before GROUP BY) ReadFromMerge - ReadFromMergeTree (default.mt1) - ReadFromMergeTree (default.mt2) - ReadFromStorage (TinyLog) + Expression + ReadFromMergeTree (default.mt1) + Expression + ReadFromMergeTree (default.mt2) + Expression + ReadFromStorage (TinyLog) From 28a467a3beafb1eefa3a878c8f1df37783b2aaa8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sat, 8 Jun 2024 07:44:34 +0000 Subject: [PATCH 503/856] fix 03165_string_functions_with_token_text_indexes --- .../03165_string_functions_with_token_text_indexes.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.sql b/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.sql index fee30af0245..bae98bd1eb6 100644 --- a/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.sql +++ b/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.sql @@ -127,7 +127,9 @@ CREATE TABLE 03165_token_ft INDEX idx_message message TYPE full_text() GRANULARITY 1 ) ENGINE = MergeTree -ORDER BY id; +ORDER BY id +-- Full text index works only with full parts. +SETTINGS min_bytes_for_full_part_storage=0; INSERT INTO 03165_token_ft VALUES(1, 'Service is not ready'); From 3bcb32c7193ce9bb1209dbe4d6761190a916b9ac Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 10 Jun 2024 17:52:46 +0000 Subject: [PATCH 504/856] Fixing another test. --- src/Storages/StorageMerge.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f05618f8488..6d12876c776 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1618,6 +1618,9 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) void ReadFromMerge::applyFilters(ActionDAGNodes added_filter_nodes) { + for (const auto & filter_info : pushed_down_filters) + added_filter_nodes.nodes.push_back(&filter_info.actions->findInOutputs(filter_info.column_name)); + SourceStepWithFilter::applyFilters(added_filter_nodes); filterTablesAndCreateChildrenPlans(); From 5f7683f3914b0270ada6334c87410780074b71fd Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 10 Jun 2024 18:02:44 +0000 Subject: [PATCH 505/856] Avoid writing to finalized buffer in File-like storages --- src/Storages/ObjectStorage/StorageObjectStorageSink.cpp | 5 +++-- src/Storages/StorageFile.cpp | 3 ++- src/Storages/StorageURL.cpp | 3 ++- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 0a3cf19a590..154c2b07251 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -83,7 +83,6 @@ void StorageObjectStorageSink::finalize() { writer->finalize(); writer->flush(); - write_buf->finalize(); } catch (...) { @@ -91,12 +90,14 @@ void StorageObjectStorageSink::finalize() release(); throw; } + + write_buf->finalize(); } void StorageObjectStorageSink::release() { writer.reset(); - write_buf.reset(); + write_buf->finalize(); } PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index aaf84f6f82c..83bfcdaf415 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1823,7 +1823,6 @@ private: { writer->finalize(); writer->flush(); - write_buf->finalize(); } catch (...) { @@ -1831,6 +1830,8 @@ private: release(); throw; } + + write_buf->finalize(); } void release() diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9302e7ef3e5..cd39ca42574 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -609,7 +609,6 @@ void StorageURLSink::finalize() { writer->finalize(); writer->flush(); - write_buf->finalize(); } catch (...) { @@ -617,6 +616,8 @@ void StorageURLSink::finalize() release(); throw; } + + write_buf->finalize(); } void StorageURLSink::release() From 68e1d8701c973def035f0b9ea17f6ca3d224d73c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 10 Jun 2024 18:09:07 +0000 Subject: [PATCH 506/856] 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 507/856] 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 71a42d427a0be4c5803fdfcacd088cced97377af Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 16:42:13 +0000 Subject: [PATCH 508/856] Minor fixups --- src/Functions/FunctionBase64Conversion.h | 30 ++++++++++--------- .../0_stateless/00732_base64_functions.sql | 3 +- .../03167_base64_url_functions.sql | 26 ++++++++-------- 3 files changed, 30 insertions(+), 29 deletions(-) diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 8d73d0c70df..e569a351591 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -12,7 +12,7 @@ # include # include -# include +# include namespace DB { @@ -28,13 +28,15 @@ enum class Base64Variant : uint8_t Url }; -inline std::string preprocessBase64Url(std::span src) +inline std::string preprocessBase64Url(std::string_view src) { std::string padded_src; + padded_src.reserve(src.size() + 3); + // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 - for (size_t i = 0; i < src.size(); ++i) + for (auto s : src) { - switch (src[i]) + switch (s) { case '_': padded_src += '/'; @@ -43,12 +45,12 @@ inline std::string preprocessBase64Url(std::span src) padded_src += '+'; break; default: - padded_src += src[i]; + padded_src += s; break; } } - // insert padding to please aklomp library + /// Insert padding to please aklomp library size_t remainder = src.size() % 4; switch (remainder) { @@ -100,10 +102,10 @@ struct Base64Encode return ((string_length - string_count) / 3 + string_count) * 4 + string_count; } - static size_t perform(std::span src, UInt8 * dst) + static size_t perform(std::string_view src, UInt8 * dst) { size_t outlen = 0; - base64_encode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); + base64_encode(src.data(), src.size(), reinterpret_cast(dst), &outlen, 0); if constexpr (variant == Base64Variant::Url) outlen = postprocessBase64Url(dst, outlen); @@ -122,7 +124,7 @@ struct Base64Decode return ((string_length - string_count) / 4 + string_count) * 3 + string_count; } - static size_t perform(std::span src, UInt8 * dst) + static size_t perform(std::string_view src, UInt8 * dst) { int rc; size_t outlen = 0; @@ -133,7 +135,7 @@ struct Base64Decode } else { - rc = base64_decode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); + rc = base64_decode(src.data(), src.size(), reinterpret_cast(dst), &outlen, 0); } if (rc != 1) @@ -157,7 +159,7 @@ struct TryBase64Decode return Base64Decode::getBufferSize(string_length, string_count); } - static size_t perform(std::span src, UInt8 * dst) + static size_t perform(std::string_view src, UInt8 * dst) { int rc; size_t outlen = 0; @@ -168,7 +170,7 @@ struct TryBase64Decode } else { - rc = base64_decode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); + rc = base64_decode(src.data(), src.size(), reinterpret_cast(dst), &outlen, 0); } if (rc != 1) @@ -232,7 +234,7 @@ private: auto * dst = dst_chars.data(); auto * dst_pos = dst; - const auto * src = src_chars.data(); + const auto * src = reinterpret_cast(src_chars.data()); size_t src_offset_prev = 0; for (size_t row = 0; row < src_row_count; ++row) @@ -272,7 +274,7 @@ private: auto * dst = dst_chars.data(); auto * dst_pos = dst; - const auto * src = src_chars.data(); + const auto * src = reinterpret_cast(src_chars.data()); for (size_t row = 0; row < src_row_count; ++row) { diff --git a/tests/queries/0_stateless/00732_base64_functions.sql b/tests/queries/0_stateless/00732_base64_functions.sql index 3c60bf939fe..b4be8db4ede 100644 --- a/tests/queries/0_stateless/00732_base64_functions.sql +++ b/tests/queries/0_stateless/00732_base64_functions.sql @@ -1,6 +1,5 @@ -- Tags: no-fasttest - -SET send_logs_level = 'fatal'; +-- no-fasttest because aklomp-base64 library is required SELECT base64Encode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64Decode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/03167_base64_url_functions.sql b/tests/queries/0_stateless/03167_base64_url_functions.sql index 2152002e412..674f1ae498b 100644 --- a/tests/queries/0_stateless/03167_base64_url_functions.sql +++ b/tests/queries/0_stateless/03167_base64_url_functions.sql @@ -1,4 +1,5 @@ -- Tags: no-fasttest +-- no-fasttest because aklomp-base64 library is required -- incorrect number of arguments SELECT base64UrlEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } @@ -9,22 +10,20 @@ SELECT base64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARG SELECT tryBase64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- test with valid inputs -SELECT 'https://clickhouse.com' as original, base64UrlEncode(original) as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; --- encoding differs from base64Encode -SELECT '12?' as original, base64UrlEncode(original) as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; +SELECT 'https://clickhouse.com' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); +SELECT '12?' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); +SELECT 'https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); --- long string -SELECT 'https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode' as original, base64UrlEncode(original) as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; +-- encoded value has no padding +SELECT 'aHR0cHM6Ly9jbGlj' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); +-- encoded value has one-byte padding +SELECT 'aHR0cHM6Ly9jbGlja2g' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); +-- encoded value has two-bytes padding +SELECT 'aHR0cHM6Ly9jbGljaw' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); --- no padding -SELECT 'aHR0cHM6Ly9jbGlj' as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; --- one-byte padding -SELECT 'aHR0cHM6Ly9jbGlja2g' as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; --- two-bytes padding -SELECT 'aHR0cHM6Ly9jbGljaw' as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; +-- test with invalid inputs --- invalid inputs SELECT base64UrlDecode('https://clickhouse.com'); -- { serverError INCORRECT_DATA } SELECT tryBase64UrlDecode('https://clickhouse.com'); SELECT base64UrlDecode('12?'); -- { serverError INCORRECT_DATA } @@ -33,4 +32,5 @@ SELECT base64UrlDecode('aHR0cHM6Ly9jbGlja'); -- { serverError INCORRECT_DATA } SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja'); -- test FixedString argument -SELECT toFixedString('https://clickhouse.com', 22) as original, base64UrlEncode(original) as encoded, base64UrlDecode(encoded) as decoded, tryBase64UrlDecode(encoded) as try_decoded; + +SELECT toFixedString('https://clickhouse.com', 22) AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); From d2be2ce70544564e33d33b33fa38db5d84d082fd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 10 Jun 2024 19:01:48 +0000 Subject: [PATCH 509/856] 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 e4a0f42ab8ffc5e9af469729a6bdceee71fac2c9 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 10 Jun 2024 21:15:22 +0000 Subject: [PATCH 510/856] time_virtual_col_tests: rollback tests --- .../test_storage_azure_blob_storage/test.py | 4 ++-- tests/integration/test_storage_hdfs/test.py | 4 ++-- tests/integration/test_storage_s3/test.py | 8 ++------ ...mn_use_structure_from_insertion_table.reference | 1 + ...al_column_use_structure_from_insertion_table.sh | 13 +++++++++++++ ...mn_use_structure_from_insertion_table.reference | 1 - ...al_column_use_structure_from_insertion_table.sh | 14 -------------- 7 files changed, 20 insertions(+), 25 deletions(-) create mode 100644 tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference create mode 100755 tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh delete mode 100644 tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference delete mode 100755 tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 9f5aef1489c..f836c58ce30 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -758,12 +758,12 @@ def test_read_subcolumns(cluster): ) res = node.query( - f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()), a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv'," + f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv'," f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t0\t3\n" + assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" res = node.query( f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index cda2b8694c6..44c0223e677 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -987,10 +987,10 @@ def test_read_subcolumns(started_cluster): assert res == "2\ttest_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( - f"select x.b.d, _path, x.b, _file, dateDiff('minute', _time, now()), x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\t0\n" + assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 61c6d95f123..09b27fff1e8 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2117,12 +2117,10 @@ def test_read_subcolumns(started_cluster): assert res == "0\troot/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = instance.query( - f"select x.b.d, _path, x.b, _file, dateDiff('minute', _time, now()), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + f"select x.b.d, _path, x.b, _file, x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) - assert ( - res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t0\t42\n" - ) + assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" res = instance.query( f"select a.b.d, _path, a.b, _file, a.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" @@ -2150,8 +2148,6 @@ def test_read_subcolumns(started_cluster): res == "42\t/root/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" ) - logging.info("Some custom logging") - def test_filtering_by_file_or_path(started_cluster): bucket = started_cluster.minio_bucket diff --git a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference new file mode 100644 index 00000000000..35ef86f5339 --- /dev/null +++ b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference @@ -0,0 +1 @@ +1 2 4 diff --git a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh new file mode 100755 index 00000000000..d9e4a2c8f8b --- /dev/null +++ b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv +$CLICKHOUSE_LOCAL -nm -q " +create table test (x UInt64, y UInt32, size UInt64) engine=Memory; +insert into test select c1, c2, _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') settings use_structure_from_insertion_table_in_table_functions=1; +select * from test; +" +rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv diff --git a/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference deleted file mode 100644 index 93acdc34842..00000000000 --- a/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference +++ /dev/null @@ -1 +0,0 @@ -1 2 4 1 1 diff --git a/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh deleted file mode 100755 index ebdda0cc1d3..00000000000 --- a/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh +++ /dev/null @@ -1,14 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -sleep 1 -$CLICKHOUSE_LOCAL -nm -q " -create table test (x UInt64, y UInt32, size UInt64, d32 DateTime32, d64 DateTime64) engine=Memory; -insert into test select c1, c2, _size, _time, _time from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') settings use_structure_from_insertion_table_in_table_functions=1; -select x, y, size, (dateDiff('millisecond', d32, now()) < 4000 AND dateDiff('millisecond', d32, now()) > 0), (dateDiff('second', d64, now()) < 4 AND dateDiff('second', d64, now()) > 0) from test; -" -rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv From 26e82457b9b5f206b7d11ca982f7c8f60ef82d26 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 11 Jun 2024 03:36:26 +0000 Subject: [PATCH 511/856] add setting changes Signed-off-by: Duc Canh Le --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 6734c0dc525..313b5547f4d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -101,6 +101,7 @@ static const std::map Date: Tue, 11 Jun 2024 05:39:24 +0000 Subject: [PATCH 512/856] Test fix --- .../0_stateless/02415_all_new_functions_must_be_documented.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql index cabcd230eb6..544f29e8a7d 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql @@ -3,7 +3,7 @@ SELECT name FROM system.functions WHERE NOT is_aggregate AND origin = 'System' AND alias_to = '' AND length(description) < 10 AND name NOT IN ( 'aes_decrypt_mysql', 'aes_encrypt_mysql', 'decrypt', 'encrypt', - 'base64Decode', 'base64Encode', 'tryBase64Decode', + 'base64Decode', 'base64Encode', 'tryBase64Decode', 'base64UrlDecode', 'base64UrlEncode', 'tryBase64UrlDecode', 'convertCharset', 'detectLanguage', 'detectLanguageMixed', 'geoToH3', From a6fa7993961221381b14ba061bb0697da2203ec2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 11 Jun 2024 08:22:11 +0000 Subject: [PATCH 513/856] Fix other tests. --- src/Storages/StorageMerge.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 6d12876c776..f661fc237b6 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -34,7 +34,6 @@ #include #include #include -#include #include #include #include @@ -1167,13 +1166,6 @@ QueryPipelineBuilderPtr ReadFromMerge::buildPipeline( builder->addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); } - if (builder->getNumStreams() > 1) - { - // It's possible to have many tables read from merge, resize(1) might open too many files at the same time. - // Using concat instead. - builder->addTransform(std::make_shared(builder->getHeader(), builder->getNumStreams())); - } - return builder; } From f27e92c97b0d363f906deea41981176ac61c7bdf Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 11 Jun 2024 10:34:19 +0200 Subject: [PATCH 514/856] Update s3queue.md --- docs/en/engines/table-engines/integrations/s3queue.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index f930fab1805..c29b90f3cd7 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -75,7 +75,7 @@ Possible values: - unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper. - ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. -Default value: `unordered`. +Default value: `ordered` unti 24.5. Starting with 24.6 there is no default value, the setting becomes required to be specified manually. For tables created on earlier versions the default value will remain `Ordered` for compatibility. ### after_processing {#after_processing} @@ -181,6 +181,10 @@ For 'Ordered' mode. Defines a maximum boundary for reschedule interval for a bac Default value: `30000`. +### s3queue_buckets {#buckets} + +For 'Ordered' mode. If there are several replicas of S3Queue table, each working with the same metadata directory in keeper, the value of `s3queue_buckets` needs to be equal to at least the number of replicas. If `s3queue_processing_threads` setting is used as well, it makes sense to increase the value of `s3queue_buckets` setting even futher, as it defines the actual parallelism of `S3Queue` processing. + ## S3-related Settings {#s3-settings} Engine supports all s3 related settings. For more information about S3 settings see [here](../../../engines/table-engines/integrations/s3.md). From 00e58f522ff1a6420aa95e8c2ec360dd9b3d0017 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 11 Jun 2024 10:35:55 +0200 Subject: [PATCH 515/856] Update s3queue.md --- docs/en/engines/table-engines/integrations/s3queue.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index c29b90f3cd7..f72bc79c1e5 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -75,7 +75,7 @@ Possible values: - unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper. - ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. -Default value: `ordered` unti 24.5. Starting with 24.6 there is no default value, the setting becomes required to be specified manually. For tables created on earlier versions the default value will remain `Ordered` for compatibility. +Default value: `ordered` in versions before 24.6. Starting with 24.6 there is no default value, the setting becomes required to be specified manually. For tables created on earlier versions the default value will remain `Ordered` for compatibility. ### after_processing {#after_processing} @@ -183,7 +183,7 @@ Default value: `30000`. ### s3queue_buckets {#buckets} -For 'Ordered' mode. If there are several replicas of S3Queue table, each working with the same metadata directory in keeper, the value of `s3queue_buckets` needs to be equal to at least the number of replicas. If `s3queue_processing_threads` setting is used as well, it makes sense to increase the value of `s3queue_buckets` setting even futher, as it defines the actual parallelism of `S3Queue` processing. +For 'Ordered' mode. Available since `24.6`. If there are several replicas of S3Queue table, each working with the same metadata directory in keeper, the value of `s3queue_buckets` needs to be equal to at least the number of replicas. If `s3queue_processing_threads` setting is used as well, it makes sense to increase the value of `s3queue_buckets` setting even futher, as it defines the actual parallelism of `S3Queue` processing. ## S3-related Settings {#s3-settings} From 46df67d1dec2865aecfbe319532216d9aec670d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 11 Jun 2024 09:07:03 +0000 Subject: [PATCH 516/856] Fix infinite query duration in case of cyclic aliases. --- src/Analyzer/Resolve/ScopeAliases.h | 7 ++++++- .../0_stateless/02896_cyclic_aliases_crash.reference | 1 + tests/queries/0_stateless/02896_cyclic_aliases_crash.sql | 4 ++++ 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Resolve/ScopeAliases.h b/src/Analyzer/Resolve/ScopeAliases.h index baab843988b..830ae72144b 100644 --- a/src/Analyzer/Resolve/ScopeAliases.h +++ b/src/Analyzer/Resolve/ScopeAliases.h @@ -75,7 +75,12 @@ struct ScopeAliases if (jt == transitive_aliases.end()) return {}; - key = &(getKey(jt->second, find_option)); + const auto & new_key = getKey(jt->second, find_option); + /// Ignore potential cyclic aliases. + if (new_key == *key) + return {}; + + key = &new_key; it = alias_map.find(*key); } diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference b/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference index caf11f5c15a..e537236478d 100644 --- a/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference @@ -1,2 +1,3 @@ 1 2 3 1 5 +300 diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql index 5fb628eeb67..5440872e052 100644 --- a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql @@ -30,3 +30,7 @@ WHERE (time_stamp_utc >= toDateTime('2024-04-25 00:00:00')) AND (time_stamp_utc GROUP BY time_stamp_utc ORDER BY Impressions DESC LIMIT 1000; + +drop table test_table; +create table test_table engine MergeTree order by sum as select 100 as sum union all select 200 as sum; +select sum as sum from (select sum(sum) as sum from test_table); From 4baabb16f9e66d70fc960292d533e1ee1cfb3a9a Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 10 Jun 2024 22:43:32 +0000 Subject: [PATCH 517/856] time_virtual_col_tests: tests reintroduced --- .../test_storage_azure_blob_storage/test.py | 21 +++++++++++++++++++ tests/integration/test_storage_hdfs/test.py | 14 +++++++++++++ tests/integration/test_storage_s3/test.py | 17 +++++++++++++++ .../03169_time_virtual_column.reference | 1 + .../0_stateless/03169_time_virtual_column.sh | 12 +++++++++++ 5 files changed, 65 insertions(+) create mode 100644 tests/queries/0_stateless/03169_time_virtual_column.reference create mode 100755 tests/queries/0_stateless/03169_time_virtual_column.sh diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f836c58ce30..e7aa7d052a5 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -790,6 +790,27 @@ def test_read_subcolumns(cluster): assert res == "42\tcont/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" +def test_read_subcolumn_time(cluster): + node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," + f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + ) + + res = node.query( + f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()) < 59, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," + f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert ( + res == "2\tcont/test_subcolumn_time.tsv\t(1,2)\ttest_subcolumn_time.tsv\t1\t3\n" + ) + + def test_read_from_not_existing_container(cluster): node = cluster.instances["node"] query = ( diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 44c0223e677..15d9ee0bb26 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -999,6 +999,20 @@ def test_read_subcolumns(started_cluster): assert res == "42\ttest_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" +def test_read_subcolumn_time(started_cluster): + node = started_cluster.instances["node1"] + + node.query( + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + ) + + res = node.query( + f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()) < 59, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "2\ttest_subcolumn_time.tsv\t(1,2)\ttest_subcolumn_time.tsv\t1\t3\n" + + def test_union_schema_inference_mode(started_cluster): node = started_cluster.instances["node1"] diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 09b27fff1e8..eff066739c0 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2149,6 +2149,23 @@ def test_read_subcolumns(started_cluster): ) +def test_read_subcolumn_time(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + ) + + res = instance.query( + f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()) < 59, a.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert ( + res == "2\troot/test_subcolumn_time.tsv\t(1,2)\ttest_subcolumn_time.tsv\t1\t3\n" + ) + + def test_filtering_by_file_or_path(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] diff --git a/tests/queries/0_stateless/03169_time_virtual_column.reference b/tests/queries/0_stateless/03169_time_virtual_column.reference new file mode 100644 index 00000000000..4482956b706 --- /dev/null +++ b/tests/queries/0_stateless/03169_time_virtual_column.reference @@ -0,0 +1 @@ +4 1 diff --git a/tests/queries/0_stateless/03169_time_virtual_column.sh b/tests/queries/0_stateless/03169_time_virtual_column.sh new file mode 100755 index 00000000000..fef1de8c6f2 --- /dev/null +++ b/tests/queries/0_stateless/03169_time_virtual_column.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv +sleep 1 +$CLICKHOUSE_LOCAL -nm -q " +select _size, (dateDiff('millisecond', _time, now()) < 600000 AND dateDiff('millisecond', _time, now()) > 0) from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv'); +" +rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv From 1519efe7e21384b55780e214a5434dafdd1d1f63 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 11 Jun 2024 11:53:33 +0200 Subject: [PATCH 518/856] Addressed review comments --- src/Parsers/FunctionParameterValuesVisitor.cpp | 2 +- .../queries/0_stateless/03146_parameterized_view_with_date.sql | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Parsers/FunctionParameterValuesVisitor.cpp b/src/Parsers/FunctionParameterValuesVisitor.cpp index e791e07cdfb..eaf28bbbc41 100644 --- a/src/Parsers/FunctionParameterValuesVisitor.cpp +++ b/src/Parsers/FunctionParameterValuesVisitor.cpp @@ -23,7 +23,7 @@ class FunctionParameterValuesVisitor public: explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_, ContextPtr context_) : parameter_values(parameter_values_) - ,context(context_) + , context(context_) { } diff --git a/tests/queries/0_stateless/03146_parameterized_view_with_date.sql b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql index 53022e969ab..2cfadb70b24 100644 --- a/tests/queries/0_stateless/03146_parameterized_view_with_date.sql +++ b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql @@ -9,4 +9,6 @@ create view pv as select * from table_pv where timestamp_field > {timestamp_para select * from pv (timestamp_param=toDateTime('2024-04-01 00:00:01')); +select * from pv (timestamp_param=toDateTime('2024-040')); -- { serverError CANNOT_PARSE_DATETIME } + drop table table_pv; From 61d9280e969ecc6e4cf38adee24090fe13d96112 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 12:10:23 +0200 Subject: [PATCH 519/856] Update src/Storages/MergeTree/MergeTreeDataWriter.cpp Co-authored-by: SmitaRKulkarni --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index a5fbca111f3..1c24fd9525a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -504,7 +504,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( } if (data.getSettings()->optimize_row_order - && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if it the optimization messes up specialized MergeTree engines. + && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if this optimization messes up specialized MergeTree engines. { RowOrderOptimizer::optimize(block, sort_description, perm); perm_ptr = &perm; From a96513ff967706ac9c0de572da005cad6ace1e5e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 12:10:29 +0200 Subject: [PATCH 520/856] Update src/Storages/MergeTree/MergeTreeDataWriter.cpp Co-authored-by: SmitaRKulkarni --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 1c24fd9525a..5c8aa32949d 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -732,7 +732,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( } if (data.getSettings()->optimize_row_order - && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if it the optimization messes up specialized MergeTree engines. + && data.merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary) /// Nobody knows if this optimization messes up specialized MergeTree engines. { RowOrderOptimizer::optimize(block, sort_description, perm); perm_ptr = &perm; From 88fffcd171dfc7a7819ad819fd53050f5690470e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 11 Jun 2024 10:16:16 +0000 Subject: [PATCH 521/856] Fix INTERPOLATE (alias) for remote queries with analyzer. --- src/Analyzer/InterpolateNode.cpp | 16 +++++++++++++--- src/Analyzer/InterpolateNode.h | 7 +++++-- .../03155_analyzer_interpolate.reference | 5 +++++ .../0_stateless/03155_analyzer_interpolate.sql | 3 +++ 4 files changed, 26 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index e4f7e22b803..0de11998d1f 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -10,9 +10,12 @@ namespace DB { -InterpolateNode::InterpolateNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_) +InterpolateNode::InterpolateNode(std::shared_ptr expression_, QueryTreeNodePtr interpolate_expression_) : IQueryTreeNode(children_size) { + if(expression_) + expression_name = expression_->getIdentifier().getFullName(); + children[expression_child_index] = std::move(expression_); children[interpolate_expression_child_index] = std::move(interpolate_expression_); } @@ -41,13 +44,20 @@ void InterpolateNode::updateTreeHashImpl(HashState &, CompareOptions) const QueryTreeNodePtr InterpolateNode::cloneImpl() const { - return std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); + auto cloned = std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); + cloned->expression_name = expression_name; + return cloned; } ASTPtr InterpolateNode::toASTImpl(const ConvertToASTOptions & options) const { auto result = std::make_shared(); - result->column = getExpression()->toAST(options)->getColumnName(); + + if (const auto * identifier = getExpression()->as()) + result->column = identifier->toAST(options)->getColumnName(); + else + result->column = expression_name; + result->children.push_back(getInterpolateExpression()->toAST(options)); result->expr = result->children.back(); diff --git a/src/Analyzer/InterpolateNode.h b/src/Analyzer/InterpolateNode.h index 9269d3924f5..ec493ed8bdd 100644 --- a/src/Analyzer/InterpolateNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -19,7 +19,7 @@ class InterpolateNode final : public IQueryTreeNode { public: /// Initialize interpolate node with expression and interpolate expression - explicit InterpolateNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_); + explicit InterpolateNode(std::shared_ptr expression_, QueryTreeNodePtr interpolate_expression_); /// Get expression to interpolate const QueryTreeNodePtr & getExpression() const @@ -61,6 +61,9 @@ protected: ASTPtr toASTImpl(const ConvertToASTOptions & options) const override; + /// Initial name from column identifier. + std::string expression_name; + private: static constexpr size_t expression_child_index = 0; static constexpr size_t interpolate_expression_child_index = 1; diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.reference b/tests/queries/0_stateless/03155_analyzer_interpolate.reference index 791aaa5b2a2..eade3b45d26 100644 --- a/tests/queries/0_stateless/03155_analyzer_interpolate.reference +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.reference @@ -11,3 +11,8 @@ 5 [5] 5.5 [5] 7 [7] +2 +100500 +18 +26 +34 diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.sql b/tests/queries/0_stateless/03155_analyzer_interpolate.sql index b3c1d233f47..30423cb86ff 100644 --- a/tests/queries/0_stateless/03155_analyzer_interpolate.sql +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.sql @@ -10,3 +10,6 @@ SELECT n, number+5 AS inter FROM ( -- { serverError NOT_AN_AGGREGATE } SELECT toFloat32(number % 10) AS n, number, number*2 AS mn FROM numbers(10) WHERE number % 3 = 1 ) GROUP BY n, inter ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS mn * 2); + +-- https://github.com/ClickHouse/ClickHouse/issues/64636 +select sum(number) as s from remote('127.0.0.{1,2}', numbers(10)) where (intDiv(number, 2) as key) != 1 group by key order by key with fill interpolate (s as 100500); From 5f6904fa38fc976a79cee8cff042d8e25845b619 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 11 Jun 2024 10:23:56 +0000 Subject: [PATCH 522/856] Add a comment. --- src/Analyzer/InterpolateNode.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index 0de11998d1f..4b8797282a2 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -53,6 +53,9 @@ ASTPtr InterpolateNode::toASTImpl(const ConvertToASTOptions & options) const { auto result = std::make_shared(); + /// Interpolate parser supports only identifier node. + /// In case of alias, identifier is replaced to expression, which can't be parsed. + /// In this case, keep original alias name. if (const auto * identifier = getExpression()->as()) result->column = identifier->toAST(options)->getColumnName(); else From d9ff40851ee0e25413c959ad5cf31de244e2550d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 11 Jun 2024 10:55:27 +0000 Subject: [PATCH 523/856] Don't use finalize in release() method --- src/IO/WriteBufferFromFile.cpp | 10 +++++++++- src/IO/WriteBufferFromFileDescriptor.cpp | 9 ++++++++- .../ObjectStorage/HDFS/WriteBufferFromHDFS.cpp | 10 ++-------- src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h | 2 -- .../ObjectStorage/StorageObjectStorageSink.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- 7 files changed, 22 insertions(+), 15 deletions(-) diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index 0ca6c26f08c..d641e553671 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -77,7 +77,15 @@ WriteBufferFromFile::~WriteBufferFromFile() if (fd < 0) return; - finalize(); + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + int err = ::close(fd); /// Everything except for EBADF should be ignored in dtor, since all of /// others (EINTR/EIO/ENOSPC/EDQUOT) could be possible during writing to diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index 813ef0deab9..a758f99458d 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -105,7 +105,14 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor( WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor() { - finalize(); + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } void WriteBufferFromFileDescriptor::finalizeImpl() diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 2c14b38ce01..8277a769a11 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -132,11 +132,11 @@ void WriteBufferFromHDFS::sync() } -void WriteBufferFromHDFS::finalizeImpl() +WriteBufferFromHDFS::~WriteBufferFromHDFS() { try { - next(); + finalize(); } catch (...) { @@ -144,11 +144,5 @@ void WriteBufferFromHDFS::finalizeImpl() } } - -WriteBufferFromHDFS::~WriteBufferFromHDFS() -{ - finalize(); -} - } #endif diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h index 71e6e55addc..e3f0ae96a8f 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h @@ -38,8 +38,6 @@ public: std::string getFileName() const override { return filename; } private: - void finalizeImpl() override; - struct WriteBufferFromHDFSImpl; std::unique_ptr impl; const std::string filename; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 154c2b07251..d13aec4a4f6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -97,7 +97,7 @@ void StorageObjectStorageSink::finalize() void StorageObjectStorageSink::release() { writer.reset(); - write_buf->finalize(); + write_buf.reset(); } PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 83bfcdaf415..16c248f1b7b 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1837,7 +1837,7 @@ private: void release() { writer.reset(); - write_buf->finalize(); + write_buf.reset(); } StorageMetadataPtr metadata_snapshot; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index cd39ca42574..f8424bc3d1b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -623,7 +623,7 @@ void StorageURLSink::finalize() void StorageURLSink::release() { writer.reset(); - write_buf->finalize(); + write_buf.reset(); } class PartitionedStorageURLSink : public PartitionedSink From be154c37b726362b3cde7d0260a82275e2a1faec Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 11 Jun 2024 11:05:04 +0000 Subject: [PATCH 524/856] Refactor and cleanup. --- src/Storages/StorageMerge.cpp | 177 ++++++++++++---------------------- src/Storages/StorageMerge.h | 24 ++--- 2 files changed, 71 insertions(+), 130 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f661fc237b6..92350d4c5bc 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -440,18 +440,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu Names column_names_as_aliases; Aliases aliases; - Names real_column_names = column_names; - if (child_plan.row_policy_data_opt) - child_plan.row_policy_data_opt->extendNames(real_column_names); - - auto modified_query_info = getModifiedQueryInfo(modified_context, table, nested_storage_snaphsot, real_column_names, column_names_as_aliases, aliases); - - auto source_pipeline = buildPipeline( - child_plan.plan, - nested_storage_snaphsot, - modified_query_info, - common_processed_stage, - table); + auto source_pipeline = buildPipeline(child_plan, common_processed_stage); if (source_pipeline && source_pipeline->initialized()) { @@ -569,10 +558,8 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ if (sampling_requested && !storage->supportsSampling()) throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: table {} doesn't support sampling", storage->getStorageID().getNameForLogs()); - res.emplace_back(); - - auto & aliases = res.back().table_aliases; - auto & row_policy_data_opt = res.back().row_policy_data_opt; + Aliases aliases; + RowPolicyDataOpt row_policy_data_opt; auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, modified_context); @@ -651,9 +638,8 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ } - res.back().plan = createPlanForTable( + auto child = createPlanForTable( nested_storage_snaphsot, - aliases, modified_query_info, common_processed_stage, required_max_block_size, @@ -662,17 +648,31 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ row_policy_data_opt, modified_context, current_streams); - res.back().plan.addInterpreterContext(modified_context); + child.plan.addInterpreterContext(modified_context); - // createSources1( - // res.back().plan, - // nested_storage_snaphsot, - // modified_query_info, - // common_processed_stage, - // common_header, - // res.back().table_aliases, - // row_policy_data_opt, - // table); + if (child.plan.isInitialized()) + { + addVirtualColumns(child, modified_query_info, common_processed_stage, table); + + /// Subordinary tables could have different but convertible types, like numeric types of different width. + /// We must return streams with structure equals to structure of Merge table. + convertAndFilterSourceStream(common_header, modified_query_info, nested_storage_snaphsot, aliases, row_policy_data_opt, context, child); + + for (const auto & filter_info : pushed_down_filters) + { + auto filter_step = std::make_unique( + child.plan.getCurrentDataStream(), + filter_info.actions->clone(), + filter_info.column_name, + filter_info.do_remove_column); + + child.plan.addStep(std::move(filter_step)); + } + + child.plan.optimize(QueryPlanOptimizationSettings::fromContext(modified_context)); + } + + res.emplace_back(std::move(child)); } return res; @@ -1031,27 +1031,18 @@ bool recursivelyApplyToReadingSteps(QueryPlan::Node * node, const std::function< return ok; } -void ReadFromMerge::updatePlan( - QueryPlan & plan, - const StorageSnapshotPtr & storage_snapshot_, +void ReadFromMerge::addVirtualColumns( + ChildPlan & child, SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, - const Block & header, - const Aliases & aliases, - const RowPolicyDataOpt & row_policy_data_opt, const StorageWithLockAndName & storage_with_lock) const { - if (!plan.isInitialized()) - return; - - const auto & [database_name, storage, _, table_name] = storage_with_lock; + const auto & [database_name, _, storage, table_name] = storage_with_lock; bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; - auto storage_stage - = storage->getQueryProcessingStage(context, processed_stage, storage_snapshot_, modified_query_info); /// Add virtual columns if we don't already have them. - Block plan_header = plan.getCurrentDataStream().header; + Block plan_header = child.plan.getCurrentDataStream().header; if (allow_experimental_analyzer) { @@ -1061,7 +1052,7 @@ void ReadFromMerge::updatePlan( String table_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_table" : table_alias + "._table"; if (has_database_virtual_column && common_header.has(database_column) - && storage_stage == QueryProcessingStage::FetchColumns && !plan_header.has(database_column)) + && child.stage == QueryProcessingStage::FetchColumns && !plan_header.has(database_column)) { ColumnWithTypeAndName column; column.name = database_column; @@ -1069,13 +1060,13 @@ void ReadFromMerge::updatePlan( column.column = column.type->createColumnConst(0, Field(database_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(plan.getCurrentDataStream(), adding_column_dag); - plan.addStep(std::move(expression_step)); - plan_header = plan.getCurrentDataStream().header; + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + child.plan.addStep(std::move(expression_step)); + plan_header = child.plan.getCurrentDataStream().header; } if (has_table_virtual_column && common_header.has(table_column) - && storage_stage == QueryProcessingStage::FetchColumns && !plan_header.has(table_column)) + && child.stage == QueryProcessingStage::FetchColumns && !plan_header.has(table_column)) { ColumnWithTypeAndName column; column.name = table_column; @@ -1083,9 +1074,9 @@ void ReadFromMerge::updatePlan( column.column = column.type->createColumnConst(0, Field(table_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(plan.getCurrentDataStream(), adding_column_dag); - plan.addStep(std::move(expression_step)); - plan_header = plan.getCurrentDataStream().header; + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + child.plan.addStep(std::move(expression_step)); + plan_header = child.plan.getCurrentDataStream().header; } } else @@ -1098,9 +1089,9 @@ void ReadFromMerge::updatePlan( column.column = column.type->createColumnConst(0, Field(database_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(plan.getCurrentDataStream(), adding_column_dag); - plan.addStep(std::move(expression_step)); - plan_header = plan.getCurrentDataStream().header; + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + child.plan.addStep(std::move(expression_step)); + plan_header = child.plan.getCurrentDataStream().header; } if (has_table_virtual_column && common_header.has("_table") && !plan_header.has("_table")) @@ -1111,53 +1102,30 @@ void ReadFromMerge::updatePlan( column.column = column.type->createColumnConst(0, Field(table_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(plan.getCurrentDataStream(), adding_column_dag); - plan.addStep(std::move(expression_step)); - plan_header = plan.getCurrentDataStream().header; + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + child.plan.addStep(std::move(expression_step)); + plan_header = child.plan.getCurrentDataStream().header; } } - - /// Subordinary tables could have different but convertible types, like numeric types of different width. - /// We must return streams with structure equals to structure of Merge table. - convertAndFilterSourceStream( - header, modified_query_info, storage_snapshot_, aliases, row_policy_data_opt, context, plan, storage_stage); - - for (const auto & filter_info : pushed_down_filters) - { - auto filter_step = std::make_unique( - plan.getCurrentDataStream(), - filter_info.actions->clone(), - filter_info.column_name, - filter_info.do_remove_column); - - plan.addStep(std::move(filter_step)); - } } QueryPipelineBuilderPtr ReadFromMerge::buildPipeline( - QueryPlan & plan, - const StorageSnapshotPtr & storage_snapshot_, - SelectQueryInfo & modified_query_info, - QueryProcessingStage::Enum processed_stage, - const StorageWithLockAndName & storage_with_lock) const + ChildPlan & child, + QueryProcessingStage::Enum processed_stage) const { - if (!plan.isInitialized()) + if (!child.plan.isInitialized()) return nullptr; - const auto & [database_name, storage, _, table_name] = storage_with_lock; - auto storage_stage - = storage->getQueryProcessingStage(context, processed_stage, storage_snapshot_, modified_query_info); - auto optimisation_settings = QueryPlanOptimizationSettings::fromContext(context); /// All optimisations will be done at plans creation optimisation_settings.optimize_plan = false; - auto builder = plan.buildQueryPipeline(optimisation_settings, BuildQueryPipelineSettings::fromContext(context)); + auto builder = child.plan.buildQueryPipeline(optimisation_settings, BuildQueryPipelineSettings::fromContext(context)); if (!builder->initialized()) return builder; bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; - if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) + if (processed_stage > child.stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, @@ -1169,9 +1137,8 @@ QueryPipelineBuilderPtr ReadFromMerge::buildPipeline( return builder; } -QueryPlan ReadFromMerge::createPlanForTable( +ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( const StorageSnapshotPtr & storage_snapshot_, - const Aliases & table_aliases, SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, @@ -1253,20 +1220,7 @@ QueryPlan ReadFromMerge::createPlanForTable( } } - updatePlan( - plan, - storage_snapshot_, - modified_query_info, - common_processed_stage, - common_header, - table_aliases, - row_policy_data_opt, - storage_with_lock); - - if (plan.isInitialized()) - plan.optimize(QueryPlanOptimizationSettings::fromContext(modified_context)); - - return plan; + return ChildPlan{std::move(plan), storage_stage}; } ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter_ptr, @@ -1506,10 +1460,9 @@ void ReadFromMerge::convertAndFilterSourceStream( const Aliases & aliases, const RowPolicyDataOpt & row_policy_data_opt, ContextPtr local_context, - QueryPlan & plan, - QueryProcessingStage::Enum processed_stage) + ChildPlan & child) { - Block before_block_header = plan.getCurrentDataStream().header; + Block before_block_header = child.plan.getCurrentDataStream().header; auto storage_sample_block = snapshot->metadata->getSampleBlock(); auto pipe_columns = before_block_header.getNamesAndTypesList(); @@ -1535,8 +1488,8 @@ void ReadFromMerge::convertAndFilterSourceStream( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); actions_dag->addOrReplaceInOutputs(actions_dag->addAlias(*nodes.front(), alias.name)); - auto expression_step = std::make_unique(plan.getCurrentDataStream(), actions_dag); - plan.addStep(std::move(expression_step)); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), actions_dag); + child.plan.addStep(std::move(expression_step)); } } else @@ -1550,28 +1503,26 @@ void ReadFromMerge::convertAndFilterSourceStream( auto dag = std::make_shared(pipe_columns); auto actions_dag = expression_analyzer.getActionsDAG(true, false); - auto expression_step = std::make_unique(plan.getCurrentDataStream(), actions_dag); - plan.addStep(std::move(expression_step)); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), actions_dag); + child.plan.addStep(std::move(expression_step)); } } ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; if (local_context->getSettingsRef().allow_experimental_analyzer - && (processed_stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) + && (child.stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; if (row_policy_data_opt) - { - row_policy_data_opt->addFilterTransform(plan); - } + row_policy_data_opt->addFilterTransform(child.plan); - auto convert_actions_dag = ActionsDAG::makeConvertingActions(plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), + auto convert_actions_dag = ActionsDAG::makeConvertingActions(child.plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); - auto expression_step = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); - plan.addStep(std::move(expression_step)); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), convert_actions_dag); + child.plan.addStep(std::move(expression_step)); } const ReadFromMerge::StorageListWithLocks & ReadFromMerge::getSelectedTables() diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 42544676bd8..94b34256d02 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -235,8 +235,7 @@ private: struct ChildPlan { QueryPlan plan; - Aliases table_aliases; - RowPolicyDataOpt row_policy_data_opt; + QueryProcessingStage::Enum stage; }; /// Store read plan for each child table. @@ -250,9 +249,8 @@ private: void filterTablesAndCreateChildrenPlans(); - QueryPlan createPlanForTable( + ChildPlan createPlanForTable( const StorageSnapshotPtr & storage_snapshot, - const Aliases & table_aliases, SelectQueryInfo & query_info, QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, @@ -262,22 +260,15 @@ private: ContextMutablePtr modified_context, size_t streams_num) const; - void updatePlan( - QueryPlan & plan, - const StorageSnapshotPtr & storage_snapshot, + void addVirtualColumns( + ChildPlan & child, SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, - const Block & header, - const Aliases & aliases, - const RowPolicyDataOpt & row_policy_data_opt, const StorageWithLockAndName & storage_with_lock) const; QueryPipelineBuilderPtr buildPipeline( - QueryPlan & plan, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & modified_query_info, - QueryProcessingStage::Enum processed_stage, - const StorageWithLockAndName & storage_with_lock) const; + ChildPlan & child, + QueryProcessingStage::Enum processed_stage) const; static void convertAndFilterSourceStream( const Block & header, @@ -286,8 +277,7 @@ private: const Aliases & aliases, const RowPolicyDataOpt & row_policy_data_opt, ContextPtr context, - QueryPlan & plan, - QueryProcessingStage::Enum processed_stage); + ChildPlan & child); StorageMerge::StorageListWithLocks getSelectedTables( ContextPtr query_context, From 2d2ebc918a172dfe76968210afa3b5cfb2cbfe96 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 11 Jun 2024 14:42:48 +0200 Subject: [PATCH 525/856] Update odbc-bridge.md --- docs/en/operations/utilities/odbc-bridge.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/utilities/odbc-bridge.md b/docs/en/operations/utilities/odbc-bridge.md index abb8860880e..eb849c6b6ae 100644 --- a/docs/en/operations/utilities/odbc-bridge.md +++ b/docs/en/operations/utilities/odbc-bridge.md @@ -18,7 +18,7 @@ This tool works via HTTP, not via pipes, shared memory, or TCP because: However it can be used as standalone tool from command line with the following parameters in POST-request URL: - `connection_string` -- ODBC connection string. -- `columns` -- columns in ClickHouse NamesAndTypesList format, name in backticks, +- `sample_block` -- columns description in ClickHouse NamesAndTypesList format, name in backticks, type as string. Name and type are space separated, rows separated with newline. - `max_block_size` -- optional parameter, sets maximum size of single block. From fdc2f156b1e472a81acfc8f34124fe5c396f8144 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 11 Jun 2024 12:50:16 +0000 Subject: [PATCH 526/856] time_virtual_col_tests: just one column in test_read_subcolumn_time --- .../test_storage_azure_blob_storage/test.py | 10 +++----- tests/integration/test_storage_hdfs/test.py | 6 ++--- tests/integration/test_storage_s3/test.py | 25 ++++++++----------- 3 files changed, 17 insertions(+), 24 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e7aa7d052a5..d986c1f9746 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -797,18 +797,16 @@ def test_read_subcolumn_time(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," - f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + f" 'a UInt32') select (42)", ) res = node.query( - f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()) < 59, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv'," + f"select a, dateDiff('minute', _time, now()) < 59 from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv'," f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," - f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f" 'a UInt32')" ) - assert ( - res == "2\tcont/test_subcolumn_time.tsv\t(1,2)\ttest_subcolumn_time.tsv\t1\t3\n" - ) + assert res == "42\t1\n" def test_read_from_not_existing_container(cluster): diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 15d9ee0bb26..47d8f44c0b7 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1003,14 +1003,14 @@ def test_read_subcolumn_time(started_cluster): node = started_cluster.instances["node1"] node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a UInt32') select (42)" ) res = node.query( - f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()) < 59, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select a, dateDiff('minute', _time, now()) < 59 from hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a UInt32')" ) - assert res == "2\ttest_subcolumn_time.tsv\t(1,2)\ttest_subcolumn_time.tsv\t1\t3\n" + assert res == "42\t1\n" def test_union_schema_inference_mode(started_cluster): diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index eff066739c0..8a5bac8b392 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -553,16 +553,13 @@ def test_multipart(started_cluster, maybe_auth, positive): assert csv_data == get_s3_file_content(started_cluster, bucket, filename) # select uploaded data from many threads - select_query = ( - "select sum(column1), sum(column2), sum(column3) " - "from s3('http://{host}:{port}/{bucket}/{filename}', {auth}'CSV', '{table_format}')".format( - host=started_cluster.minio_redirect_host, - port=started_cluster.minio_redirect_port, - bucket=bucket, - filename=filename, - auth=maybe_auth, - table_format=table_format, - ) + select_query = "select sum(column1), sum(column2), sum(column3) " "from s3('http://{host}:{port}/{bucket}/{filename}', {auth}'CSV', '{table_format}')".format( + host=started_cluster.minio_redirect_host, + port=started_cluster.minio_redirect_port, + bucket=bucket, + filename=filename, + auth=maybe_auth, + table_format=table_format, ) try: select_result = run_query( @@ -2154,16 +2151,14 @@ def test_read_subcolumn_time(started_cluster): instance = started_cluster.instances["dummy"] instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a UInt32') select (42)" ) res = instance.query( - f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()) < 59, a.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select a, dateDiff('minute', _time, now()) < 59 from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a UInt32')" ) - assert ( - res == "2\troot/test_subcolumn_time.tsv\t(1,2)\ttest_subcolumn_time.tsv\t1\t3\n" - ) + assert res == "42\t1\n" def test_filtering_by_file_or_path(started_cluster): From 5aa9389f851a9ae9c5a2171932fa8971d478c5ac Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 11 Jun 2024 15:13:36 +0200 Subject: [PATCH 527/856] Add failing test, setting and docuumentation --- docs/en/interfaces/formats.md | 1 + .../operations/settings/settings-formats.md | 32 + src/Core/Settings.h | 3 +- src/Core/SettingsChangesHistory.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + ...9_display_column_names_in_footer.reference | 1002 +++++++++++++++++ .../03169_display_column_names_in_footer.sql | 2 + 8 files changed, 1042 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03169_display_column_names_in_footer.reference create mode 100644 tests/queries/0_stateless/03169_display_column_names_in_footer.sql diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 3de416ae64d..b292c669047 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1490,6 +1490,7 @@ Differs from [PrettySpaceNoEscapes](#prettyspacenoescapes) in that up to 10,000 - [output_format_pretty_color](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_color) - use ANSI escape sequences to paint colors in Pretty formats. Default value - `true`. - [output_format_pretty_grid_charset](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_grid_charset) - Charset for printing grid borders. Available charsets: ASCII, UTF-8. Default value - `UTF-8`. - [output_format_pretty_row_numbers](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_row_numbers) - Add row numbers before each row for pretty output format. Default value - `true`. +- [output_format_pretty_display_column_names_footer](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_column_names_footer) - Display column names in the footer if there are more than 1000 rows. Default value - `false`. ## RowBinary {#rowbinary} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 6aae1ea62e5..c380045f085 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1706,6 +1706,38 @@ Result: └────────────┘ ``` +## output_format_pretty_display_column_names_footer + +Display column names in the footer if there are 999 or more rows. + +Possible values: + +- 0 — No column names are displayed in the footer. +- 1 — Column names are displayed in the footer if row count is greater than or equal to 999. + +**Example** + +Query: + +```sql +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 1000); +``` + +Result: + +```response + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + ... + 999. │ 998 │ UInt64 │ +1000. │ 999 │ UInt64 │ + └─number─┴─toTypeName(number)─┘ +``` + + + ## Template format settings {#template-format-settings} ### format_template_resultset {#format_template_resultset} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f99243e285..fa2994cb98c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1139,7 +1139,8 @@ class IColumn; M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \ M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \ M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \ - M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ + M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ + M(UInt64 output_format_pretty_display_column_names_footer, false, "Display column names in the footer if there are 999 or more rows.", 0) \ M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \ M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \ M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b447421671e..6af7980547f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -105,6 +105,7 @@ static const std::map Date: Tue, 11 Jun 2024 14:08:16 +0000 Subject: [PATCH 528/856] Remove upstream abseil repository --- .gitmodules | 3 --- contrib/abseil-cpp | 1 - 2 files changed, 4 deletions(-) delete mode 160000 contrib/abseil-cpp diff --git a/.gitmodules b/.gitmodules index 28696428e8c..a6ad00e434b 100644 --- a/.gitmodules +++ b/.gitmodules @@ -161,9 +161,6 @@ [submodule "contrib/xz"] path = contrib/xz url = https://github.com/xz-mirror/xz -[submodule "contrib/abseil-cpp"] - path = contrib/abseil-cpp - url = https://github.com/abseil/abseil-cpp [submodule "contrib/dragonbox"] path = contrib/dragonbox url = https://github.com/ClickHouse/dragonbox diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp deleted file mode 160000 index 696b32788ca..00000000000 --- a/contrib/abseil-cpp +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 696b32788ca887881547380530926314c521ea7d From 182f85b25cfe2ac4589d4b3edc4d623e994b1932 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Jun 2024 16:09:31 +0200 Subject: [PATCH 529/856] Test if unit-tests can be decreased --- docker/images.json | 10 +++++----- docker/test/unit/Dockerfile | 4 +--- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/docker/images.json b/docker/images.json index 7439517379b..716b76ee217 100644 --- a/docker/images.json +++ b/docker/images.json @@ -41,8 +41,7 @@ "docker/test/stateless": { "name": "clickhouse/stateless-test", "dependent": [ - "docker/test/stateful", - "docker/test/unit" + "docker/test/stateful" ] }, "docker/test/stateful": { @@ -122,15 +121,16 @@ "docker/test/base": { "name": "clickhouse/test-base", "dependent": [ + "docker/test/clickbench", "docker/test/fuzzer", - "docker/test/libfuzzer", "docker/test/integration/base", "docker/test/keeper-jepsen", + "docker/test/libfuzzer", "docker/test/server-jepsen", "docker/test/sqllogic", "docker/test/sqltest", - "docker/test/clickbench", - "docker/test/stateless" + "docker/test/stateless", + "docker/test/unit" ] }, "docker/test/integration/kerberized_hadoop": { diff --git a/docker/test/unit/Dockerfile b/docker/test/unit/Dockerfile index cf5ba1eec7f..af44dc930b2 100644 --- a/docker/test/unit/Dockerfile +++ b/docker/test/unit/Dockerfile @@ -1,9 +1,7 @@ # rebuild in #33610 # docker build -t clickhouse/unit-test . ARG FROM_TAG=latest -FROM clickhouse/stateless-test:$FROM_TAG - -RUN apt-get install gdb +FROM clickhouse/test-base:$FROM_TAG COPY run.sh / CMD ["/bin/bash", "/run.sh"] From e5dcf75968b44a10e521bd9c1c106621a56ec7cb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 14:09:56 +0000 Subject: [PATCH 530/856] Add forked abseil submodule back --- .gitmodules | 3 +++ contrib/abseil-cpp | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/abseil-cpp diff --git a/.gitmodules b/.gitmodules index a6ad00e434b..6d64c52ce00 100644 --- a/.gitmodules +++ b/.gitmodules @@ -161,6 +161,9 @@ [submodule "contrib/xz"] path = contrib/xz url = https://github.com/xz-mirror/xz +[submodule "abseil"] + path = contrib/abseil-cpp + url = https://github.com/ClickHouse/abseil-cpp.git [submodule "contrib/dragonbox"] path = contrib/dragonbox url = https://github.com/ClickHouse/dragonbox diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp new file mode 160000 index 00000000000..3916ba76a98 --- /dev/null +++ b/contrib/abseil-cpp @@ -0,0 +1 @@ +Subproject commit 3916ba76a98d3082414a10977e10bdebfdf3b177 From af83bc92ced8c08325924b5a77bfad210eb48149 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 14:16:38 +0000 Subject: [PATCH 531/856] Switch to same HEAD as before but with s390x-breaking commit reverted --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 3916ba76a98..a3c4dd3e77f 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 3916ba76a98d3082414a10977e10bdebfdf3b177 +Subproject commit a3c4dd3e77f28b526efbb0eb394b72e29c633936 From 3680ff07ead49abf2af18367f450aa1cedfbca13 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 11 Jun 2024 16:34:53 +0200 Subject: [PATCH 532/856] Analyzer: Fix AggregateFunctionsArithmericOperationsPass --- .../Passes/AggregateFunctionsArithmericOperationsPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp index 9153bc4eca2..e6798a792dd 100644 --- a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -51,7 +51,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void leaveImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions) return; From 1c5e935dfa380fbf7a474810fc8594f3227fdbc3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 11 Jun 2024 16:38:19 +0200 Subject: [PATCH 533/856] Small fix for 02340_parts_refcnt_mergetree https://s3.amazonaws.com/clickhouse-test-reports/61112/f8e3e95b97920c4bd9a21101a2d664e9b3ed60e8/stateless_tests__debug__[1_5].html --- 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 62b732a1fa97144abb5148dd885cc3490a5f2686 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 11 Jun 2024 14:41:23 +0000 Subject: [PATCH 534/856] Remove unused code. --- src/Interpreters/InterpreterSelectQuery.cpp | 47 --------------------- src/Planner/PlannerJoinTree.cpp | 1 - src/Storages/SelectQueryInfo.h | 4 -- src/Storages/StorageMerge.cpp | 2 - 4 files changed, 54 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f8f3867dfd4..6046c5ca34d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2372,49 +2372,6 @@ UInt64 InterpreterSelectQuery::maxBlockSizeByLimit() const return 0; } -/** Storages can rely that filters that for storage will be available for analysis before - * plan is fully constructed and optimized. - * - * StorageMerge common header calculation and prewhere push-down relies on this. - * - * This is similar to Planner::collectFiltersForAnalysis - */ -void collectFiltersForAnalysis( - const ASTPtr & query_ptr, - const ContextPtr & query_context, - const StorageSnapshotPtr & storage_snapshot, - const SelectQueryOptions & options, - SelectQueryInfo & query_info) -{ - auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); - - auto dummy = std::make_shared( - storage_snapshot->storage.getStorageID(), ColumnsDescription(storage_snapshot->getColumns(get_column_options)), storage_snapshot); - - QueryPlan query_plan; - InterpreterSelectQuery(query_ptr, query_context, dummy, dummy->getInMemoryMetadataPtr(), options).buildQueryPlan(query_plan); - - auto optimization_settings = QueryPlanOptimizationSettings::fromContext(query_context); - query_plan.optimize(optimization_settings); - - std::vector nodes_to_process; - nodes_to_process.push_back(query_plan.getRootNode()); - - while (!nodes_to_process.empty()) - { - const auto * node_to_process = nodes_to_process.back(); - nodes_to_process.pop_back(); - nodes_to_process.insert(nodes_to_process.end(), node_to_process->children.begin(), node_to_process->children.end()); - - auto * read_from_dummy = typeid_cast(node_to_process->step.get()); - if (!read_from_dummy) - continue; - - query_info.filter_actions_dag = read_from_dummy->getFilterActionsDAG(); - query_info.optimized_prewhere_info = read_from_dummy->getPrewhereInfo(); - } -} - void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan) { auto & query = getSelectQuery(); @@ -2544,10 +2501,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc } else if (storage) { - // if (shouldMoveToPrewhere() && settings.query_plan_optimize_prewhere && settings.query_plan_enable_optimizations - // && typeid_cast(storage.get())) - // collectFiltersForAnalysis(query_ptr, context, storage_snapshot, options, query_info); - /// Table. if (max_streams == 0) max_streams = 1; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1b2a55a50b0..851603e805a 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -643,7 +643,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions(); - table_expression_query_info.optimized_prewhere_info = table_expression_data.getPrewhereInfo(); table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; size_t max_streams = settings.max_threads; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 11e2a2fc5e7..6901b6cb5ff 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -208,10 +208,6 @@ struct SelectQueryInfo bool need_aggregate = false; PrewhereInfoPtr prewhere_info; - /// Generated by pre-run optimization with StorageDummy. - /// Currently it's used to support StorageMerge PREWHERE optimization. - PrewhereInfoPtr optimized_prewhere_info; - /// If query has aggregate functions bool has_aggregates = false; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 92350d4c5bc..ed3f43367dd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -888,8 +888,6 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo const StorageID current_storage_id = storage->getStorageID(); SelectQueryInfo modified_query_info = query_info; - if (modified_query_info.optimized_prewhere_info && !modified_query_info.prewhere_info) - modified_query_info.prewhere_info = modified_query_info.optimized_prewhere_info; if (modified_query_info.planner_context) modified_query_info.planner_context = std::make_shared(modified_context, modified_query_info.planner_context); From 66ac187b7cf34006f0067a88d13be71b43b271ab Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 11 Jun 2024 16:42:39 +0200 Subject: [PATCH 535/856] fix test --- tests/queries/0_stateless/02922_server_exit_code.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02922_server_exit_code.sh b/tests/queries/0_stateless/02922_server_exit_code.sh index 60049902410..ded0dc4763f 100755 --- a/tests/queries/0_stateless/02922_server_exit_code.sh +++ b/tests/queries/0_stateless/02922_server_exit_code.sh @@ -7,6 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We will check that the server's exit code corresponds to the exception code if it was terminated after exception. # In this example, we provide an invalid path to the server's config, ignore its logs and check the exit code. -# The exception code is 400 = CANNOT_STAT, so the exit code will be 400 % 256. +# The exception code is 76 = CANNOT_OPEN_FILE, so the exit code will be 76 % 256. -${CLICKHOUSE_SERVER_BINARY} -- --path /dev/null 2>/dev/null; [[ "$?" == "$((400 % 256))" ]] && echo 'Ok' || echo 'Fail' +${CLICKHOUSE_SERVER_BINARY} -- --path /dev/null 2>/dev/null; [[ "$?" == "$((76 % 256))" ]] && echo 'Ok' || echo 'Fail' From c795158ea9b1e32ccccba88a13bdc32622fa7a52 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 11 Jun 2024 14:45:00 +0000 Subject: [PATCH 536/856] time_virtual_col_tests: make black happy --- tests/integration/test_storage_s3/test.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 8a5bac8b392..781b68036e6 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -553,13 +553,16 @@ def test_multipart(started_cluster, maybe_auth, positive): assert csv_data == get_s3_file_content(started_cluster, bucket, filename) # select uploaded data from many threads - select_query = "select sum(column1), sum(column2), sum(column3) " "from s3('http://{host}:{port}/{bucket}/{filename}', {auth}'CSV', '{table_format}')".format( - host=started_cluster.minio_redirect_host, - port=started_cluster.minio_redirect_port, - bucket=bucket, - filename=filename, - auth=maybe_auth, - table_format=table_format, + select_query = ( + "select sum(column1), sum(column2), sum(column3) " + "from s3('http://{host}:{port}/{bucket}/{filename}', {auth}'CSV', '{table_format}')".format( + host=started_cluster.minio_redirect_host, + port=started_cluster.minio_redirect_port, + bucket=bucket, + filename=filename, + auth=maybe_auth, + table_format=table_format, + ) ) try: select_result = run_query( From 9f54007eb80e237b99001337767454a98fd8d6a3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 11 Jun 2024 14:47:35 +0000 Subject: [PATCH 537/856] Fix style. --- src/Analyzer/InterpolateNode.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index 4b8797282a2..97dc79f565b 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -13,7 +13,7 @@ namespace DB InterpolateNode::InterpolateNode(std::shared_ptr expression_, QueryTreeNodePtr interpolate_expression_) : IQueryTreeNode(children_size) { - if(expression_) + if (expression_) expression_name = expression_->getIdentifier().getFullName(); children[expression_child_index] = std::move(expression_); From e0668ab8fa1fd90fc88c6d1088230279b6fb28b7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 11 Jun 2024 17:03:54 +0200 Subject: [PATCH 538/856] write more tests --- src/Interpreters/InterpreterCreateQuery.cpp | 3 +-- src/Storages/ColumnsDescription.cpp | 6 +++++- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ++++++ src/Storages/StatisticsDescription.cpp | 4 ++-- src/Storages/StatisticsDescription.h | 2 +- .../test_manipulate_statistics/test.py | 18 ++++++++++++++++++ 6 files changed, 33 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a51d3e6dade..b2d65662eaa 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -680,8 +680,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( { if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics) throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistics is now disabled. Turn on allow_experimental_statistics"); - column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl); - column.statistics.data_type = column.type; + column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl, column.type); } if (col_decl.ttl) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 556f8a6e42d..c07583cd39d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -209,7 +209,11 @@ void ColumnDescription::readText(ReadBuffer & buf) settings = col_ast->settings->as().changes; if (col_ast->statistics_desc) - statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast); + { + statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast, type); + /// every column has name `x` here, so we have to set the name manually. + statistics.column_name = name; + } } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 8e304936747..0dbddf05697 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -471,7 +471,13 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( ColumnsStatistics statistics; if (context->getSettingsRef().materialize_statistics_on_insert) + { + for (auto col : metadata_snapshot->getColumns()) + LOG_INFO(log, "column col {} stats {}", col.name, col.statistics.column_name); statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()); + for (auto stats : statistics) + LOG_INFO(log, "writing stats {}", stats->columnName()); + } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index fc06c66b50e..f10fb78f933 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -169,7 +169,7 @@ std::vector ColumnStatisticsDescription::fromAST(co return result; } -ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column) +ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type) { const auto & stat_type_list_ast = column.statistics_desc->as().arguments; if (stat_type_list_ast->children.empty()) @@ -185,7 +185,7 @@ ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(c throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type); stats.types_to_desc.emplace(stat.type, std::move(stat)); } - + stats.data_type = data_type; return stats; } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 59ad8944850..4862fb79d45 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -55,7 +55,7 @@ struct ColumnStatisticsDescription ASTPtr getAST() const; static std::vector fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); - static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column); + static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type); using StatisticsTypeDescMap = std::map; StatisticsTypeDescMap types_to_desc; diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 53ab9682ad2..0ce90731e8d 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -155,3 +155,21 @@ def test_replicated_table_ddl(started_cluster): node2.query("SHOW CREATE TABLE test_stat") == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" ) + + node2.query("insert into test_stat values(1,2,3), (2,3,4)"); + # check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True) + node1.query("ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"}) + assert (node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n") + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "c", False) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "d", True) + node1.query("ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"}) + node1.query("ALTER TABLE test_stat ADD STATISTICS b type tdigest", settings={"alter_sync": "2"}) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "b", False) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "d", False) + node1.query("ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"}) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True) From 39914d848da11ba0d14cf345fb670667a1d4cde9 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Tue, 11 Jun 2024 14:54:50 +0000 Subject: [PATCH 539/856] Docs updated, __msan_unpoison usage fixed --- .../functions/string-functions.md | 170 ++++++++++++++++-- src/Functions/FunctionBase64Conversion.h | 18 +- src/Functions/base64Decode.cpp | 10 +- src/Functions/base64Encode.cpp | 10 +- src/Functions/base64UrlDecode.cpp | 21 +++ src/Functions/base64UrlEncode.cpp | 21 +++ src/Functions/tryBase64Decode.cpp | 10 +- src/Functions/tryBase64UrlDecode.cpp | 21 +++ ...5_all_new_functions_must_be_documented.sql | 1 - 9 files changed, 251 insertions(+), 31 deletions(-) create mode 100644 src/Functions/base64UrlDecode.cpp create mode 100644 src/Functions/base64UrlEncode.cpp create mode 100644 src/Functions/tryBase64UrlDecode.cpp diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 6c8f09e74ce..7a4b3d01b97 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1136,41 +1136,153 @@ SELECT tryBase58Decode('3dc8KtHrwM') as res, tryBase58Decode('invalid') as res_i ## base64Encode -Encodes a String or FixedString as base64. +Encodes a String or FixedString as base64, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-4). Alias: `TO_BASE64`. +**Syntax** + +```sql +base64Encode(plaintext) +``` + +**Arguments** + +- `plaintext` — [String](../data-types/string.md) column or constant. + +**Returned value** + +- A string containing the encoded value of the argument. + +**Example** + +``` sql +SELECT base64Encode('clickhouse'); +``` + +Result: + +```result +┌─base64Encode('clickhouse')─┐ +│ Y2xpY2tob3VzZQ== │ +└────────────────────────────┘ +``` + ## base64UrlEncode -Encodes an URL (String or FixedString) as base64 according to [RFC 4648](https://tools.ietf.org/html/rfc4648). +Encodes an URL (String or FixedString) as base64 with URL-specific modifications, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-5). + +**Syntax** + +```sql +base64UrlEncode(url) +``` + +**Arguments** + +- `url` — [String](../data-types/string.md) column or constant. + +**Returned value** + +- A string containing the encoded value of the argument. + +**Example** + +``` sql +SELECT base64UrlEncode('https://clickhouse.com'); +``` + +Result: + +```result +┌─base64UrlEncode('https://clickhouse.com')─┐ +│ aHR0cDovL2NsaWNraG91c2UuY29t │ +└───────────────────────────────────────────┘ +``` ## base64Decode -Decodes a base64-encoded String or FixedString. Throws an exception in case of error. +Accepts a String and decodes it from base64, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-4). Throws an exception in case of an error. Alias: `FROM_BASE64`. +**Syntax** + +```sql +base64Decode(encoded) +``` + +**Arguments** + +- `encoded` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value, an exception is thrown. + +**Returned value** + +- A string containing the decoded value of the argument. + +**Example** + +``` sql +SELECT base64Decode('Y2xpY2tob3VzZQ=='); +``` + +Result: + +```result +┌─base64Decode('Y2xpY2tob3VzZQ==')─┐ +│ clickhouse │ +└──────────────────────────────────┘ +``` + ## base64UrlDecode -Decodes a base64-encoded URL (String or FixedString) according to [RFC 4648](https://tools.ietf.org/html/rfc4648). Throws an exception in case of error. +Accepts a base64-encoded URL and decodes it from base64 with URL-specific modifications, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-5). Throws an exception in case of an error. + +**Syntax** + +```sql +base64UrlDecode(encodedUrl) +``` + +**Arguments** + +- `encodedUrl` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, an exception is thrown. + +**Returned value** + +- A string containing the decoded value of the argument. + +**Example** + +``` sql +SELECT base64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t'); +``` + +Result: + +```result +┌─base64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t')─┐ +│ https://clickhouse.com │ +└─────────────────────────────────────────────────┘ +``` ## tryBase64Decode Like `base64Decode` but returns an empty string in case of error. -## tryBase64UrlDecode - -Like `base64UrlDecode` but returns an empty string in case of error. - **Syntax** ```sql tryBase64Decode(encoded) ``` -**Parameters** +**Arguments** -- `encoded`: [String](../data-types/string.md) column or constant. If the string is not a valid Base58-encoded value, returns an empty string in case of error. +- `encoded`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value, returns an empty string. + +**Returned value** + +- A string containing the decoded value of the argument. **Examples** @@ -1181,9 +1293,41 @@ SELECT tryBase64Decode('RW5jb2RlZA==') as res, tryBase64Decode('invalid') as res ``` ```response -┌─res─────┬─res_invalid─┐ -│ Encoded │ │ -└─────────┴─────────────┘ +┌─res────────┬─res_invalid─┐ +│ clickhouse │ │ +└────────────┴─────────────┘ +``` + +## tryBase64UrlDecode + +Like `base64UrlDecode` but returns an empty string in case of error. + +**Syntax** + +```sql +tryBase64UrlDecode(encodedUrl) +``` + +**Parameters** + +- `encodedUrl`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string. + +**Returned value** + +- A string containing the decoded value of the argument. + +**Examples** + +Query: + +```sql +SELECT tryBase64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t') as res, tryBase64Decode('aHR0cHM6Ly9jbGlja') as res_invalid; +``` + +```response +┌─res────────────────────┬─res_invalid─┐ +│ https://clickhouse.com │ │ +└────────────────────────┴─────────────┘ ``` ## endsWith {#endswith} diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index e569a351591..05914be3837 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -33,7 +33,7 @@ inline std::string preprocessBase64Url(std::string_view src) std::string padded_src; padded_src.reserve(src.size() + 3); - // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 + // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#section-5 for (auto s : src) { switch (s) @@ -57,7 +57,7 @@ inline std::string preprocessBase64Url(std::string_view src) case 0: break; // no padding needed case 1: - padded_src.append("==="); // this case is impossible to occur, however, we'll insert padding anyway + padded_src.append("==="); // this case is impossible to occur with valid base64-URL encoded input, however, we'll insert padding anyway break; case 2: padded_src.append("=="); // two bytes padding @@ -72,7 +72,7 @@ inline std::string preprocessBase64Url(std::string_view src) inline size_t postprocessBase64Url(UInt8 * dst, size_t out_len) { - // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#page-7 + // Do symbol substitution as described in https://datatracker.ietf.org/doc/html/rfc4648#section-5 for (size_t i = 0; i < out_len; ++i) { switch (dst[i]) @@ -107,6 +107,10 @@ struct Base64Encode size_t outlen = 0; base64_encode(src.data(), src.size(), reinterpret_cast(dst), &outlen, 0); + /// Base64 library is using AVX-512 with some shuffle operations. + /// Memory sanitizer doesn't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle. + __msan_unpoison(dst, outlen); + if constexpr (variant == Base64Variant::Url) outlen = postprocessBase64Url(dst, outlen); @@ -242,10 +246,6 @@ private: const size_t src_length = src_offsets[row] - src_offset_prev - 1; const size_t outlen = Func::perform({src, src_length}, dst_pos); - /// Base64 library is using AVX-512 with some shuffle operations. - /// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle. - __msan_unpoison(dst_pos, outlen); - src += src_length + 1; dst_pos += outlen; *dst_pos = '\0'; @@ -280,10 +280,6 @@ private: { const auto outlen = Func::perform({src, src_n}, dst_pos); - /// Base64 library is using AVX-512 with some shuffle operations. - /// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle. - __msan_unpoison(dst_pos, outlen); - src += src_n; dst_pos += outlen; *dst_pos = '\0'; diff --git a/src/Functions/base64Decode.cpp b/src/Functions/base64Decode.cpp index 2c0cf27c592..50278c4b0b2 100644 --- a/src/Functions/base64Decode.cpp +++ b/src/Functions/base64Decode.cpp @@ -7,8 +7,14 @@ namespace DB { REGISTER_FUNCTION(Base64Decode) { - factory.registerFunction>>(); - factory.registerFunction>>(); + FunctionDocumentation::Description description = R"(Accepts a String and decodes it from base64, according to RFC 4648 (https://datatracker.ietf.org/doc/html/rfc4648#section-4). Throws an exception in case of an error. Alias: FROM_BASE64.)"; + FunctionDocumentation::Syntax syntax = "base64Decode(encoded)"; + FunctionDocumentation::Arguments arguments = {{"encoded", "String column or constant. If the string is not a valid Base64-encoded value, an exception is thrown."}}; + FunctionDocumentation::ReturnedValue returned_value = "A string containing the decoded value of the argument."; + FunctionDocumentation::Examples examples = {{"Example", "SELECT base64Decode('Y2xpY2tob3VzZQ==')", "clickhouse"}}; + FunctionDocumentation::Categories categories = {"String encoding"}; + + factory.registerFunction>>({description, syntax, arguments, returned_value, examples, categories}); /// MySQL compatibility alias. factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive); diff --git a/src/Functions/base64Encode.cpp b/src/Functions/base64Encode.cpp index 07ca28d6a87..d6e63c38a4c 100644 --- a/src/Functions/base64Encode.cpp +++ b/src/Functions/base64Encode.cpp @@ -7,8 +7,14 @@ namespace DB { REGISTER_FUNCTION(Base64Encode) { - factory.registerFunction>>(); - factory.registerFunction>>(); + FunctionDocumentation::Description description = R"(Encodes a String as base64, according to RFC 4648 (https://datatracker.ietf.org/doc/html/rfc4648#section-4). Alias: TO_BASE64.)"; + FunctionDocumentation::Syntax syntax = "base64Encode(plaintext)"; + FunctionDocumentation::Arguments arguments = {{"plaintext", "String column or constant."}}; + FunctionDocumentation::ReturnedValue returned_value = "A string containing the encoded value of the argument."; + FunctionDocumentation::Examples examples = {{"Example", "SELECT base64Encode('clickhouse')", "Y2xpY2tob3VzZQ=="}}; + FunctionDocumentation::Categories categories = {"String encoding"}; + + factory.registerFunction>>({description, syntax, arguments, returned_value, examples, categories}); /// MySQL compatibility alias. factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive); diff --git a/src/Functions/base64UrlDecode.cpp b/src/Functions/base64UrlDecode.cpp new file mode 100644 index 00000000000..59975d8f9d1 --- /dev/null +++ b/src/Functions/base64UrlDecode.cpp @@ -0,0 +1,21 @@ +#include + +#if USE_BASE64 +#include + +namespace DB +{ +REGISTER_FUNCTION(Base64UrlDecode) +{ + FunctionDocumentation::Description description = R"(Accepts a base64-encoded URL and decodes it from base64 with URL-specific modifications, according to RFC 4648 (https://datatracker.ietf.org/doc/html/rfc4648#section-5).)"; + FunctionDocumentation::Syntax syntax = "base64UrlDecode(encodedUrl)"; + FunctionDocumentation::Arguments arguments = {{"encodedUrl", "String column or constant. If the string is not a valid Base64-encoded value, an exception is thrown."}}; + FunctionDocumentation::ReturnedValue returned_value = "A string containing the decoded value of the argument."; + FunctionDocumentation::Examples examples = {{"Example", "SELECT base64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t')", "https://clickhouse.com"}}; + FunctionDocumentation::Categories categories = {"String encoding"}; + + factory.registerFunction>>({description, syntax, arguments, returned_value, examples, categories}); +} +} + +#endif diff --git a/src/Functions/base64UrlEncode.cpp b/src/Functions/base64UrlEncode.cpp new file mode 100644 index 00000000000..05d50170c14 --- /dev/null +++ b/src/Functions/base64UrlEncode.cpp @@ -0,0 +1,21 @@ +#include + +#if USE_BASE64 +#include + +namespace DB +{ +REGISTER_FUNCTION(Base64UrlEncode) +{ + FunctionDocumentation::Description description = R"(Encodes an URL (String or FixedString) as base64 with URL-specific modifications, according to RFC 4648 (https://datatracker.ietf.org/doc/html/rfc4648#section-5).)"; + FunctionDocumentation::Syntax syntax = "base64UrlEncode(url)"; + FunctionDocumentation::Arguments arguments = {{"url", "String column or constant."}}; + FunctionDocumentation::ReturnedValue returned_value = "A string containing the encoded value of the argument."; + FunctionDocumentation::Examples examples = {{"Example", "SELECT base64UrlEncode('https://clickhouse.com')", "aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ"}}; + FunctionDocumentation::Categories categories = {"String encoding"}; + + factory.registerFunction>>({description, syntax, arguments, returned_value, examples, categories}); +} +} + +#endif diff --git a/src/Functions/tryBase64Decode.cpp b/src/Functions/tryBase64Decode.cpp index 25da111492d..08eabe93200 100644 --- a/src/Functions/tryBase64Decode.cpp +++ b/src/Functions/tryBase64Decode.cpp @@ -7,8 +7,14 @@ namespace DB { REGISTER_FUNCTION(TryBase64Decode) { - factory.registerFunction>>(); - factory.registerFunction>>(); + FunctionDocumentation::Description description = R"(Decodes a String or FixedString from base64, like base64Decode but returns an empty string in case of an error.)"; + FunctionDocumentation::Syntax syntax = "tryBase64Decode(encoded)"; + FunctionDocumentation::Arguments arguments = {{"encoded", "String column or constant. If the string is not a valid Base64-encoded value, returns an empty string."}}; + FunctionDocumentation::ReturnedValue returned_value = "A string containing the decoded value of the argument."; + FunctionDocumentation::Examples examples = {{"valid", "SELECT tryBase64Decode('Y2xpY2tob3VzZQ==')", "clickhouse"}, {"invalid", "SELECT tryBase64Decode('invalid')", ""}}; + FunctionDocumentation::Categories categories = {"String encoding"}; + + factory.registerFunction>>({description, syntax, arguments, returned_value, examples, categories}); } } diff --git a/src/Functions/tryBase64UrlDecode.cpp b/src/Functions/tryBase64UrlDecode.cpp new file mode 100644 index 00000000000..b9aaf4f9273 --- /dev/null +++ b/src/Functions/tryBase64UrlDecode.cpp @@ -0,0 +1,21 @@ +#include + +#if USE_BASE64 +#include + +namespace DB +{ +REGISTER_FUNCTION(TryBase64UrlDecode) +{ + FunctionDocumentation::Description description = R"(Decodes an URL from base64, like base64UrlDecode but returns an empty string in case of an error.)"; + FunctionDocumentation::Syntax syntax = "tryBase64UrlDecode(encodedUrl)"; + FunctionDocumentation::Arguments arguments = {{"encodedUrl", "String column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string."}}; + FunctionDocumentation::ReturnedValue returned_value = "A string containing the decoded value of the argument."; + FunctionDocumentation::Examples examples = {{"valid", "SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ')", "https://clickhouse.com"}, {"invalid", "SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja')", ""}}; + FunctionDocumentation::Categories categories = {"String encoding"}; + + factory.registerFunction>>({description, syntax, arguments, returned_value, examples, categories}); +} +} + +#endif diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql index 544f29e8a7d..e9deb778075 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql @@ -3,7 +3,6 @@ SELECT name FROM system.functions WHERE NOT is_aggregate AND origin = 'System' AND alias_to = '' AND length(description) < 10 AND name NOT IN ( 'aes_decrypt_mysql', 'aes_encrypt_mysql', 'decrypt', 'encrypt', - 'base64Decode', 'base64Encode', 'tryBase64Decode', 'base64UrlDecode', 'base64UrlEncode', 'tryBase64UrlDecode', 'convertCharset', 'detectLanguage', 'detectLanguageMixed', 'geoToH3', From cb576211d57ce18a6d20c6a5965494e92a685546 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 11 Jun 2024 17:21:14 +0200 Subject: [PATCH 540/856] Analyzer: Fix aggregate function name rewriting --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 5e5ecaaa93a..07328383700 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -992,11 +992,11 @@ std::string QueryAnalyzer::rewriteAggregateFunctionNameIfNeeded( } /// Replace aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal - if (result_aggregate_function_name.ends_with("ifdistinct")) + if (aggregate_function_name_lowercase.ends_with("ifdistinct")) { size_t prefix_length = result_aggregate_function_name.size() - strlen("ifdistinct"); result_aggregate_function_name = result_aggregate_function_name.substr(0, prefix_length) + "DistinctIf"; - } + } bool need_add_or_null = settings.aggregate_functions_null_for_empty && !result_aggregate_function_name.ends_with("OrNull"); if (need_add_or_null) From 5bfca13a4cf5743ea14d0c248dbc69a414c93f75 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 11 Jun 2024 18:04:38 +0200 Subject: [PATCH 541/856] Update test.py --- tests/integration/test_replicated_database/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index ea569939c1c..73b7ae265e4 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -404,6 +404,7 @@ def test_alter_detach_part(started_cluster, engine): main_node.query(f"INSERT INTO {database}.alter_detach VALUES (123)") if engine == "MergeTree": dummy_node.query(f"INSERT INTO {database}.alter_detach VALUES (456)") + main_node.query(f"SYSTEM SYNC REPLICA {database}.alter_detach PULL") main_node.query(f"ALTER TABLE {database}.alter_detach DETACH PART '{part_name}'") detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_detach'" assert main_node.query(detached_parts_query) == f"{part_name}\n" From 7b239f110245c53dbfc6d296eea4d975d019867e Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 11 Jun 2024 17:36:49 +0000 Subject: [PATCH 542/856] 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 543/856] 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 544/856] 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 e0279e856f874c652d292d8c903975f51f2de3ba Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 19:41:22 +0000 Subject: [PATCH 545/856] Fix #63792 (hopefully) --- contrib/openssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/openssl b/contrib/openssl index f7b8721dfc6..5c4b034785b 160000 --- a/contrib/openssl +++ b/contrib/openssl @@ -1 +1 @@ -Subproject commit f7b8721dfc66abb147f24ca07b9c9d1d64f40f71 +Subproject commit 5c4b034785bf04f80380138cf49bf9743400f144 From 848054e85f743cee8797a7419163840764d76935 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 21:06:45 +0000 Subject: [PATCH 546/856] Fix another false positive leak --- contrib/openssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/openssl b/contrib/openssl index 5c4b034785b..67c0b63e578 160000 --- a/contrib/openssl +++ b/contrib/openssl @@ -1 +1 @@ -Subproject commit 5c4b034785bf04f80380138cf49bf9743400f144 +Subproject commit 67c0b63e578e4c751ac9edf490f5a96124fff8dc From d8366119c403429c2748f4ea372d0542701147f6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 11 Jun 2024 22:04:37 +0100 Subject: [PATCH 547/856] impl --- src/Processors/QueryPlan/ReadFromLoopStep.cpp | 26 ++++++++++++------- ...op_engine_with_parallel_replicas.reference | 0 ...168_loop_engine_with_parallel_replicas.sql | 9 +++++++ 3 files changed, 26 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.reference create mode 100644 tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql diff --git a/src/Processors/QueryPlan/ReadFromLoopStep.cpp b/src/Processors/QueryPlan/ReadFromLoopStep.cpp index 10436490a2a..2e5fa3ec9f7 100644 --- a/src/Processors/QueryPlan/ReadFromLoopStep.cpp +++ b/src/Processors/QueryPlan/ReadFromLoopStep.cpp @@ -1,14 +1,15 @@ -#include -#include -#include -#include -#include +#include +#include +#include #include #include -#include -#include +#include +#include #include -#include +#include +#include +#include +#include namespace DB { @@ -111,6 +112,13 @@ namespace DB std::unique_ptr executor; }; + static ContextPtr disableParallelReplicas(ContextPtr context) + { + auto modified_context = Context::createCopy(context); + modified_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + return modified_context; + } + ReadFromLoopStep::ReadFromLoopStep( const Names & column_names_, const SelectQueryInfo & query_info_, @@ -125,7 +133,7 @@ namespace DB column_names_, query_info_, storage_snapshot_, - context_) + disableParallelReplicas(context_)) , column_names(column_names_) , processed_stage(processed_stage_) , inner_storage(std::move(inner_storage_)) diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.reference b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql new file mode 100644 index 00000000000..dfcb5de9f2a --- /dev/null +++ b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql @@ -0,0 +1,9 @@ +DROP DATABASE IF EXISTS 03147_db; +CREATE DATABASE IF NOT EXISTS 03147_db; +CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n; +INSERT INTO 03147_db.t SELECT * FROM numbers(10); +USE 03147_db; + +SET allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 100; + +SELECT * FROM loop(03147_db.t) LIMIT 15 FORMAT Null; From 66a714e3ee2488883a800692b4d02e89b272b668 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 11 Jun 2024 22:45:02 +0000 Subject: [PATCH 548/856] 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 549/856] 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 1ad648517556ca5a144ba2497867da6bfe463607 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 22:50:32 +0000 Subject: [PATCH 550/856] Fix 'Tasks in BackgroundSchedulePool cannot throw' caused by MergeTreeData::loadUnexpectedDataParts() --- src/Storages/MergeTree/MergeTreeData.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 14a310364dc..89f39c65517 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1981,6 +1981,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks, std::optional runner(getUnexpectedPartsLoadingThreadPool().get(), "UnexpectedParts"); for (auto & load_state : unexpected_data_parts) @@ -2027,6 +2031,13 @@ void MergeTreeData::loadUnexpectedDataParts() unexpected_data_parts_cv.notify_all(); } } +catch (...) +{ + LOG_ERROR(log, "Loading of unexpected parts failed. " + "Will terminate to avoid undefined behaviour due to inconsistent set of parts. " + "Exception: {}", getCurrentExceptionMessage(true)); + std::terminate(); +} void MergeTreeData::loadOutdatedDataParts(bool is_async) try From 08504d7b1b0e66292fa4ebdbe522d21046b1ec2a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Jun 2023 20:42:46 +0000 Subject: [PATCH 551/856] Change default s3_throw_on_zero_files_match to true, document that pre-signed S3 URLs are not supported --- docs/en/sql-reference/table-functions/s3.md | 11 ++++++++++- src/Core/Settings.h | 6 +++--- src/Core/SettingsChangesHistory.h | 5 +++-- src/Storages/ObjectStorage/Azure/Configuration.cpp | 1 + src/Storages/ObjectStorage/HDFS/Configuration.cpp | 1 + src/Storages/ObjectStorage/S3/Configuration.cpp | 1 + src/Storages/ObjectStorage/StorageObjectStorage.h | 1 + .../ObjectStorage/StorageObjectStorageSource.cpp | 12 +++++++++--- .../ObjectStorage/StorageObjectStorageSource.h | 2 ++ src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- 10 files changed, 32 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 1a7e2b8d66a..7538d66996f 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -248,7 +248,6 @@ FROM s3( LIMIT 5; ``` - ## Working with archives Suppose that we have several archive files with following URIs on S3: @@ -266,6 +265,16 @@ FROM s3( ); ``` +## Presigned URL + +Presigned URLs are currently not supported. Use `url()` table function instead: +```sql +SELECT * +FROM url( + 'https://example.amazonaws.com/f.csv?X-Amz-Security-Token=[...]' +) +``` + ## Virtual Columns {#virtual-columns} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f99243e285..bb8a7587a84 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -115,9 +115,9 @@ class IColumn; M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ - M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ - M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ - M(Bool, azure_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, s3_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, hdfs_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, azure_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, s3_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in S3 table engine", 0) \ M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b447421671e..b4bb4716a8a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -88,8 +88,9 @@ static const std::map StorageObjectStorageSourc iterator = std::make_unique( object_storage, configuration, predicate, virtual_columns, local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size, - settings.throw_on_zero_files_match, file_progress_callback); + settings.throw_on_zero_files_match, settings.throw_on_zero_files_match_setting_name, + file_progress_callback); } else { @@ -425,6 +426,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( ObjectInfos * read_keys_, size_t list_object_keys_size, bool throw_on_zero_files_match_, + const char * throw_on_zero_files_match_setting_name_, std::function file_progress_callback_) : IIterator("GlobIterator") , WithContext(context_) @@ -432,6 +434,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( , configuration(configuration_) , virtual_columns(virtual_columns_) , throw_on_zero_files_match(throw_on_zero_files_match_) + , throw_on_zero_files_match_setting_name(throw_on_zero_files_match_setting_name_) , read_keys(read_keys_) , file_progress_callback(file_progress_callback_) { @@ -484,8 +487,11 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne if (first_iteration && !object_info && throw_on_zero_files_match) { throw Exception(ErrorCodes::FILE_DOESNT_EXIST, - "Can not match any files with path {}", - configuration->getPath()); + "Can not match any files with path {}{}", + configuration->getPath(), + throw_on_zero_files_match_setting_name + ? fmt::format(" (this error can be suppressed by setting {} = false)", throw_on_zero_files_match_setting_name) + : ""); } first_iteration = false; return object_info; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index fd7c7aa7102..5e76d8e979f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -168,6 +168,7 @@ public: ObjectInfos * read_keys_, size_t list_object_keys_size, bool throw_on_zero_files_match_, + const char * throw_on_zero_files_match_setting_name_, std::function file_progress_callback_ = {}); ~GlobIterator() override = default; @@ -184,6 +185,7 @@ private: const ConfigurationPtr configuration; const NamesAndTypesList virtual_columns; const bool throw_on_zero_files_match; + const char * throw_on_zero_files_match_setting_name; size_t index = 0; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index afb75a21b21..7e26335c691 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -486,7 +486,7 @@ std::shared_ptr StorageS3Queue::createFileIterator { auto settings = configuration->getQuerySettings(local_context); auto glob_iterator = std::make_unique( - object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match); + object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match, settings.throw_on_zero_files_match_setting_name); return std::make_shared(files_metadata, std::move(glob_iterator), shutdown_called, log); } From 26e5d9a8675488a5502a356169990f88e53e64a1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 01:28:44 +0000 Subject: [PATCH 552/856] aspell --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 84682689934..c4b70de1f65 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2758 +personal_ws-1.1 en 2912 AArch ACLs ALTERs @@ -722,6 +722,7 @@ Postgres PostgresSQL Precompiled Preprocess +Presigned PrettyCompact PrettyCompactMonoBlock PrettyCompactNoEscapes @@ -1936,9 +1937,9 @@ loghouse london lookups loongarch -lowcardinality lowCardinalityIndices lowCardinalityKeys +lowcardinality lowerUTF lowercased lttb From 13dd79f5b2a7d57e4e87e6e63849ceb8479dd495 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 06:19:12 +0000 Subject: [PATCH 553/856] Fix tests --- tests/integration/test_storage_hdfs/test.py | 13 +++++++------ .../02481_s3_throw_if_mismatch_files.reference | 4 ++-- .../02481_s3_throw_if_mismatch_files.sql | 4 ++-- ...resigned_url_and_url_with_special_characters.sql | 4 ++-- 4 files changed, 13 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index cda2b8694c6..9d17686b2cc 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -111,7 +111,7 @@ def test_storage_with_multidirectory_glob(started_cluster): try: node1.query( - "SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p4/path1,p2/path3}/postfix/data{1,2}.nonexist', TSV)" + "SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p4/path1,p2/path3}/postfix/data{1,2}.nonexist', TSV) SETTINGS hdfs_throw_on_zero_files_match=0" ) assert False, "Exception have to be thrown" except Exception as ex: @@ -220,14 +220,14 @@ def test_globs_in_read_table(started_cluster): ) print("inside_table_func ", inside_table_func) assert ( - node1.query("select * from hdfs(" + inside_table_func + ")") + node1.query("select * from hdfs(" + inside_table_func + ") settings hdfs_throw_on_zero_files_match=0") == paths_amount * some_data ) assert node1.query( - "select count(distinct _path) from hdfs(" + inside_table_func + ")" + "select count(distinct _path) from hdfs(" + inside_table_func + ") settings hdfs_throw_on_zero_files_match=0" ).rstrip() == str(paths_amount) assert node1.query( - "select count(distinct _file) from hdfs(" + inside_table_func + ")" + "select count(distinct _file) from hdfs(" + inside_table_func + ") settings hdfs_throw_on_zero_files_match=0" ).rstrip() == str(files_amount) @@ -635,6 +635,7 @@ def test_cluster_join(started_cluster): SELECT l.id,r.id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as l JOIN hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as r ON l.id = r.id + SETTINGS hdfs_throw_on_zero_files_match=0 """ ) assert "AMBIGUOUS_COLUMN_NAME" not in result @@ -643,13 +644,13 @@ def test_cluster_join(started_cluster): def test_cluster_macro(started_cluster): with_macro = node1.query( """ - SELECT id FROM hdfsCluster('{default_cluster_macro}', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') + SELECT id FROM hdfsCluster('{default_cluster_macro}', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') SETTINGS hdfs_throw_on_zero_files_match=0 """ ) no_macro = node1.query( """ - SELECT id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') + SELECT id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') SETTINGS hdfs_throw_on_zero_files_match=0 """ ) diff --git a/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.reference b/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.reference index a7096a686f5..752b12ff3bd 100644 --- a/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.reference +++ b/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.reference @@ -3,5 +3,5 @@ drop table if exists test_02481_mismatch_files; create table test_02481_mismatch_files (a UInt64, b String) engine = S3(s3_conn, filename='test_02481_mismatch_files_{_partition_id}', format=Parquet) partition by a; set s3_truncate_on_insert=1; insert into test_02481_mismatch_files values (1, 'a'), (22, 'b'), (333, 'c'); -select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } -select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=1; -- { serverError FILE_DOESNT_EXIST } +select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } +select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError FILE_DOESNT_EXIST } diff --git a/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.sql b/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.sql index 7ec1d3ebd5f..cd500b58946 100644 --- a/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.sql +++ b/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.sql @@ -7,6 +7,6 @@ create table test_02481_mismatch_files (a UInt64, b String) engine = S3(s3_conn, set s3_truncate_on_insert=1; insert into test_02481_mismatch_files values (1, 'a'), (22, 'b'), (333, 'c'); -select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } +select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } -select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=1; -- { serverError FILE_DOESNT_EXIST } +select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError FILE_DOESNT_EXIST } diff --git a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql index 1e99eb8b83d..078a5701aca 100644 --- a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql +++ b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql @@ -1,5 +1,5 @@ -- Tags: no-fasttest -select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } +select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet', NOSIGN) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } -select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_DETECT_FORMAT } +select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD', NOSIGN) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_DETECT_FORMAT } From af9f8ddbfa1ce14f199b09003430dd6ed4d9fa3a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 23:05:47 +0000 Subject: [PATCH 554/856] Move an unrelated setting to the correct section of SettingsChangesHistory.h --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b4bb4716a8a..31da77fddaf 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -106,6 +106,7 @@ static const std::map Date: Tue, 11 Jun 2024 23:13:49 +0000 Subject: [PATCH 555/856] Automatic style fix --- tests/integration/test_storage_hdfs/test.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 9d17686b2cc..3e9342c0499 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -220,14 +220,22 @@ def test_globs_in_read_table(started_cluster): ) print("inside_table_func ", inside_table_func) assert ( - node1.query("select * from hdfs(" + inside_table_func + ") settings hdfs_throw_on_zero_files_match=0") + node1.query( + "select * from hdfs(" + + inside_table_func + + ") settings hdfs_throw_on_zero_files_match=0" + ) == paths_amount * some_data ) assert node1.query( - "select count(distinct _path) from hdfs(" + inside_table_func + ") settings hdfs_throw_on_zero_files_match=0" + "select count(distinct _path) from hdfs(" + + inside_table_func + + ") settings hdfs_throw_on_zero_files_match=0" ).rstrip() == str(paths_amount) assert node1.query( - "select count(distinct _file) from hdfs(" + inside_table_func + ") settings hdfs_throw_on_zero_files_match=0" + "select count(distinct _file) from hdfs(" + + inside_table_func + + ") settings hdfs_throw_on_zero_files_match=0" ).rstrip() == str(files_amount) From 603176ef7f6f4de861d152993dd11e41b6502fd8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Jun 2024 01:22:51 +0200 Subject: [PATCH 556/856] 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 557/856] 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 558/856] 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 f632424f92dbb242369d5a17a6788ea57d9d0103 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 12 Jun 2024 04:32:34 +0000 Subject: [PATCH 559/856] remove unnecessary change Signed-off-by: Duc Canh Le --- src/Common/filesystemHelpers.cpp | 4 +--- src/Functions/FunctionsStringDistance.cpp | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 09c4508b7b2..2d053c615d9 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -1,6 +1,4 @@ #include "filesystemHelpers.h" -#include -#include #if defined(OS_LINUX) # include @@ -13,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index d0d8ebc946d..6cb23bbea9f 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -159,7 +159,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(haystack, haystack_end - haystack)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(haystack, haystack_end - haystack)); } } } @@ -186,7 +186,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(needle, needle_end - needle)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(needle, needle_end - needle)); } } } From 37e03ef320c63022d727a7c6cf201131f6e20731 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 12 Jun 2024 07:52:50 +0200 Subject: [PATCH 560/856] Modify pretty formats to display column names in the footer when row count is large --- docs/en/interfaces/formats.md | 2 +- .../operations/settings/settings-formats.md | 2 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 156 +- .../Impl/PrettyCompactBlockOutputFormat.cpp | 30 +- .../Impl/PrettyCompactBlockOutputFormat.h | 2 +- .../Impl/PrettySpaceBlockOutputFormat.cpp | 68 +- ...9_display_column_names_in_footer.reference | 17050 +++++++++++++++- .../03169_display_column_names_in_footer.sql | 14 +- 12 files changed, 16235 insertions(+), 1097 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index b292c669047..be3fc078070 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1490,7 +1490,7 @@ Differs from [PrettySpaceNoEscapes](#prettyspacenoescapes) in that up to 10,000 - [output_format_pretty_color](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_color) - use ANSI escape sequences to paint colors in Pretty formats. Default value - `true`. - [output_format_pretty_grid_charset](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_grid_charset) - Charset for printing grid borders. Available charsets: ASCII, UTF-8. Default value - `UTF-8`. - [output_format_pretty_row_numbers](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_row_numbers) - Add row numbers before each row for pretty output format. Default value - `true`. -- [output_format_pretty_display_column_names_footer](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_column_names_footer) - Display column names in the footer if there are more than 1000 rows. Default value - `false`. +- [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) - Display column names in the footer if there are more than 1000 rows. Default value - `false`. ## RowBinary {#rowbinary} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index c380045f085..87ab0d17675 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1706,7 +1706,7 @@ Result: └────────────┘ ``` -## output_format_pretty_display_column_names_footer +## output_format_pretty_display_footer_column_names Display column names in the footer if there are 999 or more rows. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fa2994cb98c..606441352e9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1140,7 +1140,7 @@ class IColumn; M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \ M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \ M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ - M(UInt64 output_format_pretty_display_column_names_footer, false, "Display column names in the footer if there are 999 or more rows.", 0) \ + M(UInt64, output_format_pretty_display_footer_column_names, false, "Display column names in the footer if there are 999 or more rows.", 0) \ M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \ M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \ M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 6af7980547f..6a6c42c67f5 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -105,7 +105,7 @@ static const std::map void { - if (i != 0) + writeCString(grid_symbols.bold_bar, out); + writeCString(" ", out); + for (size_t i = 0; i < num_columns; ++i) { - writeCString(" ", out); - writeCString(grid_symbols.bold_bar, out); - writeCString(" ", out); + if (i != 0) + { + writeCString(" ", out); + writeCString(grid_symbols.bold_bar, out); + writeCString(" ", out); + } + + const auto & col = header.getByPosition(i); + + if (color) + writeCString("\033[1m", out); + + if (col.type->shouldAlignRightInPrettyFormats()) + { + for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k) + writeChar(' ', out); + + writeString(col.name, out); + } + else + { + writeString(col.name, out); + + for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k) + writeChar(' ', out); + } + + if (color) + writeCString("\033[0m", out); } - - const auto & col = header.getByPosition(i); - - if (color) - writeCString("\033[1m", out); - - if (col.type->shouldAlignRightInPrettyFormats()) - { - for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k) - writeChar(' ', out); - - writeString(col.name, out); - } - else - { - writeString(col.name, out); - - for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k) - writeChar(' ', out); - } - - if (color) - writeCString("\033[0m", out); - } - writeCString(" ", out); - writeCString(grid_symbols.bold_bar, out); - writeCString("\n", out); + writeCString(" ", out); + writeCString(grid_symbols.bold_bar, out); + writeCString("\n", out); + }; + write_names(); if (format_settings.pretty.output_format_pretty_row_numbers) { @@ -317,9 +338,15 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind if (j != 0) writeCString(grid_symbols.bar, out); const auto & type = *header.getByPosition(j).type; - writeValueWithPadding(*columns[j], *serializations[j], i, + writeValueWithPadding( + *columns[j], + *serializations[j], + i, widths[j].empty() ? max_widths[j] : widths[j][i], - max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type)); + max_widths[j], + cut_to_width, + type.shouldAlignRightInPrettyFormats(), + isNumber(type)); } writeCString(grid_symbols.bar, out); @@ -332,8 +359,33 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind /// Write left blank writeString(String(row_number_width, ' '), out); } - writeString(bottom_separator_s, out); + /// output column names in the footer + if ((num_rows >= 999) && format_settings.pretty.output_format_pretty_display_footer_column_names) + { + writeString(footer_top_separator_s, out); + + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } + + /// output header names + write_names(); + + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } + + writeString(footer_bottom_separator_s, out); + } + else + { + writeString(bottom_separator_s, out); + } total_rows += num_rows; } diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index e1cbf69dbf0..86fbae3a671 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -57,7 +57,8 @@ PrettyCompactBlockOutputFormat::PrettyCompactBlockOutputFormat(WriteBuffer & out void PrettyCompactBlockOutputFormat::writeHeader( const Block & block, const Widths & max_widths, - const Widths & name_widths) + const Widths & name_widths, + const bool write_footer) { if (format_settings.pretty.output_format_pretty_row_numbers) { @@ -70,14 +71,20 @@ void PrettyCompactBlockOutputFormat::writeHeader( ascii_grid_symbols; /// Names - writeCString(grid_symbols.left_top_corner, out); + if (write_footer) + writeCString(grid_symbols.left_bottom_corner, out); + else + writeCString(grid_symbols.left_top_corner, out); writeCString(grid_symbols.dash, out); for (size_t i = 0; i < max_widths.size(); ++i) { if (i != 0) { writeCString(grid_symbols.dash, out); - writeCString(grid_symbols.top_separator, out); + if (write_footer) + writeCString(grid_symbols.bottom_separator, out); + else + writeCString(grid_symbols.top_separator, out); writeCString(grid_symbols.dash, out); } @@ -107,7 +114,10 @@ void PrettyCompactBlockOutputFormat::writeHeader( } } writeCString(grid_symbols.dash, out); - writeCString(grid_symbols.right_top_corner, out); + if (write_footer) + writeCString(grid_symbols.right_bottom_corner, out); + else + writeCString(grid_symbols.right_top_corner, out); writeCString("\n", out); } @@ -195,13 +205,19 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po Widths name_widths; calculateWidths(header, chunk, widths, max_widths, name_widths); - writeHeader(header, max_widths, name_widths); + writeHeader(header, max_widths, name_widths, false); for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i) writeRow(i, header, chunk, widths, max_widths); - - writeBottom(max_widths); + if ((num_rows >= 999) && format_settings.pretty.output_format_pretty_display_footer_column_names) + { + writeHeader(header, max_widths, name_widths, true); + } + else + { + writeBottom(max_widths); + } total_rows += num_rows; } diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h index 911fc2e950c..956ba57e712 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h @@ -17,7 +17,7 @@ public: String getName() const override { return "PrettyCompactBlockOutputFormat"; } private: - void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths); + void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths, const bool write_footer); void writeBottom(const Widths & max_widths); void writeRow( size_t row_num, diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 3f224f034aa..8f13563421d 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -36,39 +36,43 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port if (format_settings.pretty.output_format_pretty_row_numbers) writeString(String(row_number_width, ' '), out); /// Names - for (size_t i = 0; i < num_columns; ++i) + auto write_names = [&]() -> void { - if (i != 0) - writeCString(" ", out); - else - writeChar(' ', out); - - const ColumnWithTypeAndName & col = header.getByPosition(i); - - if (col.type->shouldAlignRightInPrettyFormats()) + for (size_t i = 0; i < num_columns; ++i) { - for (ssize_t k = 0; k < std::max(0z, static_cast(max_widths[i] - name_widths[i])); ++k) + if (i != 0) + writeCString(" ", out); + else writeChar(' ', out); - if (color) - writeCString("\033[1m", out); - writeString(col.name, out); - if (color) - writeCString("\033[0m", out); - } - else - { - if (color) - writeCString("\033[1m", out); - writeString(col.name, out); - if (color) - writeCString("\033[0m", out); + const ColumnWithTypeAndName & col = header.getByPosition(i); - for (ssize_t k = 0; k < std::max(0z, static_cast(max_widths[i] - name_widths[i])); ++k) - writeChar(' ', out); + if (col.type->shouldAlignRightInPrettyFormats()) + { + for (ssize_t k = 0; k < std::max(0z, static_cast(max_widths[i] - name_widths[i])); ++k) + writeChar(' ', out); + + if (color) + writeCString("\033[1m", out); + writeString(col.name, out); + if (color) + writeCString("\033[0m", out); + } + else + { + if (color) + writeCString("\033[1m", out); + writeString(col.name, out); + if (color) + writeCString("\033[0m", out); + + for (ssize_t k = 0; k < std::max(0z, static_cast(max_widths[i] - name_widths[i])); ++k) + writeChar(' ', out); + } } - } - writeCString("\n\n", out); + writeCString("\n\n", out); + }; + write_names(); for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row) { @@ -95,11 +99,19 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeValueWithPadding( *columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type)); } - writeReadableNumberTip(chunk); writeChar('\n', out); } + /// Write blank line between last row and footer + if (format_settings.pretty.output_format_pretty_display_footer_column_names) + writeCString("\n", out); + /// Write left blank + if (format_settings.pretty.output_format_pretty_row_numbers && format_settings.pretty.output_format_pretty_display_footer_column_names) + writeString(String(row_number_width, ' '), out); + /// Write footer + if ((num_rows >= 999) && format_settings.pretty.output_format_pretty_display_footer_column_names) + write_names(); total_rows += num_rows; } diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference index 488e7eac0d3..a8818991e1b 100644 --- a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference @@ -1,1002 +1,16048 @@ - ┌─number─┬─toTypeName(number)─┐ - 1. │ 0 │ UInt64 │ - 2. │ 1 │ UInt64 │ - 3. │ 2 │ UInt64 │ - 4. │ 3 │ UInt64 │ - 5. │ 4 │ UInt64 │ - 6. │ 5 │ UInt64 │ - 7. │ 6 │ UInt64 │ - 8. │ 7 │ UInt64 │ - 9. │ 8 │ UInt64 │ - 10. │ 9 │ UInt64 │ - 11. │ 10 │ UInt64 │ - 12. │ 11 │ UInt64 │ - 13. │ 12 │ UInt64 │ - 14. │ 13 │ UInt64 │ - 15. │ 14 │ UInt64 │ - 16. │ 15 │ UInt64 │ - 17. │ 16 │ UInt64 │ - 18. │ 17 │ UInt64 │ - 19. │ 18 │ UInt64 │ - 20. │ 19 │ UInt64 │ - 21. │ 20 │ UInt64 │ - 22. │ 21 │ UInt64 │ - 23. │ 22 │ UInt64 │ - 24. │ 23 │ UInt64 │ - 25. │ 24 │ UInt64 │ - 26. │ 25 │ UInt64 │ - 27. │ 26 │ UInt64 │ - 28. │ 27 │ UInt64 │ - 29. │ 28 │ UInt64 │ - 30. │ 29 │ UInt64 │ - 31. │ 30 │ UInt64 │ - 32. │ 31 │ UInt64 │ - 33. │ 32 │ UInt64 │ - 34. │ 33 │ UInt64 │ - 35. │ 34 │ UInt64 │ - 36. │ 35 │ UInt64 │ - 37. │ 36 │ UInt64 │ - 38. │ 37 │ UInt64 │ - 39. │ 38 │ UInt64 │ - 40. │ 39 │ UInt64 │ - 41. │ 40 │ UInt64 │ - 42. │ 41 │ UInt64 │ - 43. │ 42 │ UInt64 │ - 44. │ 43 │ UInt64 │ - 45. │ 44 │ UInt64 │ - 46. │ 45 │ UInt64 │ - 47. │ 46 │ UInt64 │ - 48. │ 47 │ UInt64 │ - 49. │ 48 │ UInt64 │ - 50. │ 49 │ UInt64 │ - 51. │ 50 │ UInt64 │ - 52. │ 51 │ UInt64 │ - 53. │ 52 │ UInt64 │ - 54. │ 53 │ UInt64 │ - 55. │ 54 │ UInt64 │ - 56. │ 55 │ UInt64 │ - 57. │ 56 │ UInt64 │ - 58. │ 57 │ UInt64 │ - 59. │ 58 │ UInt64 │ - 60. │ 59 │ UInt64 │ - 61. │ 60 │ UInt64 │ - 62. │ 61 │ UInt64 │ - 63. │ 62 │ UInt64 │ - 64. │ 63 │ UInt64 │ - 65. │ 64 │ UInt64 │ - 66. │ 65 │ UInt64 │ - 67. │ 66 │ UInt64 │ - 68. │ 67 │ UInt64 │ - 69. │ 68 │ UInt64 │ - 70. │ 69 │ UInt64 │ - 71. │ 70 │ UInt64 │ - 72. │ 71 │ UInt64 │ - 73. │ 72 │ UInt64 │ - 74. │ 73 │ UInt64 │ - 75. │ 74 │ UInt64 │ - 76. │ 75 │ UInt64 │ - 77. │ 76 │ UInt64 │ - 78. │ 77 │ UInt64 │ - 79. │ 78 │ UInt64 │ - 80. │ 79 │ UInt64 │ - 81. │ 80 │ UInt64 │ - 82. │ 81 │ UInt64 │ - 83. │ 82 │ UInt64 │ - 84. │ 83 │ UInt64 │ - 85. │ 84 │ UInt64 │ - 86. │ 85 │ UInt64 │ - 87. │ 86 │ UInt64 │ - 88. │ 87 │ UInt64 │ - 89. │ 88 │ UInt64 │ - 90. │ 89 │ UInt64 │ - 91. │ 90 │ UInt64 │ - 92. │ 91 │ UInt64 │ - 93. │ 92 │ UInt64 │ - 94. │ 93 │ UInt64 │ - 95. │ 94 │ UInt64 │ - 96. │ 95 │ UInt64 │ - 97. │ 96 │ UInt64 │ - 98. │ 97 │ UInt64 │ - 99. │ 98 │ UInt64 │ - 100. │ 99 │ UInt64 │ - 101. │ 100 │ UInt64 │ - 102. │ 101 │ UInt64 │ - 103. │ 102 │ UInt64 │ - 104. │ 103 │ UInt64 │ - 105. │ 104 │ UInt64 │ - 106. │ 105 │ UInt64 │ - 107. │ 106 │ UInt64 │ - 108. │ 107 │ UInt64 │ - 109. │ 108 │ UInt64 │ - 110. │ 109 │ UInt64 │ - 111. │ 110 │ UInt64 │ - 112. │ 111 │ UInt64 │ - 113. │ 112 │ UInt64 │ - 114. │ 113 │ UInt64 │ - 115. │ 114 │ UInt64 │ - 116. │ 115 │ UInt64 │ - 117. │ 116 │ UInt64 │ - 118. │ 117 │ UInt64 │ - 119. │ 118 │ UInt64 │ - 120. │ 119 │ UInt64 │ - 121. │ 120 │ UInt64 │ - 122. │ 121 │ UInt64 │ - 123. │ 122 │ UInt64 │ - 124. │ 123 │ UInt64 │ - 125. │ 124 │ UInt64 │ - 126. │ 125 │ UInt64 │ - 127. │ 126 │ UInt64 │ - 128. │ 127 │ UInt64 │ - 129. │ 128 │ UInt64 │ - 130. │ 129 │ UInt64 │ - 131. │ 130 │ UInt64 │ - 132. │ 131 │ UInt64 │ - 133. │ 132 │ UInt64 │ - 134. │ 133 │ UInt64 │ - 135. │ 134 │ UInt64 │ - 136. │ 135 │ UInt64 │ - 137. │ 136 │ UInt64 │ - 138. │ 137 │ UInt64 │ - 139. │ 138 │ UInt64 │ - 140. │ 139 │ UInt64 │ - 141. │ 140 │ UInt64 │ - 142. │ 141 │ UInt64 │ - 143. │ 142 │ UInt64 │ - 144. │ 143 │ UInt64 │ - 145. │ 144 │ UInt64 │ - 146. │ 145 │ UInt64 │ - 147. │ 146 │ UInt64 │ - 148. │ 147 │ UInt64 │ - 149. │ 148 │ UInt64 │ - 150. │ 149 │ UInt64 │ - 151. │ 150 │ UInt64 │ - 152. │ 151 │ UInt64 │ - 153. │ 152 │ UInt64 │ - 154. │ 153 │ UInt64 │ - 155. │ 154 │ UInt64 │ - 156. │ 155 │ UInt64 │ - 157. │ 156 │ UInt64 │ - 158. │ 157 │ UInt64 │ - 159. │ 158 │ UInt64 │ - 160. │ 159 │ UInt64 │ - 161. │ 160 │ UInt64 │ - 162. │ 161 │ UInt64 │ - 163. │ 162 │ UInt64 │ - 164. │ 163 │ UInt64 │ - 165. │ 164 │ UInt64 │ - 166. │ 165 │ UInt64 │ - 167. │ 166 │ UInt64 │ - 168. │ 167 │ UInt64 │ - 169. │ 168 │ UInt64 │ - 170. │ 169 │ UInt64 │ - 171. │ 170 │ UInt64 │ - 172. │ 171 │ UInt64 │ - 173. │ 172 │ UInt64 │ - 174. │ 173 │ UInt64 │ - 175. │ 174 │ UInt64 │ - 176. │ 175 │ UInt64 │ - 177. │ 176 │ UInt64 │ - 178. │ 177 │ UInt64 │ - 179. │ 178 │ UInt64 │ - 180. │ 179 │ UInt64 │ - 181. │ 180 │ UInt64 │ - 182. │ 181 │ UInt64 │ - 183. │ 182 │ UInt64 │ - 184. │ 183 │ UInt64 │ - 185. │ 184 │ UInt64 │ - 186. │ 185 │ UInt64 │ - 187. │ 186 │ UInt64 │ - 188. │ 187 │ UInt64 │ - 189. │ 188 │ UInt64 │ - 190. │ 189 │ UInt64 │ - 191. │ 190 │ UInt64 │ - 192. │ 191 │ UInt64 │ - 193. │ 192 │ UInt64 │ - 194. │ 193 │ UInt64 │ - 195. │ 194 │ UInt64 │ - 196. │ 195 │ UInt64 │ - 197. │ 196 │ UInt64 │ - 198. │ 197 │ UInt64 │ - 199. │ 198 │ UInt64 │ - 200. │ 199 │ UInt64 │ - 201. │ 200 │ UInt64 │ - 202. │ 201 │ UInt64 │ - 203. │ 202 │ UInt64 │ - 204. │ 203 │ UInt64 │ - 205. │ 204 │ UInt64 │ - 206. │ 205 │ UInt64 │ - 207. │ 206 │ UInt64 │ - 208. │ 207 │ UInt64 │ - 209. │ 208 │ UInt64 │ - 210. │ 209 │ UInt64 │ - 211. │ 210 │ UInt64 │ - 212. │ 211 │ UInt64 │ - 213. │ 212 │ UInt64 │ - 214. │ 213 │ UInt64 │ - 215. │ 214 │ UInt64 │ - 216. │ 215 │ UInt64 │ - 217. │ 216 │ UInt64 │ - 218. │ 217 │ UInt64 │ - 219. │ 218 │ UInt64 │ - 220. │ 219 │ UInt64 │ - 221. │ 220 │ UInt64 │ - 222. │ 221 │ UInt64 │ - 223. │ 222 │ UInt64 │ - 224. │ 223 │ UInt64 │ - 225. │ 224 │ UInt64 │ - 226. │ 225 │ UInt64 │ - 227. │ 226 │ UInt64 │ - 228. │ 227 │ UInt64 │ - 229. │ 228 │ UInt64 │ - 230. │ 229 │ UInt64 │ - 231. │ 230 │ UInt64 │ - 232. │ 231 │ UInt64 │ - 233. │ 232 │ UInt64 │ - 234. │ 233 │ UInt64 │ - 235. │ 234 │ UInt64 │ - 236. │ 235 │ UInt64 │ - 237. │ 236 │ UInt64 │ - 238. │ 237 │ UInt64 │ - 239. │ 238 │ UInt64 │ - 240. │ 239 │ UInt64 │ - 241. │ 240 │ UInt64 │ - 242. │ 241 │ UInt64 │ - 243. │ 242 │ UInt64 │ - 244. │ 243 │ UInt64 │ - 245. │ 244 │ UInt64 │ - 246. │ 245 │ UInt64 │ - 247. │ 246 │ UInt64 │ - 248. │ 247 │ UInt64 │ - 249. │ 248 │ UInt64 │ - 250. │ 249 │ UInt64 │ - 251. │ 250 │ UInt64 │ - 252. │ 251 │ UInt64 │ - 253. │ 252 │ UInt64 │ - 254. │ 253 │ UInt64 │ - 255. │ 254 │ UInt64 │ - 256. │ 255 │ UInt64 │ - 257. │ 256 │ UInt64 │ - 258. │ 257 │ UInt64 │ - 259. │ 258 │ UInt64 │ - 260. │ 259 │ UInt64 │ - 261. │ 260 │ UInt64 │ - 262. │ 261 │ UInt64 │ - 263. │ 262 │ UInt64 │ - 264. │ 263 │ UInt64 │ - 265. │ 264 │ UInt64 │ - 266. │ 265 │ UInt64 │ - 267. │ 266 │ UInt64 │ - 268. │ 267 │ UInt64 │ - 269. │ 268 │ UInt64 │ - 270. │ 269 │ UInt64 │ - 271. │ 270 │ UInt64 │ - 272. │ 271 │ UInt64 │ - 273. │ 272 │ UInt64 │ - 274. │ 273 │ UInt64 │ - 275. │ 274 │ UInt64 │ - 276. │ 275 │ UInt64 │ - 277. │ 276 │ UInt64 │ - 278. │ 277 │ UInt64 │ - 279. │ 278 │ UInt64 │ - 280. │ 279 │ UInt64 │ - 281. │ 280 │ UInt64 │ - 282. │ 281 │ UInt64 │ - 283. │ 282 │ UInt64 │ - 284. │ 283 │ UInt64 │ - 285. │ 284 │ UInt64 │ - 286. │ 285 │ UInt64 │ - 287. │ 286 │ UInt64 │ - 288. │ 287 │ UInt64 │ - 289. │ 288 │ UInt64 │ - 290. │ 289 │ UInt64 │ - 291. │ 290 │ UInt64 │ - 292. │ 291 │ UInt64 │ - 293. │ 292 │ UInt64 │ - 294. │ 293 │ UInt64 │ - 295. │ 294 │ UInt64 │ - 296. │ 295 │ UInt64 │ - 297. │ 296 │ UInt64 │ - 298. │ 297 │ UInt64 │ - 299. │ 298 │ UInt64 │ - 300. │ 299 │ UInt64 │ - 301. │ 300 │ UInt64 │ - 302. │ 301 │ UInt64 │ - 303. │ 302 │ UInt64 │ - 304. │ 303 │ UInt64 │ - 305. │ 304 │ UInt64 │ - 306. │ 305 │ UInt64 │ - 307. │ 306 │ UInt64 │ - 308. │ 307 │ UInt64 │ - 309. │ 308 │ UInt64 │ - 310. │ 309 │ UInt64 │ - 311. │ 310 │ UInt64 │ - 312. │ 311 │ UInt64 │ - 313. │ 312 │ UInt64 │ - 314. │ 313 │ UInt64 │ - 315. │ 314 │ UInt64 │ - 316. │ 315 │ UInt64 │ - 317. │ 316 │ UInt64 │ - 318. │ 317 │ UInt64 │ - 319. │ 318 │ UInt64 │ - 320. │ 319 │ UInt64 │ - 321. │ 320 │ UInt64 │ - 322. │ 321 │ UInt64 │ - 323. │ 322 │ UInt64 │ - 324. │ 323 │ UInt64 │ - 325. │ 324 │ UInt64 │ - 326. │ 325 │ UInt64 │ - 327. │ 326 │ UInt64 │ - 328. │ 327 │ UInt64 │ - 329. │ 328 │ UInt64 │ - 330. │ 329 │ UInt64 │ - 331. │ 330 │ UInt64 │ - 332. │ 331 │ UInt64 │ - 333. │ 332 │ UInt64 │ - 334. │ 333 │ UInt64 │ - 335. │ 334 │ UInt64 │ - 336. │ 335 │ UInt64 │ - 337. │ 336 │ UInt64 │ - 338. │ 337 │ UInt64 │ - 339. │ 338 │ UInt64 │ - 340. │ 339 │ UInt64 │ - 341. │ 340 │ UInt64 │ - 342. │ 341 │ UInt64 │ - 343. │ 342 │ UInt64 │ - 344. │ 343 │ UInt64 │ - 345. │ 344 │ UInt64 │ - 346. │ 345 │ UInt64 │ - 347. │ 346 │ UInt64 │ - 348. │ 347 │ UInt64 │ - 349. │ 348 │ UInt64 │ - 350. │ 349 │ UInt64 │ - 351. │ 350 │ UInt64 │ - 352. │ 351 │ UInt64 │ - 353. │ 352 │ UInt64 │ - 354. │ 353 │ UInt64 │ - 355. │ 354 │ UInt64 │ - 356. │ 355 │ UInt64 │ - 357. │ 356 │ UInt64 │ - 358. │ 357 │ UInt64 │ - 359. │ 358 │ UInt64 │ - 360. │ 359 │ UInt64 │ - 361. │ 360 │ UInt64 │ - 362. │ 361 │ UInt64 │ - 363. │ 362 │ UInt64 │ - 364. │ 363 │ UInt64 │ - 365. │ 364 │ UInt64 │ - 366. │ 365 │ UInt64 │ - 367. │ 366 │ UInt64 │ - 368. │ 367 │ UInt64 │ - 369. │ 368 │ UInt64 │ - 370. │ 369 │ UInt64 │ - 371. │ 370 │ UInt64 │ - 372. │ 371 │ UInt64 │ - 373. │ 372 │ UInt64 │ - 374. │ 373 │ UInt64 │ - 375. │ 374 │ UInt64 │ - 376. │ 375 │ UInt64 │ - 377. │ 376 │ UInt64 │ - 378. │ 377 │ UInt64 │ - 379. │ 378 │ UInt64 │ - 380. │ 379 │ UInt64 │ - 381. │ 380 │ UInt64 │ - 382. │ 381 │ UInt64 │ - 383. │ 382 │ UInt64 │ - 384. │ 383 │ UInt64 │ - 385. │ 384 │ UInt64 │ - 386. │ 385 │ UInt64 │ - 387. │ 386 │ UInt64 │ - 388. │ 387 │ UInt64 │ - 389. │ 388 │ UInt64 │ - 390. │ 389 │ UInt64 │ - 391. │ 390 │ UInt64 │ - 392. │ 391 │ UInt64 │ - 393. │ 392 │ UInt64 │ - 394. │ 393 │ UInt64 │ - 395. │ 394 │ UInt64 │ - 396. │ 395 │ UInt64 │ - 397. │ 396 │ UInt64 │ - 398. │ 397 │ UInt64 │ - 399. │ 398 │ UInt64 │ - 400. │ 399 │ UInt64 │ - 401. │ 400 │ UInt64 │ - 402. │ 401 │ UInt64 │ - 403. │ 402 │ UInt64 │ - 404. │ 403 │ UInt64 │ - 405. │ 404 │ UInt64 │ - 406. │ 405 │ UInt64 │ - 407. │ 406 │ UInt64 │ - 408. │ 407 │ UInt64 │ - 409. │ 408 │ UInt64 │ - 410. │ 409 │ UInt64 │ - 411. │ 410 │ UInt64 │ - 412. │ 411 │ UInt64 │ - 413. │ 412 │ UInt64 │ - 414. │ 413 │ UInt64 │ - 415. │ 414 │ UInt64 │ - 416. │ 415 │ UInt64 │ - 417. │ 416 │ UInt64 │ - 418. │ 417 │ UInt64 │ - 419. │ 418 │ UInt64 │ - 420. │ 419 │ UInt64 │ - 421. │ 420 │ UInt64 │ - 422. │ 421 │ UInt64 │ - 423. │ 422 │ UInt64 │ - 424. │ 423 │ UInt64 │ - 425. │ 424 │ UInt64 │ - 426. │ 425 │ UInt64 │ - 427. │ 426 │ UInt64 │ - 428. │ 427 │ UInt64 │ - 429. │ 428 │ UInt64 │ - 430. │ 429 │ UInt64 │ - 431. │ 430 │ UInt64 │ - 432. │ 431 │ UInt64 │ - 433. │ 432 │ UInt64 │ - 434. │ 433 │ UInt64 │ - 435. │ 434 │ UInt64 │ - 436. │ 435 │ UInt64 │ - 437. │ 436 │ UInt64 │ - 438. │ 437 │ UInt64 │ - 439. │ 438 │ UInt64 │ - 440. │ 439 │ UInt64 │ - 441. │ 440 │ UInt64 │ - 442. │ 441 │ UInt64 │ - 443. │ 442 │ UInt64 │ - 444. │ 443 │ UInt64 │ - 445. │ 444 │ UInt64 │ - 446. │ 445 │ UInt64 │ - 447. │ 446 │ UInt64 │ - 448. │ 447 │ UInt64 │ - 449. │ 448 │ UInt64 │ - 450. │ 449 │ UInt64 │ - 451. │ 450 │ UInt64 │ - 452. │ 451 │ UInt64 │ - 453. │ 452 │ UInt64 │ - 454. │ 453 │ UInt64 │ - 455. │ 454 │ UInt64 │ - 456. │ 455 │ UInt64 │ - 457. │ 456 │ UInt64 │ - 458. │ 457 │ UInt64 │ - 459. │ 458 │ UInt64 │ - 460. │ 459 │ UInt64 │ - 461. │ 460 │ UInt64 │ - 462. │ 461 │ UInt64 │ - 463. │ 462 │ UInt64 │ - 464. │ 463 │ UInt64 │ - 465. │ 464 │ UInt64 │ - 466. │ 465 │ UInt64 │ - 467. │ 466 │ UInt64 │ - 468. │ 467 │ UInt64 │ - 469. │ 468 │ UInt64 │ - 470. │ 469 │ UInt64 │ - 471. │ 470 │ UInt64 │ - 472. │ 471 │ UInt64 │ - 473. │ 472 │ UInt64 │ - 474. │ 473 │ UInt64 │ - 475. │ 474 │ UInt64 │ - 476. │ 475 │ UInt64 │ - 477. │ 476 │ UInt64 │ - 478. │ 477 │ UInt64 │ - 479. │ 478 │ UInt64 │ - 480. │ 479 │ UInt64 │ - 481. │ 480 │ UInt64 │ - 482. │ 481 │ UInt64 │ - 483. │ 482 │ UInt64 │ - 484. │ 483 │ UInt64 │ - 485. │ 484 │ UInt64 │ - 486. │ 485 │ UInt64 │ - 487. │ 486 │ UInt64 │ - 488. │ 487 │ UInt64 │ - 489. │ 488 │ UInt64 │ - 490. │ 489 │ UInt64 │ - 491. │ 490 │ UInt64 │ - 492. │ 491 │ UInt64 │ - 493. │ 492 │ UInt64 │ - 494. │ 493 │ UInt64 │ - 495. │ 494 │ UInt64 │ - 496. │ 495 │ UInt64 │ - 497. │ 496 │ UInt64 │ - 498. │ 497 │ UInt64 │ - 499. │ 498 │ UInt64 │ - 500. │ 499 │ UInt64 │ - 501. │ 500 │ UInt64 │ - 502. │ 501 │ UInt64 │ - 503. │ 502 │ UInt64 │ - 504. │ 503 │ UInt64 │ - 505. │ 504 │ UInt64 │ - 506. │ 505 │ UInt64 │ - 507. │ 506 │ UInt64 │ - 508. │ 507 │ UInt64 │ - 509. │ 508 │ UInt64 │ - 510. │ 509 │ UInt64 │ - 511. │ 510 │ UInt64 │ - 512. │ 511 │ UInt64 │ - 513. │ 512 │ UInt64 │ - 514. │ 513 │ UInt64 │ - 515. │ 514 │ UInt64 │ - 516. │ 515 │ UInt64 │ - 517. │ 516 │ UInt64 │ - 518. │ 517 │ UInt64 │ - 519. │ 518 │ UInt64 │ - 520. │ 519 │ UInt64 │ - 521. │ 520 │ UInt64 │ - 522. │ 521 │ UInt64 │ - 523. │ 522 │ UInt64 │ - 524. │ 523 │ UInt64 │ - 525. │ 524 │ UInt64 │ - 526. │ 525 │ UInt64 │ - 527. │ 526 │ UInt64 │ - 528. │ 527 │ UInt64 │ - 529. │ 528 │ UInt64 │ - 530. │ 529 │ UInt64 │ - 531. │ 530 │ UInt64 │ - 532. │ 531 │ UInt64 │ - 533. │ 532 │ UInt64 │ - 534. │ 533 │ UInt64 │ - 535. │ 534 │ UInt64 │ - 536. │ 535 │ UInt64 │ - 537. │ 536 │ UInt64 │ - 538. │ 537 │ UInt64 │ - 539. │ 538 │ UInt64 │ - 540. │ 539 │ UInt64 │ - 541. │ 540 │ UInt64 │ - 542. │ 541 │ UInt64 │ - 543. │ 542 │ UInt64 │ - 544. │ 543 │ UInt64 │ - 545. │ 544 │ UInt64 │ - 546. │ 545 │ UInt64 │ - 547. │ 546 │ UInt64 │ - 548. │ 547 │ UInt64 │ - 549. │ 548 │ UInt64 │ - 550. │ 549 │ UInt64 │ - 551. │ 550 │ UInt64 │ - 552. │ 551 │ UInt64 │ - 553. │ 552 │ UInt64 │ - 554. │ 553 │ UInt64 │ - 555. │ 554 │ UInt64 │ - 556. │ 555 │ UInt64 │ - 557. │ 556 │ UInt64 │ - 558. │ 557 │ UInt64 │ - 559. │ 558 │ UInt64 │ - 560. │ 559 │ UInt64 │ - 561. │ 560 │ UInt64 │ - 562. │ 561 │ UInt64 │ - 563. │ 562 │ UInt64 │ - 564. │ 563 │ UInt64 │ - 565. │ 564 │ UInt64 │ - 566. │ 565 │ UInt64 │ - 567. │ 566 │ UInt64 │ - 568. │ 567 │ UInt64 │ - 569. │ 568 │ UInt64 │ - 570. │ 569 │ UInt64 │ - 571. │ 570 │ UInt64 │ - 572. │ 571 │ UInt64 │ - 573. │ 572 │ UInt64 │ - 574. │ 573 │ UInt64 │ - 575. │ 574 │ UInt64 │ - 576. │ 575 │ UInt64 │ - 577. │ 576 │ UInt64 │ - 578. │ 577 │ UInt64 │ - 579. │ 578 │ UInt64 │ - 580. │ 579 │ UInt64 │ - 581. │ 580 │ UInt64 │ - 582. │ 581 │ UInt64 │ - 583. │ 582 │ UInt64 │ - 584. │ 583 │ UInt64 │ - 585. │ 584 │ UInt64 │ - 586. │ 585 │ UInt64 │ - 587. │ 586 │ UInt64 │ - 588. │ 587 │ UInt64 │ - 589. │ 588 │ UInt64 │ - 590. │ 589 │ UInt64 │ - 591. │ 590 │ UInt64 │ - 592. │ 591 │ UInt64 │ - 593. │ 592 │ UInt64 │ - 594. │ 593 │ UInt64 │ - 595. │ 594 │ UInt64 │ - 596. │ 595 │ UInt64 │ - 597. │ 596 │ UInt64 │ - 598. │ 597 │ UInt64 │ - 599. │ 598 │ UInt64 │ - 600. │ 599 │ UInt64 │ - 601. │ 600 │ UInt64 │ - 602. │ 601 │ UInt64 │ - 603. │ 602 │ UInt64 │ - 604. │ 603 │ UInt64 │ - 605. │ 604 │ UInt64 │ - 606. │ 605 │ UInt64 │ - 607. │ 606 │ UInt64 │ - 608. │ 607 │ UInt64 │ - 609. │ 608 │ UInt64 │ - 610. │ 609 │ UInt64 │ - 611. │ 610 │ UInt64 │ - 612. │ 611 │ UInt64 │ - 613. │ 612 │ UInt64 │ - 614. │ 613 │ UInt64 │ - 615. │ 614 │ UInt64 │ - 616. │ 615 │ UInt64 │ - 617. │ 616 │ UInt64 │ - 618. │ 617 │ UInt64 │ - 619. │ 618 │ UInt64 │ - 620. │ 619 │ UInt64 │ - 621. │ 620 │ UInt64 │ - 622. │ 621 │ UInt64 │ - 623. │ 622 │ UInt64 │ - 624. │ 623 │ UInt64 │ - 625. │ 624 │ UInt64 │ - 626. │ 625 │ UInt64 │ - 627. │ 626 │ UInt64 │ - 628. │ 627 │ UInt64 │ - 629. │ 628 │ UInt64 │ - 630. │ 629 │ UInt64 │ - 631. │ 630 │ UInt64 │ - 632. │ 631 │ UInt64 │ - 633. │ 632 │ UInt64 │ - 634. │ 633 │ UInt64 │ - 635. │ 634 │ UInt64 │ - 636. │ 635 │ UInt64 │ - 637. │ 636 │ UInt64 │ - 638. │ 637 │ UInt64 │ - 639. │ 638 │ UInt64 │ - 640. │ 639 │ UInt64 │ - 641. │ 640 │ UInt64 │ - 642. │ 641 │ UInt64 │ - 643. │ 642 │ UInt64 │ - 644. │ 643 │ UInt64 │ - 645. │ 644 │ UInt64 │ - 646. │ 645 │ UInt64 │ - 647. │ 646 │ UInt64 │ - 648. │ 647 │ UInt64 │ - 649. │ 648 │ UInt64 │ - 650. │ 649 │ UInt64 │ - 651. │ 650 │ UInt64 │ - 652. │ 651 │ UInt64 │ - 653. │ 652 │ UInt64 │ - 654. │ 653 │ UInt64 │ - 655. │ 654 │ UInt64 │ - 656. │ 655 │ UInt64 │ - 657. │ 656 │ UInt64 │ - 658. │ 657 │ UInt64 │ - 659. │ 658 │ UInt64 │ - 660. │ 659 │ UInt64 │ - 661. │ 660 │ UInt64 │ - 662. │ 661 │ UInt64 │ - 663. │ 662 │ UInt64 │ - 664. │ 663 │ UInt64 │ - 665. │ 664 │ UInt64 │ - 666. │ 665 │ UInt64 │ - 667. │ 666 │ UInt64 │ - 668. │ 667 │ UInt64 │ - 669. │ 668 │ UInt64 │ - 670. │ 669 │ UInt64 │ - 671. │ 670 │ UInt64 │ - 672. │ 671 │ UInt64 │ - 673. │ 672 │ UInt64 │ - 674. │ 673 │ UInt64 │ - 675. │ 674 │ UInt64 │ - 676. │ 675 │ UInt64 │ - 677. │ 676 │ UInt64 │ - 678. │ 677 │ UInt64 │ - 679. │ 678 │ UInt64 │ - 680. │ 679 │ UInt64 │ - 681. │ 680 │ UInt64 │ - 682. │ 681 │ UInt64 │ - 683. │ 682 │ UInt64 │ - 684. │ 683 │ UInt64 │ - 685. │ 684 │ UInt64 │ - 686. │ 685 │ UInt64 │ - 687. │ 686 │ UInt64 │ - 688. │ 687 │ UInt64 │ - 689. │ 688 │ UInt64 │ - 690. │ 689 │ UInt64 │ - 691. │ 690 │ UInt64 │ - 692. │ 691 │ UInt64 │ - 693. │ 692 │ UInt64 │ - 694. │ 693 │ UInt64 │ - 695. │ 694 │ UInt64 │ - 696. │ 695 │ UInt64 │ - 697. │ 696 │ UInt64 │ - 698. │ 697 │ UInt64 │ - 699. │ 698 │ UInt64 │ - 700. │ 699 │ UInt64 │ - 701. │ 700 │ UInt64 │ - 702. │ 701 │ UInt64 │ - 703. │ 702 │ UInt64 │ - 704. │ 703 │ UInt64 │ - 705. │ 704 │ UInt64 │ - 706. │ 705 │ UInt64 │ - 707. │ 706 │ UInt64 │ - 708. │ 707 │ UInt64 │ - 709. │ 708 │ UInt64 │ - 710. │ 709 │ UInt64 │ - 711. │ 710 │ UInt64 │ - 712. │ 711 │ UInt64 │ - 713. │ 712 │ UInt64 │ - 714. │ 713 │ UInt64 │ - 715. │ 714 │ UInt64 │ - 716. │ 715 │ UInt64 │ - 717. │ 716 │ UInt64 │ - 718. │ 717 │ UInt64 │ - 719. │ 718 │ UInt64 │ - 720. │ 719 │ UInt64 │ - 721. │ 720 │ UInt64 │ - 722. │ 721 │ UInt64 │ - 723. │ 722 │ UInt64 │ - 724. │ 723 │ UInt64 │ - 725. │ 724 │ UInt64 │ - 726. │ 725 │ UInt64 │ - 727. │ 726 │ UInt64 │ - 728. │ 727 │ UInt64 │ - 729. │ 728 │ UInt64 │ - 730. │ 729 │ UInt64 │ - 731. │ 730 │ UInt64 │ - 732. │ 731 │ UInt64 │ - 733. │ 732 │ UInt64 │ - 734. │ 733 │ UInt64 │ - 735. │ 734 │ UInt64 │ - 736. │ 735 │ UInt64 │ - 737. │ 736 │ UInt64 │ - 738. │ 737 │ UInt64 │ - 739. │ 738 │ UInt64 │ - 740. │ 739 │ UInt64 │ - 741. │ 740 │ UInt64 │ - 742. │ 741 │ UInt64 │ - 743. │ 742 │ UInt64 │ - 744. │ 743 │ UInt64 │ - 745. │ 744 │ UInt64 │ - 746. │ 745 │ UInt64 │ - 747. │ 746 │ UInt64 │ - 748. │ 747 │ UInt64 │ - 749. │ 748 │ UInt64 │ - 750. │ 749 │ UInt64 │ - 751. │ 750 │ UInt64 │ - 752. │ 751 │ UInt64 │ - 753. │ 752 │ UInt64 │ - 754. │ 753 │ UInt64 │ - 755. │ 754 │ UInt64 │ - 756. │ 755 │ UInt64 │ - 757. │ 756 │ UInt64 │ - 758. │ 757 │ UInt64 │ - 759. │ 758 │ UInt64 │ - 760. │ 759 │ UInt64 │ - 761. │ 760 │ UInt64 │ - 762. │ 761 │ UInt64 │ - 763. │ 762 │ UInt64 │ - 764. │ 763 │ UInt64 │ - 765. │ 764 │ UInt64 │ - 766. │ 765 │ UInt64 │ - 767. │ 766 │ UInt64 │ - 768. │ 767 │ UInt64 │ - 769. │ 768 │ UInt64 │ - 770. │ 769 │ UInt64 │ - 771. │ 770 │ UInt64 │ - 772. │ 771 │ UInt64 │ - 773. │ 772 │ UInt64 │ - 774. │ 773 │ UInt64 │ - 775. │ 774 │ UInt64 │ - 776. │ 775 │ UInt64 │ - 777. │ 776 │ UInt64 │ - 778. │ 777 │ UInt64 │ - 779. │ 778 │ UInt64 │ - 780. │ 779 │ UInt64 │ - 781. │ 780 │ UInt64 │ - 782. │ 781 │ UInt64 │ - 783. │ 782 │ UInt64 │ - 784. │ 783 │ UInt64 │ - 785. │ 784 │ UInt64 │ - 786. │ 785 │ UInt64 │ - 787. │ 786 │ UInt64 │ - 788. │ 787 │ UInt64 │ - 789. │ 788 │ UInt64 │ - 790. │ 789 │ UInt64 │ - 791. │ 790 │ UInt64 │ - 792. │ 791 │ UInt64 │ - 793. │ 792 │ UInt64 │ - 794. │ 793 │ UInt64 │ - 795. │ 794 │ UInt64 │ - 796. │ 795 │ UInt64 │ - 797. │ 796 │ UInt64 │ - 798. │ 797 │ UInt64 │ - 799. │ 798 │ UInt64 │ - 800. │ 799 │ UInt64 │ - 801. │ 800 │ UInt64 │ - 802. │ 801 │ UInt64 │ - 803. │ 802 │ UInt64 │ - 804. │ 803 │ UInt64 │ - 805. │ 804 │ UInt64 │ - 806. │ 805 │ UInt64 │ - 807. │ 806 │ UInt64 │ - 808. │ 807 │ UInt64 │ - 809. │ 808 │ UInt64 │ - 810. │ 809 │ UInt64 │ - 811. │ 810 │ UInt64 │ - 812. │ 811 │ UInt64 │ - 813. │ 812 │ UInt64 │ - 814. │ 813 │ UInt64 │ - 815. │ 814 │ UInt64 │ - 816. │ 815 │ UInt64 │ - 817. │ 816 │ UInt64 │ - 818. │ 817 │ UInt64 │ - 819. │ 818 │ UInt64 │ - 820. │ 819 │ UInt64 │ - 821. │ 820 │ UInt64 │ - 822. │ 821 │ UInt64 │ - 823. │ 822 │ UInt64 │ - 824. │ 823 │ UInt64 │ - 825. │ 824 │ UInt64 │ - 826. │ 825 │ UInt64 │ - 827. │ 826 │ UInt64 │ - 828. │ 827 │ UInt64 │ - 829. │ 828 │ UInt64 │ - 830. │ 829 │ UInt64 │ - 831. │ 830 │ UInt64 │ - 832. │ 831 │ UInt64 │ - 833. │ 832 │ UInt64 │ - 834. │ 833 │ UInt64 │ - 835. │ 834 │ UInt64 │ - 836. │ 835 │ UInt64 │ - 837. │ 836 │ UInt64 │ - 838. │ 837 │ UInt64 │ - 839. │ 838 │ UInt64 │ - 840. │ 839 │ UInt64 │ - 841. │ 840 │ UInt64 │ - 842. │ 841 │ UInt64 │ - 843. │ 842 │ UInt64 │ - 844. │ 843 │ UInt64 │ - 845. │ 844 │ UInt64 │ - 846. │ 845 │ UInt64 │ - 847. │ 846 │ UInt64 │ - 848. │ 847 │ UInt64 │ - 849. │ 848 │ UInt64 │ - 850. │ 849 │ UInt64 │ - 851. │ 850 │ UInt64 │ - 852. │ 851 │ UInt64 │ - 853. │ 852 │ UInt64 │ - 854. │ 853 │ UInt64 │ - 855. │ 854 │ UInt64 │ - 856. │ 855 │ UInt64 │ - 857. │ 856 │ UInt64 │ - 858. │ 857 │ UInt64 │ - 859. │ 858 │ UInt64 │ - 860. │ 859 │ UInt64 │ - 861. │ 860 │ UInt64 │ - 862. │ 861 │ UInt64 │ - 863. │ 862 │ UInt64 │ - 864. │ 863 │ UInt64 │ - 865. │ 864 │ UInt64 │ - 866. │ 865 │ UInt64 │ - 867. │ 866 │ UInt64 │ - 868. │ 867 │ UInt64 │ - 869. │ 868 │ UInt64 │ - 870. │ 869 │ UInt64 │ - 871. │ 870 │ UInt64 │ - 872. │ 871 │ UInt64 │ - 873. │ 872 │ UInt64 │ - 874. │ 873 │ UInt64 │ - 875. │ 874 │ UInt64 │ - 876. │ 875 │ UInt64 │ - 877. │ 876 │ UInt64 │ - 878. │ 877 │ UInt64 │ - 879. │ 878 │ UInt64 │ - 880. │ 879 │ UInt64 │ - 881. │ 880 │ UInt64 │ - 882. │ 881 │ UInt64 │ - 883. │ 882 │ UInt64 │ - 884. │ 883 │ UInt64 │ - 885. │ 884 │ UInt64 │ - 886. │ 885 │ UInt64 │ - 887. │ 886 │ UInt64 │ - 888. │ 887 │ UInt64 │ - 889. │ 888 │ UInt64 │ - 890. │ 889 │ UInt64 │ - 891. │ 890 │ UInt64 │ - 892. │ 891 │ UInt64 │ - 893. │ 892 │ UInt64 │ - 894. │ 893 │ UInt64 │ - 895. │ 894 │ UInt64 │ - 896. │ 895 │ UInt64 │ - 897. │ 896 │ UInt64 │ - 898. │ 897 │ UInt64 │ - 899. │ 898 │ UInt64 │ - 900. │ 899 │ UInt64 │ - 901. │ 900 │ UInt64 │ - 902. │ 901 │ UInt64 │ - 903. │ 902 │ UInt64 │ - 904. │ 903 │ UInt64 │ - 905. │ 904 │ UInt64 │ - 906. │ 905 │ UInt64 │ - 907. │ 906 │ UInt64 │ - 908. │ 907 │ UInt64 │ - 909. │ 908 │ UInt64 │ - 910. │ 909 │ UInt64 │ - 911. │ 910 │ UInt64 │ - 912. │ 911 │ UInt64 │ - 913. │ 912 │ UInt64 │ - 914. │ 913 │ UInt64 │ - 915. │ 914 │ UInt64 │ - 916. │ 915 │ UInt64 │ - 917. │ 916 │ UInt64 │ - 918. │ 917 │ UInt64 │ - 919. │ 918 │ UInt64 │ - 920. │ 919 │ UInt64 │ - 921. │ 920 │ UInt64 │ - 922. │ 921 │ UInt64 │ - 923. │ 922 │ UInt64 │ - 924. │ 923 │ UInt64 │ - 925. │ 924 │ UInt64 │ - 926. │ 925 │ UInt64 │ - 927. │ 926 │ UInt64 │ - 928. │ 927 │ UInt64 │ - 929. │ 928 │ UInt64 │ - 930. │ 929 │ UInt64 │ - 931. │ 930 │ UInt64 │ - 932. │ 931 │ UInt64 │ - 933. │ 932 │ UInt64 │ - 934. │ 933 │ UInt64 │ - 935. │ 934 │ UInt64 │ - 936. │ 935 │ UInt64 │ - 937. │ 936 │ UInt64 │ - 938. │ 937 │ UInt64 │ - 939. │ 938 │ UInt64 │ - 940. │ 939 │ UInt64 │ - 941. │ 940 │ UInt64 │ - 942. │ 941 │ UInt64 │ - 943. │ 942 │ UInt64 │ - 944. │ 943 │ UInt64 │ - 945. │ 944 │ UInt64 │ - 946. │ 945 │ UInt64 │ - 947. │ 946 │ UInt64 │ - 948. │ 947 │ UInt64 │ - 949. │ 948 │ UInt64 │ - 950. │ 949 │ UInt64 │ - 951. │ 950 │ UInt64 │ - 952. │ 951 │ UInt64 │ - 953. │ 952 │ UInt64 │ - 954. │ 953 │ UInt64 │ - 955. │ 954 │ UInt64 │ - 956. │ 955 │ UInt64 │ - 957. │ 956 │ UInt64 │ - 958. │ 957 │ UInt64 │ - 959. │ 958 │ UInt64 │ - 960. │ 959 │ UInt64 │ - 961. │ 960 │ UInt64 │ - 962. │ 961 │ UInt64 │ - 963. │ 962 │ UInt64 │ - 964. │ 963 │ UInt64 │ - 965. │ 964 │ UInt64 │ - 966. │ 965 │ UInt64 │ - 967. │ 966 │ UInt64 │ - 968. │ 967 │ UInt64 │ - 969. │ 968 │ UInt64 │ - 970. │ 969 │ UInt64 │ - 971. │ 970 │ UInt64 │ - 972. │ 971 │ UInt64 │ - 973. │ 972 │ UInt64 │ - 974. │ 973 │ UInt64 │ - 975. │ 974 │ UInt64 │ - 976. │ 975 │ UInt64 │ - 977. │ 976 │ UInt64 │ - 978. │ 977 │ UInt64 │ - 979. │ 978 │ UInt64 │ - 980. │ 979 │ UInt64 │ - 981. │ 980 │ UInt64 │ - 982. │ 981 │ UInt64 │ - 983. │ 982 │ UInt64 │ - 984. │ 983 │ UInt64 │ - 985. │ 984 │ UInt64 │ - 986. │ 985 │ UInt64 │ - 987. │ 986 │ UInt64 │ - 988. │ 987 │ UInt64 │ - 989. │ 988 │ UInt64 │ - 990. │ 989 │ UInt64 │ - 991. │ 990 │ UInt64 │ - 992. │ 991 │ UInt64 │ - 993. │ 992 │ UInt64 │ - 994. │ 993 │ UInt64 │ - 995. │ 994 │ UInt64 │ - 996. │ 995 │ UInt64 │ - 997. │ 996 │ UInt64 │ - 998. │ 997 │ UInt64 │ - 999. │ 998 │ UInt64 │ -1000. │ 999 │ UInt64 │ - └─number─┴─toTypeName(number)─┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 100. │ 99 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 101. │ 100 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 102. │ 101 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 103. │ 102 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 104. │ 103 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 105. │ 104 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 106. │ 105 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 107. │ 106 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 108. │ 107 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 109. │ 108 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 110. │ 109 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 111. │ 110 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 112. │ 111 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 113. │ 112 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 114. │ 113 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 115. │ 114 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 116. │ 115 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 117. │ 116 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 118. │ 117 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 119. │ 118 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 120. │ 119 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 121. │ 120 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 122. │ 121 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 123. │ 122 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 124. │ 123 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 125. │ 124 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 126. │ 125 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 127. │ 126 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 128. │ 127 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 129. │ 128 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 130. │ 129 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 131. │ 130 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 132. │ 131 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 133. │ 132 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 134. │ 133 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 135. │ 134 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 136. │ 135 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 137. │ 136 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 138. │ 137 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 139. │ 138 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 140. │ 139 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 141. │ 140 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 142. │ 141 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 143. │ 142 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 144. │ 143 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 145. │ 144 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 146. │ 145 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 147. │ 146 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 148. │ 147 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 149. │ 148 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 150. │ 149 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 151. │ 150 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 152. │ 151 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 153. │ 152 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 154. │ 153 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 155. │ 154 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 156. │ 155 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 157. │ 156 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 158. │ 157 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 159. │ 158 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 160. │ 159 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 161. │ 160 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 162. │ 161 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 163. │ 162 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 164. │ 163 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 165. │ 164 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 166. │ 165 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 167. │ 166 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 168. │ 167 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 169. │ 168 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 170. │ 169 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 171. │ 170 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 172. │ 171 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 173. │ 172 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 174. │ 173 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 175. │ 174 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 176. │ 175 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 177. │ 176 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 178. │ 177 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 179. │ 178 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 180. │ 179 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 181. │ 180 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 182. │ 181 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 183. │ 182 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 184. │ 183 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 185. │ 184 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 186. │ 185 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 187. │ 186 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 188. │ 187 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 189. │ 188 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 190. │ 189 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 191. │ 190 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 192. │ 191 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 193. │ 192 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 194. │ 193 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 195. │ 194 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 196. │ 195 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 197. │ 196 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 198. │ 197 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 199. │ 198 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 200. │ 199 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 201. │ 200 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 202. │ 201 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 203. │ 202 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 204. │ 203 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 205. │ 204 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 206. │ 205 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 207. │ 206 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 208. │ 207 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 209. │ 208 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 210. │ 209 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 211. │ 210 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 212. │ 211 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 213. │ 212 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 214. │ 213 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 215. │ 214 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 216. │ 215 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 217. │ 216 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 218. │ 217 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 219. │ 218 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 220. │ 219 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 221. │ 220 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 222. │ 221 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 223. │ 222 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 224. │ 223 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 225. │ 224 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 226. │ 225 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 227. │ 226 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 228. │ 227 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 229. │ 228 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 230. │ 229 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 231. │ 230 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 232. │ 231 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 233. │ 232 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 234. │ 233 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 235. │ 234 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 236. │ 235 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 237. │ 236 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 238. │ 237 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 239. │ 238 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 240. │ 239 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 241. │ 240 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 242. │ 241 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 243. │ 242 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 244. │ 243 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 245. │ 244 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 246. │ 245 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 247. │ 246 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 248. │ 247 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 249. │ 248 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 250. │ 249 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 251. │ 250 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 252. │ 251 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 253. │ 252 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 254. │ 253 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 255. │ 254 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 256. │ 255 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 257. │ 256 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 258. │ 257 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 259. │ 258 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 260. │ 259 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 261. │ 260 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 262. │ 261 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 263. │ 262 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 264. │ 263 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 265. │ 264 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 266. │ 265 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 267. │ 266 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 268. │ 267 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 269. │ 268 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 270. │ 269 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 271. │ 270 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 272. │ 271 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 273. │ 272 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 274. │ 273 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 275. │ 274 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 276. │ 275 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 277. │ 276 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 278. │ 277 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 279. │ 278 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 280. │ 279 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 281. │ 280 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 282. │ 281 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 283. │ 282 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 284. │ 283 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 285. │ 284 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 286. │ 285 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 287. │ 286 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 288. │ 287 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 289. │ 288 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 290. │ 289 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 291. │ 290 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 292. │ 291 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 293. │ 292 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 294. │ 293 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 295. │ 294 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 296. │ 295 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 297. │ 296 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 298. │ 297 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 299. │ 298 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 300. │ 299 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 301. │ 300 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 302. │ 301 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 303. │ 302 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 304. │ 303 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 305. │ 304 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 306. │ 305 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 307. │ 306 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 308. │ 307 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 309. │ 308 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 310. │ 309 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 311. │ 310 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 312. │ 311 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 313. │ 312 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 314. │ 313 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 315. │ 314 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 316. │ 315 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 317. │ 316 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 318. │ 317 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 319. │ 318 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 320. │ 319 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 321. │ 320 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 322. │ 321 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 323. │ 322 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 324. │ 323 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 325. │ 324 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 326. │ 325 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 327. │ 326 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 328. │ 327 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 329. │ 328 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 330. │ 329 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 331. │ 330 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 332. │ 331 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 333. │ 332 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 334. │ 333 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 335. │ 334 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 336. │ 335 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 337. │ 336 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 338. │ 337 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 339. │ 338 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 340. │ 339 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 341. │ 340 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 342. │ 341 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 343. │ 342 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 344. │ 343 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 345. │ 344 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 346. │ 345 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 347. │ 346 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 348. │ 347 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 349. │ 348 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 350. │ 349 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 351. │ 350 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 352. │ 351 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 353. │ 352 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 354. │ 353 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 355. │ 354 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 356. │ 355 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 357. │ 356 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 358. │ 357 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 359. │ 358 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 360. │ 359 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 361. │ 360 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 362. │ 361 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 363. │ 362 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 364. │ 363 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 365. │ 364 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 366. │ 365 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 367. │ 366 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 368. │ 367 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 369. │ 368 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 370. │ 369 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 371. │ 370 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 372. │ 371 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 373. │ 372 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 374. │ 373 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 375. │ 374 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 376. │ 375 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 377. │ 376 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 378. │ 377 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 379. │ 378 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 380. │ 379 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 381. │ 380 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 382. │ 381 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 383. │ 382 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 384. │ 383 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 385. │ 384 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 386. │ 385 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 387. │ 386 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 388. │ 387 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 389. │ 388 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 390. │ 389 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 391. │ 390 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 392. │ 391 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 393. │ 392 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 394. │ 393 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 395. │ 394 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 396. │ 395 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 397. │ 396 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 398. │ 397 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 399. │ 398 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 400. │ 399 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 401. │ 400 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 402. │ 401 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 403. │ 402 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 404. │ 403 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 405. │ 404 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 406. │ 405 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 407. │ 406 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 408. │ 407 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 409. │ 408 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 410. │ 409 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 411. │ 410 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 412. │ 411 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 413. │ 412 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 414. │ 413 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 415. │ 414 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 416. │ 415 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 417. │ 416 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 418. │ 417 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 419. │ 418 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 420. │ 419 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 421. │ 420 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 422. │ 421 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 423. │ 422 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 424. │ 423 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 425. │ 424 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 426. │ 425 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 427. │ 426 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 428. │ 427 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 429. │ 428 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 430. │ 429 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 431. │ 430 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 432. │ 431 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 433. │ 432 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 434. │ 433 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 435. │ 434 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 436. │ 435 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 437. │ 436 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 438. │ 437 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 439. │ 438 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 440. │ 439 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 441. │ 440 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 442. │ 441 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 443. │ 442 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 444. │ 443 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 445. │ 444 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 446. │ 445 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 447. │ 446 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 448. │ 447 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 449. │ 448 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 450. │ 449 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 451. │ 450 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 452. │ 451 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 453. │ 452 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 454. │ 453 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 455. │ 454 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 456. │ 455 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 457. │ 456 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 458. │ 457 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 459. │ 458 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 460. │ 459 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 461. │ 460 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 462. │ 461 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 463. │ 462 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 464. │ 463 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 465. │ 464 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 466. │ 465 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 467. │ 466 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 468. │ 467 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 469. │ 468 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 470. │ 469 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 471. │ 470 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 472. │ 471 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 473. │ 472 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 474. │ 473 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 475. │ 474 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 476. │ 475 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 477. │ 476 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 478. │ 477 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 479. │ 478 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 480. │ 479 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 481. │ 480 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 482. │ 481 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 483. │ 482 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 484. │ 483 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 485. │ 484 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 486. │ 485 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 487. │ 486 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 488. │ 487 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 489. │ 488 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 490. │ 489 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 491. │ 490 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 492. │ 491 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 493. │ 492 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 494. │ 493 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 495. │ 494 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 496. │ 495 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 497. │ 496 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 498. │ 497 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 499. │ 498 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 500. │ 499 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 501. │ 500 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 502. │ 501 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 503. │ 502 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 504. │ 503 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 505. │ 504 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 506. │ 505 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 507. │ 506 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 508. │ 507 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 509. │ 508 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 510. │ 509 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 511. │ 510 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 512. │ 511 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 513. │ 512 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 514. │ 513 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 515. │ 514 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 516. │ 515 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 517. │ 516 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 518. │ 517 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 519. │ 518 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 520. │ 519 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 521. │ 520 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 522. │ 521 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 523. │ 522 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 524. │ 523 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 525. │ 524 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 526. │ 525 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 527. │ 526 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 528. │ 527 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 529. │ 528 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 530. │ 529 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 531. │ 530 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 532. │ 531 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 533. │ 532 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 534. │ 533 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 535. │ 534 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 536. │ 535 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 537. │ 536 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 538. │ 537 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 539. │ 538 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 540. │ 539 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 541. │ 540 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 542. │ 541 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 543. │ 542 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 544. │ 543 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 545. │ 544 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 546. │ 545 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 547. │ 546 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 548. │ 547 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 549. │ 548 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 550. │ 549 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 551. │ 550 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 552. │ 551 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 553. │ 552 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 554. │ 553 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 555. │ 554 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 556. │ 555 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 557. │ 556 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 558. │ 557 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 559. │ 558 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 560. │ 559 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 561. │ 560 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 562. │ 561 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 563. │ 562 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 564. │ 563 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 565. │ 564 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 566. │ 565 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 567. │ 566 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 568. │ 567 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 569. │ 568 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 570. │ 569 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 571. │ 570 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 572. │ 571 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 573. │ 572 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 574. │ 573 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 575. │ 574 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 576. │ 575 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 577. │ 576 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 578. │ 577 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 579. │ 578 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 580. │ 579 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 581. │ 580 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 582. │ 581 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 583. │ 582 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 584. │ 583 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 585. │ 584 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 586. │ 585 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 587. │ 586 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 588. │ 587 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 589. │ 588 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 590. │ 589 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 591. │ 590 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 592. │ 591 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 593. │ 592 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 594. │ 593 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 595. │ 594 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 596. │ 595 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 597. │ 596 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 598. │ 597 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 599. │ 598 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 600. │ 599 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 601. │ 600 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 602. │ 601 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 603. │ 602 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 604. │ 603 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 605. │ 604 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 606. │ 605 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 607. │ 606 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 608. │ 607 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 609. │ 608 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 610. │ 609 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 611. │ 610 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 612. │ 611 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 613. │ 612 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 614. │ 613 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 615. │ 614 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 616. │ 615 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 617. │ 616 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 618. │ 617 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 619. │ 618 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 620. │ 619 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 621. │ 620 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 622. │ 621 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 623. │ 622 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 624. │ 623 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 625. │ 624 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 626. │ 625 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 627. │ 626 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 628. │ 627 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 629. │ 628 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 630. │ 629 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 631. │ 630 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 632. │ 631 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 633. │ 632 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 634. │ 633 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 635. │ 634 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 636. │ 635 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 637. │ 636 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 638. │ 637 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 639. │ 638 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 640. │ 639 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 641. │ 640 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 642. │ 641 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 643. │ 642 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 644. │ 643 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 645. │ 644 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 646. │ 645 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 647. │ 646 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 648. │ 647 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 649. │ 648 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 650. │ 649 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 651. │ 650 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 652. │ 651 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 653. │ 652 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 654. │ 653 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 655. │ 654 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 656. │ 655 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 657. │ 656 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 658. │ 657 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 659. │ 658 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 660. │ 659 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 661. │ 660 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 662. │ 661 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 663. │ 662 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 664. │ 663 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 665. │ 664 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 666. │ 665 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 667. │ 666 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 668. │ 667 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 669. │ 668 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 670. │ 669 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 671. │ 670 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 672. │ 671 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 673. │ 672 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 674. │ 673 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 675. │ 674 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 676. │ 675 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 677. │ 676 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 678. │ 677 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 679. │ 678 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 680. │ 679 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 681. │ 680 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 682. │ 681 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 683. │ 682 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 684. │ 683 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 685. │ 684 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 686. │ 685 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 687. │ 686 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 688. │ 687 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 689. │ 688 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 690. │ 689 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 691. │ 690 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 692. │ 691 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 693. │ 692 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 694. │ 693 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 695. │ 694 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 696. │ 695 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 697. │ 696 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 698. │ 697 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 699. │ 698 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 700. │ 699 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 701. │ 700 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 702. │ 701 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 703. │ 702 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 704. │ 703 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 705. │ 704 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 706. │ 705 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 707. │ 706 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 708. │ 707 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 709. │ 708 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 710. │ 709 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 711. │ 710 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 712. │ 711 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 713. │ 712 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 714. │ 713 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 715. │ 714 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 716. │ 715 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 717. │ 716 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 718. │ 717 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 719. │ 718 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 720. │ 719 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 721. │ 720 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 722. │ 721 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 723. │ 722 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 724. │ 723 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 725. │ 724 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 726. │ 725 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 727. │ 726 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 728. │ 727 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 729. │ 728 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 730. │ 729 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 731. │ 730 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 732. │ 731 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 733. │ 732 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 734. │ 733 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 735. │ 734 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 736. │ 735 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 737. │ 736 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 738. │ 737 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 739. │ 738 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 740. │ 739 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 741. │ 740 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 742. │ 741 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 743. │ 742 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 744. │ 743 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 745. │ 744 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 746. │ 745 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 747. │ 746 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 748. │ 747 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 749. │ 748 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 750. │ 749 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 751. │ 750 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 752. │ 751 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 753. │ 752 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 754. │ 753 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 755. │ 754 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 756. │ 755 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 757. │ 756 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 758. │ 757 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 759. │ 758 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 760. │ 759 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 761. │ 760 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 762. │ 761 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 763. │ 762 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 764. │ 763 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 765. │ 764 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 766. │ 765 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 767. │ 766 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 768. │ 767 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 769. │ 768 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 770. │ 769 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 771. │ 770 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 772. │ 771 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 773. │ 772 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 774. │ 773 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 775. │ 774 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 776. │ 775 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 777. │ 776 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 778. │ 777 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 779. │ 778 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 780. │ 779 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 781. │ 780 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 782. │ 781 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 783. │ 782 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 784. │ 783 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 785. │ 784 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 786. │ 785 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 787. │ 786 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 788. │ 787 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 789. │ 788 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 790. │ 789 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 791. │ 790 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 792. │ 791 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 793. │ 792 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 794. │ 793 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 795. │ 794 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 796. │ 795 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 797. │ 796 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 798. │ 797 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 799. │ 798 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 800. │ 799 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 801. │ 800 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 802. │ 801 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 803. │ 802 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 804. │ 803 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 805. │ 804 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 806. │ 805 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 807. │ 806 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 808. │ 807 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 809. │ 808 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 810. │ 809 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 811. │ 810 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 812. │ 811 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 813. │ 812 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 814. │ 813 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 815. │ 814 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 816. │ 815 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 817. │ 816 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 818. │ 817 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 819. │ 818 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 820. │ 819 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 821. │ 820 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 822. │ 821 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 823. │ 822 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 824. │ 823 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 825. │ 824 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 826. │ 825 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 827. │ 826 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 828. │ 827 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 829. │ 828 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 830. │ 829 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 831. │ 830 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 832. │ 831 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 833. │ 832 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 834. │ 833 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 835. │ 834 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 836. │ 835 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 837. │ 836 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 838. │ 837 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 839. │ 838 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 840. │ 839 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 841. │ 840 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 842. │ 841 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 843. │ 842 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 844. │ 843 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 845. │ 844 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 846. │ 845 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 847. │ 846 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 848. │ 847 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 849. │ 848 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 850. │ 849 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 851. │ 850 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 852. │ 851 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 853. │ 852 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 854. │ 853 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 855. │ 854 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 856. │ 855 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 857. │ 856 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 858. │ 857 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 859. │ 858 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 860. │ 859 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 861. │ 860 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 862. │ 861 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 863. │ 862 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 864. │ 863 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 865. │ 864 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 866. │ 865 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 867. │ 866 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 868. │ 867 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 869. │ 868 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 870. │ 869 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 871. │ 870 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 872. │ 871 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 873. │ 872 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 874. │ 873 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 875. │ 874 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 876. │ 875 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 877. │ 876 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 878. │ 877 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 879. │ 878 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 880. │ 879 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 881. │ 880 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 882. │ 881 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 883. │ 882 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 884. │ 883 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 885. │ 884 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 886. │ 885 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 887. │ 886 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 888. │ 887 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 889. │ 888 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 890. │ 889 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 891. │ 890 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 892. │ 891 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 893. │ 892 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 894. │ 893 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 895. │ 894 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 896. │ 895 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 897. │ 896 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 898. │ 897 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 899. │ 898 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 900. │ 899 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 901. │ 900 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 902. │ 901 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 903. │ 902 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 904. │ 903 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 905. │ 904 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 906. │ 905 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 907. │ 906 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 908. │ 907 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 909. │ 908 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 910. │ 909 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 911. │ 910 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 912. │ 911 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 913. │ 912 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 914. │ 913 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 915. │ 914 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 916. │ 915 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 917. │ 916 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 918. │ 917 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 919. │ 918 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 920. │ 919 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 921. │ 920 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 922. │ 921 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 923. │ 922 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 924. │ 923 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 925. │ 924 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 926. │ 925 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 927. │ 926 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 928. │ 927 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 929. │ 928 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 930. │ 929 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 931. │ 930 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 932. │ 931 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 933. │ 932 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 934. │ 933 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 935. │ 934 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 936. │ 935 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 937. │ 936 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 938. │ 937 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 939. │ 938 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 940. │ 939 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 941. │ 940 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 942. │ 941 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 943. │ 942 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 944. │ 943 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 945. │ 944 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 946. │ 945 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 947. │ 946 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 948. │ 947 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 949. │ 948 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 950. │ 949 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 951. │ 950 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 952. │ 951 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 953. │ 952 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 954. │ 953 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 955. │ 954 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 956. │ 955 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 957. │ 956 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 958. │ 957 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 959. │ 958 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 960. │ 959 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 961. │ 960 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 962. │ 961 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 963. │ 962 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 964. │ 963 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 965. │ 964 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 966. │ 965 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 967. │ 966 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 968. │ 967 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 969. │ 968 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 970. │ 969 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 971. │ 970 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 972. │ 971 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 973. │ 972 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 974. │ 973 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 975. │ 974 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 976. │ 975 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 977. │ 976 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 978. │ 977 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 979. │ 978 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 980. │ 979 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 981. │ 980 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 982. │ 981 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 983. │ 982 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 984. │ 983 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 985. │ 984 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 986. │ 985 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 987. │ 986 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 988. │ 987 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 989. │ 988 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 990. │ 989 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 991. │ 990 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 992. │ 991 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 993. │ 992 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 994. │ 993 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 995. │ 994 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 996. │ 995 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 997. │ 996 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 998. │ 997 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 999. │ 998 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +1000. │ 999 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 100. │ 99 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 101. │ 100 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 102. │ 101 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 103. │ 102 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 104. │ 103 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 105. │ 104 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 106. │ 105 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 107. │ 106 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 108. │ 107 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 109. │ 108 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 110. │ 109 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 111. │ 110 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 112. │ 111 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 113. │ 112 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 114. │ 113 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 115. │ 114 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 116. │ 115 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 117. │ 116 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 118. │ 117 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 119. │ 118 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 120. │ 119 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 121. │ 120 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 122. │ 121 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 123. │ 122 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 124. │ 123 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 125. │ 124 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 126. │ 125 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 127. │ 126 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 128. │ 127 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 129. │ 128 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 130. │ 129 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 131. │ 130 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 132. │ 131 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 133. │ 132 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 134. │ 133 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 135. │ 134 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 136. │ 135 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 137. │ 136 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 138. │ 137 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 139. │ 138 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 140. │ 139 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 141. │ 140 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 142. │ 141 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 143. │ 142 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 144. │ 143 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 145. │ 144 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 146. │ 145 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 147. │ 146 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 148. │ 147 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 149. │ 148 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 150. │ 149 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 151. │ 150 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 152. │ 151 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 153. │ 152 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 154. │ 153 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 155. │ 154 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 156. │ 155 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 157. │ 156 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 158. │ 157 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 159. │ 158 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 160. │ 159 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 161. │ 160 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 162. │ 161 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 163. │ 162 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 164. │ 163 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 165. │ 164 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 166. │ 165 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 167. │ 166 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 168. │ 167 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 169. │ 168 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 170. │ 169 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 171. │ 170 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 172. │ 171 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 173. │ 172 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 174. │ 173 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 175. │ 174 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 176. │ 175 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 177. │ 176 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 178. │ 177 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 179. │ 178 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 180. │ 179 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 181. │ 180 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 182. │ 181 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 183. │ 182 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 184. │ 183 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 185. │ 184 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 186. │ 185 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 187. │ 186 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 188. │ 187 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 189. │ 188 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 190. │ 189 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 191. │ 190 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 192. │ 191 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 193. │ 192 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 194. │ 193 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 195. │ 194 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 196. │ 195 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 197. │ 196 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 198. │ 197 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 199. │ 198 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 200. │ 199 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 201. │ 200 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 202. │ 201 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 203. │ 202 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 204. │ 203 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 205. │ 204 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 206. │ 205 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 207. │ 206 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 208. │ 207 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 209. │ 208 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 210. │ 209 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 211. │ 210 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 212. │ 211 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 213. │ 212 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 214. │ 213 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 215. │ 214 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 216. │ 215 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 217. │ 216 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 218. │ 217 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 219. │ 218 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 220. │ 219 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 221. │ 220 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 222. │ 221 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 223. │ 222 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 224. │ 223 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 225. │ 224 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 226. │ 225 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 227. │ 226 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 228. │ 227 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 229. │ 228 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 230. │ 229 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 231. │ 230 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 232. │ 231 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 233. │ 232 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 234. │ 233 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 235. │ 234 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 236. │ 235 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 237. │ 236 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 238. │ 237 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 239. │ 238 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 240. │ 239 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 241. │ 240 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 242. │ 241 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 243. │ 242 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 244. │ 243 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 245. │ 244 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 246. │ 245 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 247. │ 246 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 248. │ 247 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 249. │ 248 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 250. │ 249 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 251. │ 250 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 252. │ 251 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 253. │ 252 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 254. │ 253 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 255. │ 254 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 256. │ 255 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 257. │ 256 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 258. │ 257 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 259. │ 258 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 260. │ 259 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 261. │ 260 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 262. │ 261 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 263. │ 262 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 264. │ 263 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 265. │ 264 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 266. │ 265 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 267. │ 266 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 268. │ 267 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 269. │ 268 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 270. │ 269 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 271. │ 270 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 272. │ 271 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 273. │ 272 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 274. │ 273 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 275. │ 274 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 276. │ 275 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 277. │ 276 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 278. │ 277 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 279. │ 278 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 280. │ 279 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 281. │ 280 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 282. │ 281 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 283. │ 282 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 284. │ 283 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 285. │ 284 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 286. │ 285 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 287. │ 286 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 288. │ 287 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 289. │ 288 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 290. │ 289 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 291. │ 290 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 292. │ 291 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 293. │ 292 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 294. │ 293 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 295. │ 294 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 296. │ 295 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 297. │ 296 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 298. │ 297 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 299. │ 298 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 300. │ 299 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 301. │ 300 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 302. │ 301 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 303. │ 302 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 304. │ 303 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 305. │ 304 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 306. │ 305 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 307. │ 306 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 308. │ 307 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 309. │ 308 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 310. │ 309 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 311. │ 310 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 312. │ 311 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 313. │ 312 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 314. │ 313 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 315. │ 314 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 316. │ 315 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 317. │ 316 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 318. │ 317 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 319. │ 318 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 320. │ 319 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 321. │ 320 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 322. │ 321 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 323. │ 322 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 324. │ 323 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 325. │ 324 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 326. │ 325 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 327. │ 326 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 328. │ 327 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 329. │ 328 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 330. │ 329 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 331. │ 330 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 332. │ 331 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 333. │ 332 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 334. │ 333 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 335. │ 334 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 336. │ 335 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 337. │ 336 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 338. │ 337 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 339. │ 338 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 340. │ 339 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 341. │ 340 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 342. │ 341 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 343. │ 342 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 344. │ 343 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 345. │ 344 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 346. │ 345 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 347. │ 346 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 348. │ 347 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 349. │ 348 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 350. │ 349 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 351. │ 350 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 352. │ 351 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 353. │ 352 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 354. │ 353 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 355. │ 354 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 356. │ 355 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 357. │ 356 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 358. │ 357 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 359. │ 358 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 360. │ 359 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 361. │ 360 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 362. │ 361 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 363. │ 362 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 364. │ 363 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 365. │ 364 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 366. │ 365 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 367. │ 366 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 368. │ 367 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 369. │ 368 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 370. │ 369 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 371. │ 370 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 372. │ 371 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 373. │ 372 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 374. │ 373 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 375. │ 374 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 376. │ 375 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 377. │ 376 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 378. │ 377 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 379. │ 378 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 380. │ 379 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 381. │ 380 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 382. │ 381 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 383. │ 382 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 384. │ 383 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 385. │ 384 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 386. │ 385 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 387. │ 386 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 388. │ 387 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 389. │ 388 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 390. │ 389 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 391. │ 390 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 392. │ 391 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 393. │ 392 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 394. │ 393 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 395. │ 394 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 396. │ 395 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 397. │ 396 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 398. │ 397 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 399. │ 398 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 400. │ 399 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 401. │ 400 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 402. │ 401 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 403. │ 402 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 404. │ 403 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 405. │ 404 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 406. │ 405 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 407. │ 406 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 408. │ 407 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 409. │ 408 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 410. │ 409 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 411. │ 410 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 412. │ 411 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 413. │ 412 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 414. │ 413 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 415. │ 414 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 416. │ 415 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 417. │ 416 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 418. │ 417 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 419. │ 418 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 420. │ 419 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 421. │ 420 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 422. │ 421 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 423. │ 422 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 424. │ 423 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 425. │ 424 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 426. │ 425 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 427. │ 426 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 428. │ 427 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 429. │ 428 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 430. │ 429 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 431. │ 430 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 432. │ 431 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 433. │ 432 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 434. │ 433 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 435. │ 434 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 436. │ 435 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 437. │ 436 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 438. │ 437 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 439. │ 438 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 440. │ 439 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 441. │ 440 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 442. │ 441 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 443. │ 442 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 444. │ 443 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 445. │ 444 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 446. │ 445 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 447. │ 446 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 448. │ 447 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 449. │ 448 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 450. │ 449 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 451. │ 450 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 452. │ 451 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 453. │ 452 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 454. │ 453 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 455. │ 454 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 456. │ 455 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 457. │ 456 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 458. │ 457 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 459. │ 458 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 460. │ 459 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 461. │ 460 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 462. │ 461 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 463. │ 462 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 464. │ 463 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 465. │ 464 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 466. │ 465 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 467. │ 466 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 468. │ 467 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 469. │ 468 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 470. │ 469 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 471. │ 470 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 472. │ 471 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 473. │ 472 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 474. │ 473 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 475. │ 474 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 476. │ 475 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 477. │ 476 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 478. │ 477 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 479. │ 478 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 480. │ 479 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 481. │ 480 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 482. │ 481 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 483. │ 482 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 484. │ 483 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 485. │ 484 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 486. │ 485 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 487. │ 486 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 488. │ 487 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 489. │ 488 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 490. │ 489 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 491. │ 490 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 492. │ 491 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 493. │ 492 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 494. │ 493 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 495. │ 494 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 496. │ 495 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 497. │ 496 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 498. │ 497 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 499. │ 498 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 500. │ 499 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 501. │ 500 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 502. │ 501 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 503. │ 502 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 504. │ 503 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 505. │ 504 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 506. │ 505 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 507. │ 506 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 508. │ 507 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 509. │ 508 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 510. │ 509 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 511. │ 510 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 512. │ 511 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 513. │ 512 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 514. │ 513 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 515. │ 514 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 516. │ 515 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 517. │ 516 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 518. │ 517 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 519. │ 518 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 520. │ 519 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 521. │ 520 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 522. │ 521 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 523. │ 522 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 524. │ 523 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 525. │ 524 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 526. │ 525 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 527. │ 526 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 528. │ 527 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 529. │ 528 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 530. │ 529 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 531. │ 530 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 532. │ 531 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 533. │ 532 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 534. │ 533 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 535. │ 534 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 536. │ 535 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 537. │ 536 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 538. │ 537 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 539. │ 538 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 540. │ 539 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 541. │ 540 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 542. │ 541 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 543. │ 542 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 544. │ 543 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 545. │ 544 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 546. │ 545 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 547. │ 546 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 548. │ 547 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 549. │ 548 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 550. │ 549 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 551. │ 550 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 552. │ 551 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 553. │ 552 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 554. │ 553 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 555. │ 554 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 556. │ 555 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 557. │ 556 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 558. │ 557 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 559. │ 558 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 560. │ 559 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 561. │ 560 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 562. │ 561 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 563. │ 562 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 564. │ 563 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 565. │ 564 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 566. │ 565 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 567. │ 566 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 568. │ 567 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 569. │ 568 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 570. │ 569 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 571. │ 570 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 572. │ 571 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 573. │ 572 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 574. │ 573 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 575. │ 574 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 576. │ 575 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 577. │ 576 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 578. │ 577 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 579. │ 578 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 580. │ 579 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 581. │ 580 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 582. │ 581 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 583. │ 582 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 584. │ 583 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 585. │ 584 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 586. │ 585 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 587. │ 586 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 588. │ 587 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 589. │ 588 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 590. │ 589 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 591. │ 590 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 592. │ 591 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 593. │ 592 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 594. │ 593 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 595. │ 594 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 596. │ 595 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 597. │ 596 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 598. │ 597 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 599. │ 598 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 600. │ 599 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 601. │ 600 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 602. │ 601 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 603. │ 602 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 604. │ 603 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 605. │ 604 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 606. │ 605 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 607. │ 606 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 608. │ 607 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 609. │ 608 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 610. │ 609 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 611. │ 610 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 612. │ 611 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 613. │ 612 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 614. │ 613 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 615. │ 614 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 616. │ 615 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 617. │ 616 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 618. │ 617 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 619. │ 618 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 620. │ 619 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 621. │ 620 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 622. │ 621 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 623. │ 622 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 624. │ 623 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 625. │ 624 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 626. │ 625 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 627. │ 626 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 628. │ 627 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 629. │ 628 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 630. │ 629 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 631. │ 630 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 632. │ 631 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 633. │ 632 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 634. │ 633 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 635. │ 634 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 636. │ 635 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 637. │ 636 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 638. │ 637 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 639. │ 638 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 640. │ 639 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 641. │ 640 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 642. │ 641 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 643. │ 642 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 644. │ 643 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 645. │ 644 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 646. │ 645 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 647. │ 646 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 648. │ 647 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 649. │ 648 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 650. │ 649 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 651. │ 650 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 652. │ 651 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 653. │ 652 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 654. │ 653 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 655. │ 654 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 656. │ 655 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 657. │ 656 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 658. │ 657 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 659. │ 658 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 660. │ 659 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 661. │ 660 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 662. │ 661 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 663. │ 662 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 664. │ 663 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 665. │ 664 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 666. │ 665 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 667. │ 666 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 668. │ 667 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 669. │ 668 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 670. │ 669 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 671. │ 670 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 672. │ 671 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 673. │ 672 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 674. │ 673 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 675. │ 674 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 676. │ 675 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 677. │ 676 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 678. │ 677 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 679. │ 678 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 680. │ 679 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 681. │ 680 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 682. │ 681 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 683. │ 682 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 684. │ 683 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 685. │ 684 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 686. │ 685 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 687. │ 686 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 688. │ 687 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 689. │ 688 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 690. │ 689 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 691. │ 690 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 692. │ 691 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 693. │ 692 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 694. │ 693 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 695. │ 694 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 696. │ 695 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 697. │ 696 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 698. │ 697 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 699. │ 698 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 700. │ 699 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 701. │ 700 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 702. │ 701 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 703. │ 702 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 704. │ 703 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 705. │ 704 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 706. │ 705 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 707. │ 706 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 708. │ 707 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 709. │ 708 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 710. │ 709 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 711. │ 710 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 712. │ 711 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 713. │ 712 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 714. │ 713 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 715. │ 714 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 716. │ 715 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 717. │ 716 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 718. │ 717 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 719. │ 718 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 720. │ 719 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 721. │ 720 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 722. │ 721 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 723. │ 722 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 724. │ 723 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 725. │ 724 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 726. │ 725 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 727. │ 726 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 728. │ 727 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 729. │ 728 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 730. │ 729 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 731. │ 730 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 732. │ 731 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 733. │ 732 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 734. │ 733 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 735. │ 734 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 736. │ 735 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 737. │ 736 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 738. │ 737 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 739. │ 738 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 740. │ 739 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 741. │ 740 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 742. │ 741 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 743. │ 742 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 744. │ 743 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 745. │ 744 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 746. │ 745 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 747. │ 746 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 748. │ 747 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 749. │ 748 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 750. │ 749 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 751. │ 750 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 752. │ 751 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 753. │ 752 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 754. │ 753 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 755. │ 754 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 756. │ 755 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 757. │ 756 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 758. │ 757 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 759. │ 758 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 760. │ 759 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 761. │ 760 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 762. │ 761 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 763. │ 762 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 764. │ 763 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 765. │ 764 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 766. │ 765 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 767. │ 766 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 768. │ 767 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 769. │ 768 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 770. │ 769 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 771. │ 770 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 772. │ 771 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 773. │ 772 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 774. │ 773 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 775. │ 774 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 776. │ 775 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 777. │ 776 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 778. │ 777 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 779. │ 778 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 780. │ 779 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 781. │ 780 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 782. │ 781 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 783. │ 782 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 784. │ 783 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 785. │ 784 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 786. │ 785 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 787. │ 786 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 788. │ 787 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 789. │ 788 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 790. │ 789 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 791. │ 790 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 792. │ 791 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 793. │ 792 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 794. │ 793 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 795. │ 794 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 796. │ 795 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 797. │ 796 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 798. │ 797 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 799. │ 798 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 800. │ 799 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 801. │ 800 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 802. │ 801 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 803. │ 802 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 804. │ 803 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 805. │ 804 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 806. │ 805 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 807. │ 806 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 808. │ 807 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 809. │ 808 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 810. │ 809 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 811. │ 810 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 812. │ 811 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 813. │ 812 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 814. │ 813 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 815. │ 814 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 816. │ 815 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 817. │ 816 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 818. │ 817 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 819. │ 818 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 820. │ 819 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 821. │ 820 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 822. │ 821 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 823. │ 822 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 824. │ 823 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 825. │ 824 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 826. │ 825 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 827. │ 826 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 828. │ 827 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 829. │ 828 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 830. │ 829 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 831. │ 830 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 832. │ 831 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 833. │ 832 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 834. │ 833 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 835. │ 834 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 836. │ 835 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 837. │ 836 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 838. │ 837 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 839. │ 838 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 840. │ 839 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 841. │ 840 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 842. │ 841 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 843. │ 842 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 844. │ 843 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 845. │ 844 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 846. │ 845 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 847. │ 846 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 848. │ 847 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 849. │ 848 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 850. │ 849 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 851. │ 850 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 852. │ 851 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 853. │ 852 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 854. │ 853 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 855. │ 854 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 856. │ 855 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 857. │ 856 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 858. │ 857 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 859. │ 858 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 860. │ 859 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 861. │ 860 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 862. │ 861 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 863. │ 862 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 864. │ 863 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 865. │ 864 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 866. │ 865 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 867. │ 866 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 868. │ 867 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 869. │ 868 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 870. │ 869 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 871. │ 870 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 872. │ 871 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 873. │ 872 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 874. │ 873 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 875. │ 874 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 876. │ 875 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 877. │ 876 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 878. │ 877 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 879. │ 878 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 880. │ 879 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 881. │ 880 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 882. │ 881 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 883. │ 882 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 884. │ 883 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 885. │ 884 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 886. │ 885 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 887. │ 886 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 888. │ 887 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 889. │ 888 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 890. │ 889 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 891. │ 890 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 892. │ 891 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 893. │ 892 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 894. │ 893 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 895. │ 894 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 896. │ 895 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 897. │ 896 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 898. │ 897 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 899. │ 898 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 900. │ 899 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 901. │ 900 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 902. │ 901 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 903. │ 902 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 904. │ 903 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 905. │ 904 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 906. │ 905 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 907. │ 906 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 908. │ 907 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 909. │ 908 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 910. │ 909 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 911. │ 910 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 912. │ 911 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 913. │ 912 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 914. │ 913 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 915. │ 914 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 916. │ 915 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 917. │ 916 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 918. │ 917 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 919. │ 918 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 920. │ 919 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 921. │ 920 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 922. │ 921 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 923. │ 922 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 924. │ 923 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 925. │ 924 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 926. │ 925 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 927. │ 926 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 928. │ 927 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 929. │ 928 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 930. │ 929 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 931. │ 930 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 932. │ 931 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 933. │ 932 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 934. │ 933 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 935. │ 934 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 936. │ 935 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 937. │ 936 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 938. │ 937 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 939. │ 938 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 940. │ 939 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 941. │ 940 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 942. │ 941 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 943. │ 942 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 944. │ 943 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 945. │ 944 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 946. │ 945 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 947. │ 946 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 948. │ 947 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 949. │ 948 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 950. │ 949 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 951. │ 950 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 952. │ 951 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 953. │ 952 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 954. │ 953 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 955. │ 954 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 956. │ 955 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 957. │ 956 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 958. │ 957 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 959. │ 958 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 960. │ 959 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 961. │ 960 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 962. │ 961 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 963. │ 962 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 964. │ 963 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 965. │ 964 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 966. │ 965 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 967. │ 966 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 968. │ 967 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 969. │ 968 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 970. │ 969 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 971. │ 970 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 972. │ 971 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 973. │ 972 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 974. │ 973 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 975. │ 974 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 976. │ 975 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 977. │ 976 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 978. │ 977 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 979. │ 978 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 980. │ 979 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 981. │ 980 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 982. │ 981 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 983. │ 982 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 984. │ 983 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 985. │ 984 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 986. │ 985 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 987. │ 986 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 988. │ 987 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 989. │ 988 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 990. │ 989 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 991. │ 990 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 992. │ 991 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 993. │ 992 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 994. │ 993 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 995. │ 994 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 996. │ 995 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 997. │ 996 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 998. │ 997 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 999. │ 998 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +1000. │ 999 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 100. │ 99 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 101. │ 100 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 102. │ 101 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 103. │ 102 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 104. │ 103 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 105. │ 104 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 106. │ 105 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 107. │ 106 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 108. │ 107 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 109. │ 108 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 110. │ 109 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 111. │ 110 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 112. │ 111 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 113. │ 112 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 114. │ 113 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 115. │ 114 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 116. │ 115 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 117. │ 116 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 118. │ 117 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 119. │ 118 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 120. │ 119 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 121. │ 120 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 122. │ 121 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 123. │ 122 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 124. │ 123 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 125. │ 124 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 126. │ 125 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 127. │ 126 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 128. │ 127 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 129. │ 128 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 130. │ 129 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 131. │ 130 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 132. │ 131 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 133. │ 132 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 134. │ 133 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 135. │ 134 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 136. │ 135 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 137. │ 136 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 138. │ 137 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 139. │ 138 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 140. │ 139 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 141. │ 140 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 142. │ 141 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 143. │ 142 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 144. │ 143 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 145. │ 144 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 146. │ 145 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 147. │ 146 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 148. │ 147 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 149. │ 148 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 150. │ 149 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 151. │ 150 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 152. │ 151 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 153. │ 152 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 154. │ 153 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 155. │ 154 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 156. │ 155 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 157. │ 156 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 158. │ 157 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 159. │ 158 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 160. │ 159 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 161. │ 160 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 162. │ 161 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 163. │ 162 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 164. │ 163 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 165. │ 164 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 166. │ 165 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 167. │ 166 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 168. │ 167 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 169. │ 168 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 170. │ 169 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 171. │ 170 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 172. │ 171 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 173. │ 172 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 174. │ 173 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 175. │ 174 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 176. │ 175 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 177. │ 176 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 178. │ 177 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 179. │ 178 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 180. │ 179 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 181. │ 180 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 182. │ 181 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 183. │ 182 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 184. │ 183 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 185. │ 184 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 186. │ 185 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 187. │ 186 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 188. │ 187 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 189. │ 188 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 190. │ 189 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 191. │ 190 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 192. │ 191 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 193. │ 192 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 194. │ 193 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 195. │ 194 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 196. │ 195 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 197. │ 196 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 198. │ 197 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 199. │ 198 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 200. │ 199 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 201. │ 200 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 202. │ 201 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 203. │ 202 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 204. │ 203 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 205. │ 204 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 206. │ 205 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 207. │ 206 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 208. │ 207 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 209. │ 208 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 210. │ 209 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 211. │ 210 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 212. │ 211 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 213. │ 212 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 214. │ 213 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 215. │ 214 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 216. │ 215 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 217. │ 216 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 218. │ 217 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 219. │ 218 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 220. │ 219 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 221. │ 220 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 222. │ 221 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 223. │ 222 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 224. │ 223 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 225. │ 224 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 226. │ 225 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 227. │ 226 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 228. │ 227 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 229. │ 228 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 230. │ 229 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 231. │ 230 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 232. │ 231 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 233. │ 232 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 234. │ 233 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 235. │ 234 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 236. │ 235 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 237. │ 236 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 238. │ 237 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 239. │ 238 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 240. │ 239 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 241. │ 240 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 242. │ 241 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 243. │ 242 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 244. │ 243 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 245. │ 244 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 246. │ 245 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 247. │ 246 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 248. │ 247 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 249. │ 248 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 250. │ 249 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 251. │ 250 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 252. │ 251 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 253. │ 252 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 254. │ 253 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 255. │ 254 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 256. │ 255 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 257. │ 256 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 258. │ 257 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 259. │ 258 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 260. │ 259 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 261. │ 260 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 262. │ 261 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 263. │ 262 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 264. │ 263 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 265. │ 264 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 266. │ 265 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 267. │ 266 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 268. │ 267 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 269. │ 268 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 270. │ 269 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 271. │ 270 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 272. │ 271 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 273. │ 272 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 274. │ 273 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 275. │ 274 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 276. │ 275 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 277. │ 276 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 278. │ 277 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 279. │ 278 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 280. │ 279 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 281. │ 280 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 282. │ 281 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 283. │ 282 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 284. │ 283 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 285. │ 284 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 286. │ 285 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 287. │ 286 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 288. │ 287 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 289. │ 288 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 290. │ 289 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 291. │ 290 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 292. │ 291 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 293. │ 292 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 294. │ 293 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 295. │ 294 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 296. │ 295 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 297. │ 296 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 298. │ 297 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 299. │ 298 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 300. │ 299 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 301. │ 300 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 302. │ 301 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 303. │ 302 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 304. │ 303 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 305. │ 304 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 306. │ 305 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 307. │ 306 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 308. │ 307 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 309. │ 308 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 310. │ 309 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 311. │ 310 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 312. │ 311 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 313. │ 312 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 314. │ 313 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 315. │ 314 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 316. │ 315 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 317. │ 316 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 318. │ 317 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 319. │ 318 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 320. │ 319 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 321. │ 320 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 322. │ 321 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 323. │ 322 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 324. │ 323 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 325. │ 324 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 326. │ 325 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 327. │ 326 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 328. │ 327 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 329. │ 328 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 330. │ 329 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 331. │ 330 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 332. │ 331 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 333. │ 332 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 334. │ 333 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 335. │ 334 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 336. │ 335 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 337. │ 336 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 338. │ 337 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 339. │ 338 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 340. │ 339 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 341. │ 340 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 342. │ 341 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 343. │ 342 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 344. │ 343 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 345. │ 344 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 346. │ 345 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 347. │ 346 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 348. │ 347 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 349. │ 348 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 350. │ 349 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 351. │ 350 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 352. │ 351 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 353. │ 352 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 354. │ 353 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 355. │ 354 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 356. │ 355 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 357. │ 356 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 358. │ 357 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 359. │ 358 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 360. │ 359 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 361. │ 360 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 362. │ 361 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 363. │ 362 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 364. │ 363 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 365. │ 364 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 366. │ 365 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 367. │ 366 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 368. │ 367 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 369. │ 368 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 370. │ 369 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 371. │ 370 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 372. │ 371 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 373. │ 372 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 374. │ 373 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 375. │ 374 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 376. │ 375 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 377. │ 376 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 378. │ 377 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 379. │ 378 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 380. │ 379 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 381. │ 380 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 382. │ 381 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 383. │ 382 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 384. │ 383 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 385. │ 384 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 386. │ 385 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 387. │ 386 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 388. │ 387 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 389. │ 388 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 390. │ 389 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 391. │ 390 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 392. │ 391 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 393. │ 392 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 394. │ 393 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 395. │ 394 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 396. │ 395 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 397. │ 396 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 398. │ 397 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 399. │ 398 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 400. │ 399 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 401. │ 400 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 402. │ 401 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 403. │ 402 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 404. │ 403 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 405. │ 404 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 406. │ 405 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 407. │ 406 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 408. │ 407 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 409. │ 408 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 410. │ 409 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 411. │ 410 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 412. │ 411 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 413. │ 412 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 414. │ 413 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 415. │ 414 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 416. │ 415 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 417. │ 416 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 418. │ 417 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 419. │ 418 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 420. │ 419 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 421. │ 420 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 422. │ 421 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 423. │ 422 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 424. │ 423 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 425. │ 424 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 426. │ 425 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 427. │ 426 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 428. │ 427 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 429. │ 428 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 430. │ 429 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 431. │ 430 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 432. │ 431 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 433. │ 432 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 434. │ 433 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 435. │ 434 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 436. │ 435 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 437. │ 436 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 438. │ 437 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 439. │ 438 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 440. │ 439 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 441. │ 440 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 442. │ 441 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 443. │ 442 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 444. │ 443 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 445. │ 444 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 446. │ 445 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 447. │ 446 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 448. │ 447 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 449. │ 448 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 450. │ 449 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 451. │ 450 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 452. │ 451 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 453. │ 452 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 454. │ 453 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 455. │ 454 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 456. │ 455 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 457. │ 456 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 458. │ 457 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 459. │ 458 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 460. │ 459 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 461. │ 460 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 462. │ 461 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 463. │ 462 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 464. │ 463 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 465. │ 464 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 466. │ 465 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 467. │ 466 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 468. │ 467 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 469. │ 468 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 470. │ 469 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 471. │ 470 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 472. │ 471 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 473. │ 472 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 474. │ 473 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 475. │ 474 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 476. │ 475 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 477. │ 476 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 478. │ 477 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 479. │ 478 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 480. │ 479 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 481. │ 480 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 482. │ 481 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 483. │ 482 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 484. │ 483 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 485. │ 484 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 486. │ 485 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 487. │ 486 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 488. │ 487 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 489. │ 488 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 490. │ 489 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 491. │ 490 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 492. │ 491 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 493. │ 492 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 494. │ 493 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 495. │ 494 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 496. │ 495 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 497. │ 496 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 498. │ 497 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 499. │ 498 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 500. │ 499 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 501. │ 500 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 502. │ 501 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 503. │ 502 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 504. │ 503 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 505. │ 504 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 506. │ 505 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 507. │ 506 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 508. │ 507 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 509. │ 508 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 510. │ 509 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 511. │ 510 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 512. │ 511 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 513. │ 512 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 514. │ 513 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 515. │ 514 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 516. │ 515 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 517. │ 516 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 518. │ 517 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 519. │ 518 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 520. │ 519 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 521. │ 520 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 522. │ 521 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 523. │ 522 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 524. │ 523 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 525. │ 524 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 526. │ 525 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 527. │ 526 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 528. │ 527 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 529. │ 528 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 530. │ 529 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 531. │ 530 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 532. │ 531 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 533. │ 532 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 534. │ 533 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 535. │ 534 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 536. │ 535 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 537. │ 536 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 538. │ 537 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 539. │ 538 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 540. │ 539 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 541. │ 540 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 542. │ 541 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 543. │ 542 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 544. │ 543 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 545. │ 544 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 546. │ 545 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 547. │ 546 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 548. │ 547 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 549. │ 548 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 550. │ 549 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 551. │ 550 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 552. │ 551 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 553. │ 552 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 554. │ 553 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 555. │ 554 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 556. │ 555 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 557. │ 556 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 558. │ 557 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 559. │ 558 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 560. │ 559 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 561. │ 560 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 562. │ 561 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 563. │ 562 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 564. │ 563 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 565. │ 564 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 566. │ 565 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 567. │ 566 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 568. │ 567 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 569. │ 568 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 570. │ 569 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 571. │ 570 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 572. │ 571 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 573. │ 572 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 574. │ 573 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 575. │ 574 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 576. │ 575 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 577. │ 576 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 578. │ 577 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 579. │ 578 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 580. │ 579 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 581. │ 580 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 582. │ 581 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 583. │ 582 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 584. │ 583 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 585. │ 584 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 586. │ 585 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 587. │ 586 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 588. │ 587 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 589. │ 588 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 590. │ 589 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 591. │ 590 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 592. │ 591 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 593. │ 592 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 594. │ 593 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 595. │ 594 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 596. │ 595 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 597. │ 596 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 598. │ 597 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 599. │ 598 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 600. │ 599 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 601. │ 600 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 602. │ 601 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 603. │ 602 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 604. │ 603 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 605. │ 604 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 606. │ 605 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 607. │ 606 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 608. │ 607 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 609. │ 608 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 610. │ 609 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 611. │ 610 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 612. │ 611 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 613. │ 612 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 614. │ 613 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 615. │ 614 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 616. │ 615 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 617. │ 616 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 618. │ 617 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 619. │ 618 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 620. │ 619 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 621. │ 620 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 622. │ 621 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 623. │ 622 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 624. │ 623 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 625. │ 624 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 626. │ 625 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 627. │ 626 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 628. │ 627 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 629. │ 628 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 630. │ 629 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 631. │ 630 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 632. │ 631 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 633. │ 632 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 634. │ 633 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 635. │ 634 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 636. │ 635 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 637. │ 636 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 638. │ 637 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 639. │ 638 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 640. │ 639 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 641. │ 640 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 642. │ 641 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 643. │ 642 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 644. │ 643 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 645. │ 644 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 646. │ 645 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 647. │ 646 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 648. │ 647 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 649. │ 648 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 650. │ 649 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 651. │ 650 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 652. │ 651 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 653. │ 652 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 654. │ 653 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 655. │ 654 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 656. │ 655 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 657. │ 656 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 658. │ 657 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 659. │ 658 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 660. │ 659 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 661. │ 660 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 662. │ 661 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 663. │ 662 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 664. │ 663 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 665. │ 664 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 666. │ 665 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 667. │ 666 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 668. │ 667 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 669. │ 668 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 670. │ 669 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 671. │ 670 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 672. │ 671 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 673. │ 672 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 674. │ 673 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 675. │ 674 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 676. │ 675 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 677. │ 676 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 678. │ 677 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 679. │ 678 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 680. │ 679 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 681. │ 680 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 682. │ 681 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 683. │ 682 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 684. │ 683 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 685. │ 684 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 686. │ 685 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 687. │ 686 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 688. │ 687 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 689. │ 688 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 690. │ 689 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 691. │ 690 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 692. │ 691 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 693. │ 692 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 694. │ 693 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 695. │ 694 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 696. │ 695 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 697. │ 696 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 698. │ 697 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 699. │ 698 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 700. │ 699 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 701. │ 700 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 702. │ 701 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 703. │ 702 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 704. │ 703 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 705. │ 704 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 706. │ 705 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 707. │ 706 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 708. │ 707 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 709. │ 708 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 710. │ 709 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 711. │ 710 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 712. │ 711 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 713. │ 712 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 714. │ 713 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 715. │ 714 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 716. │ 715 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 717. │ 716 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 718. │ 717 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 719. │ 718 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 720. │ 719 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 721. │ 720 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 722. │ 721 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 723. │ 722 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 724. │ 723 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 725. │ 724 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 726. │ 725 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 727. │ 726 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 728. │ 727 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 729. │ 728 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 730. │ 729 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 731. │ 730 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 732. │ 731 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 733. │ 732 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 734. │ 733 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 735. │ 734 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 736. │ 735 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 737. │ 736 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 738. │ 737 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 739. │ 738 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 740. │ 739 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 741. │ 740 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 742. │ 741 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 743. │ 742 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 744. │ 743 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 745. │ 744 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 746. │ 745 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 747. │ 746 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 748. │ 747 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 749. │ 748 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 750. │ 749 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 751. │ 750 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 752. │ 751 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 753. │ 752 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 754. │ 753 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 755. │ 754 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 756. │ 755 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 757. │ 756 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 758. │ 757 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 759. │ 758 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 760. │ 759 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 761. │ 760 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 762. │ 761 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 763. │ 762 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 764. │ 763 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 765. │ 764 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 766. │ 765 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 767. │ 766 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 768. │ 767 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 769. │ 768 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 770. │ 769 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 771. │ 770 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 772. │ 771 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 773. │ 772 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 774. │ 773 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 775. │ 774 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 776. │ 775 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 777. │ 776 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 778. │ 777 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 779. │ 778 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 780. │ 779 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 781. │ 780 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 782. │ 781 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 783. │ 782 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 784. │ 783 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 785. │ 784 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 786. │ 785 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 787. │ 786 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 788. │ 787 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 789. │ 788 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 790. │ 789 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 791. │ 790 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 792. │ 791 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 793. │ 792 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 794. │ 793 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 795. │ 794 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 796. │ 795 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 797. │ 796 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 798. │ 797 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 799. │ 798 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 800. │ 799 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 801. │ 800 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 802. │ 801 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 803. │ 802 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 804. │ 803 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 805. │ 804 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 806. │ 805 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 807. │ 806 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 808. │ 807 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 809. │ 808 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 810. │ 809 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 811. │ 810 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 812. │ 811 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 813. │ 812 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 814. │ 813 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 815. │ 814 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 816. │ 815 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 817. │ 816 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 818. │ 817 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 819. │ 818 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 820. │ 819 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 821. │ 820 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 822. │ 821 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 823. │ 822 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 824. │ 823 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 825. │ 824 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 826. │ 825 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 827. │ 826 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 828. │ 827 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 829. │ 828 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 830. │ 829 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 831. │ 830 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 832. │ 831 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 833. │ 832 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 834. │ 833 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 835. │ 834 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 836. │ 835 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 837. │ 836 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 838. │ 837 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 839. │ 838 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 840. │ 839 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 841. │ 840 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 842. │ 841 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 843. │ 842 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 844. │ 843 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 845. │ 844 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 846. │ 845 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 847. │ 846 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 848. │ 847 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 849. │ 848 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 850. │ 849 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 851. │ 850 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 852. │ 851 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 853. │ 852 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 854. │ 853 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 855. │ 854 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 856. │ 855 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 857. │ 856 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 858. │ 857 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 859. │ 858 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 860. │ 859 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 861. │ 860 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 862. │ 861 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 863. │ 862 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 864. │ 863 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 865. │ 864 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 866. │ 865 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 867. │ 866 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 868. │ 867 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 869. │ 868 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 870. │ 869 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 871. │ 870 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 872. │ 871 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 873. │ 872 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 874. │ 873 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 875. │ 874 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 876. │ 875 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 877. │ 876 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 878. │ 877 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 879. │ 878 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 880. │ 879 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 881. │ 880 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 882. │ 881 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 883. │ 882 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 884. │ 883 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 885. │ 884 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 886. │ 885 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 887. │ 886 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 888. │ 887 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 889. │ 888 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 890. │ 889 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 891. │ 890 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 892. │ 891 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 893. │ 892 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 894. │ 893 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 895. │ 894 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 896. │ 895 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 897. │ 896 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 898. │ 897 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 899. │ 898 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 900. │ 899 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 901. │ 900 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 902. │ 901 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 903. │ 902 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 904. │ 903 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 905. │ 904 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 906. │ 905 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 907. │ 906 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 908. │ 907 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 909. │ 908 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 910. │ 909 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 911. │ 910 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 912. │ 911 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 913. │ 912 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 914. │ 913 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 915. │ 914 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 916. │ 915 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 917. │ 916 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 918. │ 917 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 919. │ 918 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 920. │ 919 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 921. │ 920 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 922. │ 921 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 923. │ 922 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 924. │ 923 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 925. │ 924 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 926. │ 925 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 927. │ 926 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 928. │ 927 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 929. │ 928 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 930. │ 929 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 931. │ 930 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 932. │ 931 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 933. │ 932 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 934. │ 933 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 935. │ 934 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 936. │ 935 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 937. │ 936 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 938. │ 937 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 939. │ 938 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 940. │ 939 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 941. │ 940 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 942. │ 941 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 943. │ 942 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 944. │ 943 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 945. │ 944 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 946. │ 945 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 947. │ 946 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 948. │ 947 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 949. │ 948 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 950. │ 949 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 951. │ 950 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 952. │ 951 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 953. │ 952 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 954. │ 953 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 955. │ 954 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 956. │ 955 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 957. │ 956 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 958. │ 957 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 959. │ 958 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 960. │ 959 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 961. │ 960 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 962. │ 961 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 963. │ 962 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 964. │ 963 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 965. │ 964 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 966. │ 965 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 967. │ 966 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 968. │ 967 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 969. │ 968 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 970. │ 969 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 971. │ 970 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 972. │ 971 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 973. │ 972 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 974. │ 973 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 975. │ 974 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 976. │ 975 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 977. │ 976 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 978. │ 977 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 979. │ 978 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 980. │ 979 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 981. │ 980 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 982. │ 981 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 983. │ 982 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 984. │ 983 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 985. │ 984 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 986. │ 985 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 987. │ 986 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 988. │ 987 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 989. │ 988 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 990. │ 989 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 991. │ 990 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 992. │ 991 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 993. │ 992 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 994. │ 993 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 995. │ 994 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 996. │ 995 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 997. │ 996 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 998. │ 997 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 999. │ 998 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +1000. │ 999 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 100. │ 99 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 101. │ 100 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 102. │ 101 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 103. │ 102 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 104. │ 103 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 105. │ 104 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 106. │ 105 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 107. │ 106 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 108. │ 107 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 109. │ 108 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 110. │ 109 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 111. │ 110 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 112. │ 111 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 113. │ 112 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 114. │ 113 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 115. │ 114 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 116. │ 115 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 117. │ 116 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 118. │ 117 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 119. │ 118 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 120. │ 119 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 121. │ 120 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 122. │ 121 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 123. │ 122 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 124. │ 123 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 125. │ 124 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 126. │ 125 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 127. │ 126 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 128. │ 127 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 129. │ 128 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 130. │ 129 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 131. │ 130 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 132. │ 131 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 133. │ 132 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 134. │ 133 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 135. │ 134 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 136. │ 135 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 137. │ 136 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 138. │ 137 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 139. │ 138 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 140. │ 139 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 141. │ 140 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 142. │ 141 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 143. │ 142 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 144. │ 143 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 145. │ 144 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 146. │ 145 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 147. │ 146 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 148. │ 147 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 149. │ 148 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 150. │ 149 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 151. │ 150 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 152. │ 151 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 153. │ 152 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 154. │ 153 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 155. │ 154 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 156. │ 155 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 157. │ 156 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 158. │ 157 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 159. │ 158 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 160. │ 159 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 161. │ 160 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 162. │ 161 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 163. │ 162 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 164. │ 163 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 165. │ 164 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 166. │ 165 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 167. │ 166 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 168. │ 167 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 169. │ 168 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 170. │ 169 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 171. │ 170 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 172. │ 171 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 173. │ 172 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 174. │ 173 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 175. │ 174 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 176. │ 175 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 177. │ 176 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 178. │ 177 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 179. │ 178 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 180. │ 179 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 181. │ 180 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 182. │ 181 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 183. │ 182 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 184. │ 183 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 185. │ 184 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 186. │ 185 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 187. │ 186 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 188. │ 187 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 189. │ 188 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 190. │ 189 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 191. │ 190 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 192. │ 191 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 193. │ 192 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 194. │ 193 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 195. │ 194 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 196. │ 195 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 197. │ 196 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 198. │ 197 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 199. │ 198 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 200. │ 199 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 201. │ 200 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 202. │ 201 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 203. │ 202 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 204. │ 203 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 205. │ 204 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 206. │ 205 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 207. │ 206 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 208. │ 207 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 209. │ 208 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 210. │ 209 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 211. │ 210 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 212. │ 211 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 213. │ 212 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 214. │ 213 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 215. │ 214 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 216. │ 215 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 217. │ 216 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 218. │ 217 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 219. │ 218 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 220. │ 219 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 221. │ 220 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 222. │ 221 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 223. │ 222 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 224. │ 223 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 225. │ 224 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 226. │ 225 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 227. │ 226 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 228. │ 227 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 229. │ 228 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 230. │ 229 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 231. │ 230 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 232. │ 231 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 233. │ 232 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 234. │ 233 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 235. │ 234 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 236. │ 235 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 237. │ 236 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 238. │ 237 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 239. │ 238 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 240. │ 239 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 241. │ 240 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 242. │ 241 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 243. │ 242 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 244. │ 243 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 245. │ 244 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 246. │ 245 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 247. │ 246 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 248. │ 247 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 249. │ 248 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 250. │ 249 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 251. │ 250 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 252. │ 251 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 253. │ 252 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 254. │ 253 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 255. │ 254 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 256. │ 255 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 257. │ 256 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 258. │ 257 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 259. │ 258 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 260. │ 259 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 261. │ 260 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 262. │ 261 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 263. │ 262 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 264. │ 263 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 265. │ 264 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 266. │ 265 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 267. │ 266 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 268. │ 267 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 269. │ 268 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 270. │ 269 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 271. │ 270 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 272. │ 271 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 273. │ 272 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 274. │ 273 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 275. │ 274 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 276. │ 275 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 277. │ 276 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 278. │ 277 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 279. │ 278 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 280. │ 279 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 281. │ 280 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 282. │ 281 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 283. │ 282 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 284. │ 283 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 285. │ 284 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 286. │ 285 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 287. │ 286 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 288. │ 287 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 289. │ 288 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 290. │ 289 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 291. │ 290 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 292. │ 291 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 293. │ 292 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 294. │ 293 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 295. │ 294 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 296. │ 295 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 297. │ 296 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 298. │ 297 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 299. │ 298 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 300. │ 299 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 301. │ 300 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 302. │ 301 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 303. │ 302 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 304. │ 303 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 305. │ 304 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 306. │ 305 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 307. │ 306 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 308. │ 307 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 309. │ 308 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 310. │ 309 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 311. │ 310 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 312. │ 311 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 313. │ 312 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 314. │ 313 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 315. │ 314 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 316. │ 315 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 317. │ 316 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 318. │ 317 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 319. │ 318 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 320. │ 319 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 321. │ 320 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 322. │ 321 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 323. │ 322 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 324. │ 323 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 325. │ 324 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 326. │ 325 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 327. │ 326 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 328. │ 327 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 329. │ 328 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 330. │ 329 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 331. │ 330 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 332. │ 331 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 333. │ 332 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 334. │ 333 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 335. │ 334 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 336. │ 335 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 337. │ 336 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 338. │ 337 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 339. │ 338 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 340. │ 339 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 341. │ 340 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 342. │ 341 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 343. │ 342 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 344. │ 343 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 345. │ 344 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 346. │ 345 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 347. │ 346 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 348. │ 347 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 349. │ 348 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 350. │ 349 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 351. │ 350 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 352. │ 351 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 353. │ 352 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 354. │ 353 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 355. │ 354 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 356. │ 355 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 357. │ 356 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 358. │ 357 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 359. │ 358 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 360. │ 359 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 361. │ 360 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 362. │ 361 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 363. │ 362 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 364. │ 363 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 365. │ 364 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 366. │ 365 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 367. │ 366 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 368. │ 367 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 369. │ 368 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 370. │ 369 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 371. │ 370 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 372. │ 371 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 373. │ 372 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 374. │ 373 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 375. │ 374 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 376. │ 375 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 377. │ 376 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 378. │ 377 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 379. │ 378 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 380. │ 379 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 381. │ 380 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 382. │ 381 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 383. │ 382 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 384. │ 383 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 385. │ 384 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 386. │ 385 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 387. │ 386 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 388. │ 387 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 389. │ 388 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 390. │ 389 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 391. │ 390 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 392. │ 391 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 393. │ 392 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 394. │ 393 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 395. │ 394 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 396. │ 395 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 397. │ 396 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 398. │ 397 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 399. │ 398 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 400. │ 399 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 401. │ 400 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 402. │ 401 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 403. │ 402 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 404. │ 403 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 405. │ 404 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 406. │ 405 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 407. │ 406 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 408. │ 407 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 409. │ 408 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 410. │ 409 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 411. │ 410 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 412. │ 411 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 413. │ 412 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 414. │ 413 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 415. │ 414 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 416. │ 415 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 417. │ 416 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 418. │ 417 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 419. │ 418 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 420. │ 419 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 421. │ 420 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 422. │ 421 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 423. │ 422 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 424. │ 423 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 425. │ 424 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 426. │ 425 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 427. │ 426 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 428. │ 427 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 429. │ 428 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 430. │ 429 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 431. │ 430 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 432. │ 431 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 433. │ 432 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 434. │ 433 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 435. │ 434 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 436. │ 435 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 437. │ 436 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 438. │ 437 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 439. │ 438 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 440. │ 439 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 441. │ 440 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 442. │ 441 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 443. │ 442 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 444. │ 443 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 445. │ 444 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 446. │ 445 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 447. │ 446 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 448. │ 447 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 449. │ 448 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 450. │ 449 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 451. │ 450 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 452. │ 451 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 453. │ 452 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 454. │ 453 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 455. │ 454 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 456. │ 455 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 457. │ 456 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 458. │ 457 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 459. │ 458 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 460. │ 459 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 461. │ 460 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 462. │ 461 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 463. │ 462 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 464. │ 463 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 465. │ 464 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 466. │ 465 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 467. │ 466 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 468. │ 467 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 469. │ 468 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 470. │ 469 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 471. │ 470 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 472. │ 471 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 473. │ 472 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 474. │ 473 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 475. │ 474 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 476. │ 475 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 477. │ 476 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 478. │ 477 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 479. │ 478 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 480. │ 479 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 481. │ 480 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 482. │ 481 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 483. │ 482 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 484. │ 483 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 485. │ 484 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 486. │ 485 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 487. │ 486 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 488. │ 487 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 489. │ 488 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 490. │ 489 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 491. │ 490 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 492. │ 491 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 493. │ 492 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 494. │ 493 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 495. │ 494 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 496. │ 495 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 497. │ 496 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 498. │ 497 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 499. │ 498 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 500. │ 499 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 501. │ 500 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 502. │ 501 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 503. │ 502 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 504. │ 503 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 505. │ 504 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 506. │ 505 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 507. │ 506 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 508. │ 507 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 509. │ 508 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 510. │ 509 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 511. │ 510 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 512. │ 511 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 513. │ 512 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 514. │ 513 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 515. │ 514 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 516. │ 515 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 517. │ 516 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 518. │ 517 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 519. │ 518 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 520. │ 519 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 521. │ 520 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 522. │ 521 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 523. │ 522 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 524. │ 523 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 525. │ 524 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 526. │ 525 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 527. │ 526 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 528. │ 527 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 529. │ 528 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 530. │ 529 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 531. │ 530 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 532. │ 531 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 533. │ 532 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 534. │ 533 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 535. │ 534 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 536. │ 535 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 537. │ 536 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 538. │ 537 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 539. │ 538 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 540. │ 539 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 541. │ 540 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 542. │ 541 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 543. │ 542 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 544. │ 543 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 545. │ 544 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 546. │ 545 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 547. │ 546 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 548. │ 547 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 549. │ 548 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 550. │ 549 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 551. │ 550 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 552. │ 551 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 553. │ 552 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 554. │ 553 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 555. │ 554 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 556. │ 555 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 557. │ 556 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 558. │ 557 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 559. │ 558 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 560. │ 559 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 561. │ 560 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 562. │ 561 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 563. │ 562 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 564. │ 563 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 565. │ 564 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 566. │ 565 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 567. │ 566 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 568. │ 567 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 569. │ 568 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 570. │ 569 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 571. │ 570 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 572. │ 571 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 573. │ 572 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 574. │ 573 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 575. │ 574 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 576. │ 575 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 577. │ 576 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 578. │ 577 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 579. │ 578 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 580. │ 579 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 581. │ 580 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 582. │ 581 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 583. │ 582 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 584. │ 583 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 585. │ 584 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 586. │ 585 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 587. │ 586 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 588. │ 587 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 589. │ 588 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 590. │ 589 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 591. │ 590 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 592. │ 591 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 593. │ 592 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 594. │ 593 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 595. │ 594 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 596. │ 595 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 597. │ 596 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 598. │ 597 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 599. │ 598 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 600. │ 599 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 601. │ 600 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 602. │ 601 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 603. │ 602 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 604. │ 603 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 605. │ 604 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 606. │ 605 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 607. │ 606 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 608. │ 607 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 609. │ 608 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 610. │ 609 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 611. │ 610 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 612. │ 611 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 613. │ 612 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 614. │ 613 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 615. │ 614 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 616. │ 615 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 617. │ 616 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 618. │ 617 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 619. │ 618 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 620. │ 619 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 621. │ 620 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 622. │ 621 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 623. │ 622 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 624. │ 623 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 625. │ 624 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 626. │ 625 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 627. │ 626 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 628. │ 627 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 629. │ 628 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 630. │ 629 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 631. │ 630 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 632. │ 631 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 633. │ 632 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 634. │ 633 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 635. │ 634 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 636. │ 635 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 637. │ 636 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 638. │ 637 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 639. │ 638 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 640. │ 639 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 641. │ 640 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 642. │ 641 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 643. │ 642 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 644. │ 643 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 645. │ 644 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 646. │ 645 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 647. │ 646 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 648. │ 647 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 649. │ 648 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 650. │ 649 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 651. │ 650 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 652. │ 651 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 653. │ 652 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 654. │ 653 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 655. │ 654 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 656. │ 655 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 657. │ 656 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 658. │ 657 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 659. │ 658 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 660. │ 659 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 661. │ 660 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 662. │ 661 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 663. │ 662 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 664. │ 663 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 665. │ 664 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 666. │ 665 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 667. │ 666 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 668. │ 667 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 669. │ 668 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 670. │ 669 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 671. │ 670 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 672. │ 671 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 673. │ 672 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 674. │ 673 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 675. │ 674 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 676. │ 675 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 677. │ 676 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 678. │ 677 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 679. │ 678 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 680. │ 679 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 681. │ 680 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 682. │ 681 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 683. │ 682 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 684. │ 683 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 685. │ 684 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 686. │ 685 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 687. │ 686 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 688. │ 687 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 689. │ 688 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 690. │ 689 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 691. │ 690 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 692. │ 691 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 693. │ 692 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 694. │ 693 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 695. │ 694 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 696. │ 695 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 697. │ 696 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 698. │ 697 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 699. │ 698 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 700. │ 699 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 701. │ 700 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 702. │ 701 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 703. │ 702 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 704. │ 703 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 705. │ 704 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 706. │ 705 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 707. │ 706 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 708. │ 707 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 709. │ 708 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 710. │ 709 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 711. │ 710 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 712. │ 711 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 713. │ 712 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 714. │ 713 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 715. │ 714 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 716. │ 715 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 717. │ 716 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 718. │ 717 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 719. │ 718 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 720. │ 719 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 721. │ 720 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 722. │ 721 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 723. │ 722 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 724. │ 723 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 725. │ 724 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 726. │ 725 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 727. │ 726 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 728. │ 727 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 729. │ 728 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 730. │ 729 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 731. │ 730 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 732. │ 731 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 733. │ 732 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 734. │ 733 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 735. │ 734 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 736. │ 735 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 737. │ 736 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 738. │ 737 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 739. │ 738 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 740. │ 739 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 741. │ 740 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 742. │ 741 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 743. │ 742 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 744. │ 743 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 745. │ 744 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 746. │ 745 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 747. │ 746 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 748. │ 747 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 749. │ 748 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 750. │ 749 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 751. │ 750 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 752. │ 751 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 753. │ 752 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 754. │ 753 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 755. │ 754 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 756. │ 755 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 757. │ 756 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 758. │ 757 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 759. │ 758 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 760. │ 759 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 761. │ 760 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 762. │ 761 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 763. │ 762 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 764. │ 763 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 765. │ 764 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 766. │ 765 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 767. │ 766 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 768. │ 767 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 769. │ 768 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 770. │ 769 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 771. │ 770 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 772. │ 771 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 773. │ 772 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 774. │ 773 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 775. │ 774 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 776. │ 775 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 777. │ 776 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 778. │ 777 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 779. │ 778 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 780. │ 779 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 781. │ 780 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 782. │ 781 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 783. │ 782 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 784. │ 783 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 785. │ 784 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 786. │ 785 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 787. │ 786 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 788. │ 787 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 789. │ 788 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 790. │ 789 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 791. │ 790 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 792. │ 791 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 793. │ 792 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 794. │ 793 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 795. │ 794 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 796. │ 795 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 797. │ 796 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 798. │ 797 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 799. │ 798 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 800. │ 799 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 801. │ 800 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 802. │ 801 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 803. │ 802 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 804. │ 803 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 805. │ 804 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 806. │ 805 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 807. │ 806 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 808. │ 807 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 809. │ 808 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 810. │ 809 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 811. │ 810 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 812. │ 811 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 813. │ 812 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 814. │ 813 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 815. │ 814 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 816. │ 815 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 817. │ 816 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 818. │ 817 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 819. │ 818 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 820. │ 819 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 821. │ 820 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 822. │ 821 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 823. │ 822 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 824. │ 823 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 825. │ 824 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 826. │ 825 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 827. │ 826 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 828. │ 827 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 829. │ 828 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 830. │ 829 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 831. │ 830 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 832. │ 831 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 833. │ 832 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 834. │ 833 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 835. │ 834 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 836. │ 835 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 837. │ 836 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 838. │ 837 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 839. │ 838 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 840. │ 839 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 841. │ 840 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 842. │ 841 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 843. │ 842 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 844. │ 843 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 845. │ 844 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 846. │ 845 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 847. │ 846 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 848. │ 847 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 849. │ 848 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 850. │ 849 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 851. │ 850 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 852. │ 851 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 853. │ 852 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 854. │ 853 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 855. │ 854 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 856. │ 855 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 857. │ 856 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 858. │ 857 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 859. │ 858 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 860. │ 859 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 861. │ 860 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 862. │ 861 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 863. │ 862 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 864. │ 863 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 865. │ 864 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 866. │ 865 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 867. │ 866 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 868. │ 867 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 869. │ 868 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 870. │ 869 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 871. │ 870 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 872. │ 871 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 873. │ 872 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 874. │ 873 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 875. │ 874 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 876. │ 875 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 877. │ 876 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 878. │ 877 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 879. │ 878 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 880. │ 879 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 881. │ 880 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 882. │ 881 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 883. │ 882 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 884. │ 883 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 885. │ 884 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 886. │ 885 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 887. │ 886 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 888. │ 887 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 889. │ 888 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 890. │ 889 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 891. │ 890 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 892. │ 891 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 893. │ 892 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 894. │ 893 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 895. │ 894 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 896. │ 895 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 897. │ 896 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 898. │ 897 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 899. │ 898 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 900. │ 899 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 901. │ 900 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 902. │ 901 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 903. │ 902 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 904. │ 903 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 905. │ 904 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 906. │ 905 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 907. │ 906 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 908. │ 907 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 909. │ 908 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 910. │ 909 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 911. │ 910 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 912. │ 911 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 913. │ 912 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 914. │ 913 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 915. │ 914 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 916. │ 915 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 917. │ 916 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 918. │ 917 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 919. │ 918 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 920. │ 919 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 921. │ 920 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 922. │ 921 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 923. │ 922 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 924. │ 923 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 925. │ 924 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 926. │ 925 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 927. │ 926 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 928. │ 927 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 929. │ 928 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 930. │ 929 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 931. │ 930 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 932. │ 931 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 933. │ 932 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 934. │ 933 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 935. │ 934 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 936. │ 935 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 937. │ 936 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 938. │ 937 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 939. │ 938 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 940. │ 939 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 941. │ 940 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 942. │ 941 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 943. │ 942 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 944. │ 943 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 945. │ 944 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 946. │ 945 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 947. │ 946 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 948. │ 947 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 949. │ 948 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 950. │ 949 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 951. │ 950 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 952. │ 951 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 953. │ 952 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 954. │ 953 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 955. │ 954 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 956. │ 955 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 957. │ 956 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 958. │ 957 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 959. │ 958 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 960. │ 959 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 961. │ 960 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 962. │ 961 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 963. │ 962 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 964. │ 963 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 965. │ 964 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 966. │ 965 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 967. │ 966 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 968. │ 967 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 969. │ 968 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 970. │ 969 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 971. │ 970 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 972. │ 971 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 973. │ 972 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 974. │ 973 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 975. │ 974 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 976. │ 975 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 977. │ 976 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 978. │ 977 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 979. │ 978 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 980. │ 979 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 981. │ 980 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 982. │ 981 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 983. │ 982 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 984. │ 983 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 985. │ 984 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 986. │ 985 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 987. │ 986 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 988. │ 987 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 989. │ 988 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 990. │ 989 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 991. │ 990 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 992. │ 991 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 993. │ 992 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 994. │ 993 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 995. │ 994 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 996. │ 995 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 997. │ 996 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 998. │ 997 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 999. │ 998 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +1000. │ 999 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ + 100. │ 99 │ UInt64 │ 1 │ + 101. │ 100 │ UInt64 │ 0 │ + 102. │ 101 │ UInt64 │ 1 │ + 103. │ 102 │ UInt64 │ 0 │ + 104. │ 103 │ UInt64 │ 1 │ + 105. │ 104 │ UInt64 │ 0 │ + 106. │ 105 │ UInt64 │ 1 │ + 107. │ 106 │ UInt64 │ 0 │ + 108. │ 107 │ UInt64 │ 1 │ + 109. │ 108 │ UInt64 │ 0 │ + 110. │ 109 │ UInt64 │ 1 │ + 111. │ 110 │ UInt64 │ 0 │ + 112. │ 111 │ UInt64 │ 1 │ + 113. │ 112 │ UInt64 │ 0 │ + 114. │ 113 │ UInt64 │ 1 │ + 115. │ 114 │ UInt64 │ 0 │ + 116. │ 115 │ UInt64 │ 1 │ + 117. │ 116 │ UInt64 │ 0 │ + 118. │ 117 │ UInt64 │ 1 │ + 119. │ 118 │ UInt64 │ 0 │ + 120. │ 119 │ UInt64 │ 1 │ + 121. │ 120 │ UInt64 │ 0 │ + 122. │ 121 │ UInt64 │ 1 │ + 123. │ 122 │ UInt64 │ 0 │ + 124. │ 123 │ UInt64 │ 1 │ + 125. │ 124 │ UInt64 │ 0 │ + 126. │ 125 │ UInt64 │ 1 │ + 127. │ 126 │ UInt64 │ 0 │ + 128. │ 127 │ UInt64 │ 1 │ + 129. │ 128 │ UInt64 │ 0 │ + 130. │ 129 │ UInt64 │ 1 │ + 131. │ 130 │ UInt64 │ 0 │ + 132. │ 131 │ UInt64 │ 1 │ + 133. │ 132 │ UInt64 │ 0 │ + 134. │ 133 │ UInt64 │ 1 │ + 135. │ 134 │ UInt64 │ 0 │ + 136. │ 135 │ UInt64 │ 1 │ + 137. │ 136 │ UInt64 │ 0 │ + 138. │ 137 │ UInt64 │ 1 │ + 139. │ 138 │ UInt64 │ 0 │ + 140. │ 139 │ UInt64 │ 1 │ + 141. │ 140 │ UInt64 │ 0 │ + 142. │ 141 │ UInt64 │ 1 │ + 143. │ 142 │ UInt64 │ 0 │ + 144. │ 143 │ UInt64 │ 1 │ + 145. │ 144 │ UInt64 │ 0 │ + 146. │ 145 │ UInt64 │ 1 │ + 147. │ 146 │ UInt64 │ 0 │ + 148. │ 147 │ UInt64 │ 1 │ + 149. │ 148 │ UInt64 │ 0 │ + 150. │ 149 │ UInt64 │ 1 │ + 151. │ 150 │ UInt64 │ 0 │ + 152. │ 151 │ UInt64 │ 1 │ + 153. │ 152 │ UInt64 │ 0 │ + 154. │ 153 │ UInt64 │ 1 │ + 155. │ 154 │ UInt64 │ 0 │ + 156. │ 155 │ UInt64 │ 1 │ + 157. │ 156 │ UInt64 │ 0 │ + 158. │ 157 │ UInt64 │ 1 │ + 159. │ 158 │ UInt64 │ 0 │ + 160. │ 159 │ UInt64 │ 1 │ + 161. │ 160 │ UInt64 │ 0 │ + 162. │ 161 │ UInt64 │ 1 │ + 163. │ 162 │ UInt64 │ 0 │ + 164. │ 163 │ UInt64 │ 1 │ + 165. │ 164 │ UInt64 │ 0 │ + 166. │ 165 │ UInt64 │ 1 │ + 167. │ 166 │ UInt64 │ 0 │ + 168. │ 167 │ UInt64 │ 1 │ + 169. │ 168 │ UInt64 │ 0 │ + 170. │ 169 │ UInt64 │ 1 │ + 171. │ 170 │ UInt64 │ 0 │ + 172. │ 171 │ UInt64 │ 1 │ + 173. │ 172 │ UInt64 │ 0 │ + 174. │ 173 │ UInt64 │ 1 │ + 175. │ 174 │ UInt64 │ 0 │ + 176. │ 175 │ UInt64 │ 1 │ + 177. │ 176 │ UInt64 │ 0 │ + 178. │ 177 │ UInt64 │ 1 │ + 179. │ 178 │ UInt64 │ 0 │ + 180. │ 179 │ UInt64 │ 1 │ + 181. │ 180 │ UInt64 │ 0 │ + 182. │ 181 │ UInt64 │ 1 │ + 183. │ 182 │ UInt64 │ 0 │ + 184. │ 183 │ UInt64 │ 1 │ + 185. │ 184 │ UInt64 │ 0 │ + 186. │ 185 │ UInt64 │ 1 │ + 187. │ 186 │ UInt64 │ 0 │ + 188. │ 187 │ UInt64 │ 1 │ + 189. │ 188 │ UInt64 │ 0 │ + 190. │ 189 │ UInt64 │ 1 │ + 191. │ 190 │ UInt64 │ 0 │ + 192. │ 191 │ UInt64 │ 1 │ + 193. │ 192 │ UInt64 │ 0 │ + 194. │ 193 │ UInt64 │ 1 │ + 195. │ 194 │ UInt64 │ 0 │ + 196. │ 195 │ UInt64 │ 1 │ + 197. │ 196 │ UInt64 │ 0 │ + 198. │ 197 │ UInt64 │ 1 │ + 199. │ 198 │ UInt64 │ 0 │ + 200. │ 199 │ UInt64 │ 1 │ + 201. │ 200 │ UInt64 │ 0 │ + 202. │ 201 │ UInt64 │ 1 │ + 203. │ 202 │ UInt64 │ 0 │ + 204. │ 203 │ UInt64 │ 1 │ + 205. │ 204 │ UInt64 │ 0 │ + 206. │ 205 │ UInt64 │ 1 │ + 207. │ 206 │ UInt64 │ 0 │ + 208. │ 207 │ UInt64 │ 1 │ + 209. │ 208 │ UInt64 │ 0 │ + 210. │ 209 │ UInt64 │ 1 │ + 211. │ 210 │ UInt64 │ 0 │ + 212. │ 211 │ UInt64 │ 1 │ + 213. │ 212 │ UInt64 │ 0 │ + 214. │ 213 │ UInt64 │ 1 │ + 215. │ 214 │ UInt64 │ 0 │ + 216. │ 215 │ UInt64 │ 1 │ + 217. │ 216 │ UInt64 │ 0 │ + 218. │ 217 │ UInt64 │ 1 │ + 219. │ 218 │ UInt64 │ 0 │ + 220. │ 219 │ UInt64 │ 1 │ + 221. │ 220 │ UInt64 │ 0 │ + 222. │ 221 │ UInt64 │ 1 │ + 223. │ 222 │ UInt64 │ 0 │ + 224. │ 223 │ UInt64 │ 1 │ + 225. │ 224 │ UInt64 │ 0 │ + 226. │ 225 │ UInt64 │ 1 │ + 227. │ 226 │ UInt64 │ 0 │ + 228. │ 227 │ UInt64 │ 1 │ + 229. │ 228 │ UInt64 │ 0 │ + 230. │ 229 │ UInt64 │ 1 │ + 231. │ 230 │ UInt64 │ 0 │ + 232. │ 231 │ UInt64 │ 1 │ + 233. │ 232 │ UInt64 │ 0 │ + 234. │ 233 │ UInt64 │ 1 │ + 235. │ 234 │ UInt64 │ 0 │ + 236. │ 235 │ UInt64 │ 1 │ + 237. │ 236 │ UInt64 │ 0 │ + 238. │ 237 │ UInt64 │ 1 │ + 239. │ 238 │ UInt64 │ 0 │ + 240. │ 239 │ UInt64 │ 1 │ + 241. │ 240 │ UInt64 │ 0 │ + 242. │ 241 │ UInt64 │ 1 │ + 243. │ 242 │ UInt64 │ 0 │ + 244. │ 243 │ UInt64 │ 1 │ + 245. │ 244 │ UInt64 │ 0 │ + 246. │ 245 │ UInt64 │ 1 │ + 247. │ 246 │ UInt64 │ 0 │ + 248. │ 247 │ UInt64 │ 1 │ + 249. │ 248 │ UInt64 │ 0 │ + 250. │ 249 │ UInt64 │ 1 │ + 251. │ 250 │ UInt64 │ 0 │ + 252. │ 251 │ UInt64 │ 1 │ + 253. │ 252 │ UInt64 │ 0 │ + 254. │ 253 │ UInt64 │ 1 │ + 255. │ 254 │ UInt64 │ 0 │ + 256. │ 255 │ UInt64 │ 1 │ + 257. │ 256 │ UInt64 │ 0 │ + 258. │ 257 │ UInt64 │ 1 │ + 259. │ 258 │ UInt64 │ 0 │ + 260. │ 259 │ UInt64 │ 1 │ + 261. │ 260 │ UInt64 │ 0 │ + 262. │ 261 │ UInt64 │ 1 │ + 263. │ 262 │ UInt64 │ 0 │ + 264. │ 263 │ UInt64 │ 1 │ + 265. │ 264 │ UInt64 │ 0 │ + 266. │ 265 │ UInt64 │ 1 │ + 267. │ 266 │ UInt64 │ 0 │ + 268. │ 267 │ UInt64 │ 1 │ + 269. │ 268 │ UInt64 │ 0 │ + 270. │ 269 │ UInt64 │ 1 │ + 271. │ 270 │ UInt64 │ 0 │ + 272. │ 271 │ UInt64 │ 1 │ + 273. │ 272 │ UInt64 │ 0 │ + 274. │ 273 │ UInt64 │ 1 │ + 275. │ 274 │ UInt64 │ 0 │ + 276. │ 275 │ UInt64 │ 1 │ + 277. │ 276 │ UInt64 │ 0 │ + 278. │ 277 │ UInt64 │ 1 │ + 279. │ 278 │ UInt64 │ 0 │ + 280. │ 279 │ UInt64 │ 1 │ + 281. │ 280 │ UInt64 │ 0 │ + 282. │ 281 │ UInt64 │ 1 │ + 283. │ 282 │ UInt64 │ 0 │ + 284. │ 283 │ UInt64 │ 1 │ + 285. │ 284 │ UInt64 │ 0 │ + 286. │ 285 │ UInt64 │ 1 │ + 287. │ 286 │ UInt64 │ 0 │ + 288. │ 287 │ UInt64 │ 1 │ + 289. │ 288 │ UInt64 │ 0 │ + 290. │ 289 │ UInt64 │ 1 │ + 291. │ 290 │ UInt64 │ 0 │ + 292. │ 291 │ UInt64 │ 1 │ + 293. │ 292 │ UInt64 │ 0 │ + 294. │ 293 │ UInt64 │ 1 │ + 295. │ 294 │ UInt64 │ 0 │ + 296. │ 295 │ UInt64 │ 1 │ + 297. │ 296 │ UInt64 │ 0 │ + 298. │ 297 │ UInt64 │ 1 │ + 299. │ 298 │ UInt64 │ 0 │ + 300. │ 299 │ UInt64 │ 1 │ + 301. │ 300 │ UInt64 │ 0 │ + 302. │ 301 │ UInt64 │ 1 │ + 303. │ 302 │ UInt64 │ 0 │ + 304. │ 303 │ UInt64 │ 1 │ + 305. │ 304 │ UInt64 │ 0 │ + 306. │ 305 │ UInt64 │ 1 │ + 307. │ 306 │ UInt64 │ 0 │ + 308. │ 307 │ UInt64 │ 1 │ + 309. │ 308 │ UInt64 │ 0 │ + 310. │ 309 │ UInt64 │ 1 │ + 311. │ 310 │ UInt64 │ 0 │ + 312. │ 311 │ UInt64 │ 1 │ + 313. │ 312 │ UInt64 │ 0 │ + 314. │ 313 │ UInt64 │ 1 │ + 315. │ 314 │ UInt64 │ 0 │ + 316. │ 315 │ UInt64 │ 1 │ + 317. │ 316 │ UInt64 │ 0 │ + 318. │ 317 │ UInt64 │ 1 │ + 319. │ 318 │ UInt64 │ 0 │ + 320. │ 319 │ UInt64 │ 1 │ + 321. │ 320 │ UInt64 │ 0 │ + 322. │ 321 │ UInt64 │ 1 │ + 323. │ 322 │ UInt64 │ 0 │ + 324. │ 323 │ UInt64 │ 1 │ + 325. │ 324 │ UInt64 │ 0 │ + 326. │ 325 │ UInt64 │ 1 │ + 327. │ 326 │ UInt64 │ 0 │ + 328. │ 327 │ UInt64 │ 1 │ + 329. │ 328 │ UInt64 │ 0 │ + 330. │ 329 │ UInt64 │ 1 │ + 331. │ 330 │ UInt64 │ 0 │ + 332. │ 331 │ UInt64 │ 1 │ + 333. │ 332 │ UInt64 │ 0 │ + 334. │ 333 │ UInt64 │ 1 │ + 335. │ 334 │ UInt64 │ 0 │ + 336. │ 335 │ UInt64 │ 1 │ + 337. │ 336 │ UInt64 │ 0 │ + 338. │ 337 │ UInt64 │ 1 │ + 339. │ 338 │ UInt64 │ 0 │ + 340. │ 339 │ UInt64 │ 1 │ + 341. │ 340 │ UInt64 │ 0 │ + 342. │ 341 │ UInt64 │ 1 │ + 343. │ 342 │ UInt64 │ 0 │ + 344. │ 343 │ UInt64 │ 1 │ + 345. │ 344 │ UInt64 │ 0 │ + 346. │ 345 │ UInt64 │ 1 │ + 347. │ 346 │ UInt64 │ 0 │ + 348. │ 347 │ UInt64 │ 1 │ + 349. │ 348 │ UInt64 │ 0 │ + 350. │ 349 │ UInt64 │ 1 │ + 351. │ 350 │ UInt64 │ 0 │ + 352. │ 351 │ UInt64 │ 1 │ + 353. │ 352 │ UInt64 │ 0 │ + 354. │ 353 │ UInt64 │ 1 │ + 355. │ 354 │ UInt64 │ 0 │ + 356. │ 355 │ UInt64 │ 1 │ + 357. │ 356 │ UInt64 │ 0 │ + 358. │ 357 │ UInt64 │ 1 │ + 359. │ 358 │ UInt64 │ 0 │ + 360. │ 359 │ UInt64 │ 1 │ + 361. │ 360 │ UInt64 │ 0 │ + 362. │ 361 │ UInt64 │ 1 │ + 363. │ 362 │ UInt64 │ 0 │ + 364. │ 363 │ UInt64 │ 1 │ + 365. │ 364 │ UInt64 │ 0 │ + 366. │ 365 │ UInt64 │ 1 │ + 367. │ 366 │ UInt64 │ 0 │ + 368. │ 367 │ UInt64 │ 1 │ + 369. │ 368 │ UInt64 │ 0 │ + 370. │ 369 │ UInt64 │ 1 │ + 371. │ 370 │ UInt64 │ 0 │ + 372. │ 371 │ UInt64 │ 1 │ + 373. │ 372 │ UInt64 │ 0 │ + 374. │ 373 │ UInt64 │ 1 │ + 375. │ 374 │ UInt64 │ 0 │ + 376. │ 375 │ UInt64 │ 1 │ + 377. │ 376 │ UInt64 │ 0 │ + 378. │ 377 │ UInt64 │ 1 │ + 379. │ 378 │ UInt64 │ 0 │ + 380. │ 379 │ UInt64 │ 1 │ + 381. │ 380 │ UInt64 │ 0 │ + 382. │ 381 │ UInt64 │ 1 │ + 383. │ 382 │ UInt64 │ 0 │ + 384. │ 383 │ UInt64 │ 1 │ + 385. │ 384 │ UInt64 │ 0 │ + 386. │ 385 │ UInt64 │ 1 │ + 387. │ 386 │ UInt64 │ 0 │ + 388. │ 387 │ UInt64 │ 1 │ + 389. │ 388 │ UInt64 │ 0 │ + 390. │ 389 │ UInt64 │ 1 │ + 391. │ 390 │ UInt64 │ 0 │ + 392. │ 391 │ UInt64 │ 1 │ + 393. │ 392 │ UInt64 │ 0 │ + 394. │ 393 │ UInt64 │ 1 │ + 395. │ 394 │ UInt64 │ 0 │ + 396. │ 395 │ UInt64 │ 1 │ + 397. │ 396 │ UInt64 │ 0 │ + 398. │ 397 │ UInt64 │ 1 │ + 399. │ 398 │ UInt64 │ 0 │ + 400. │ 399 │ UInt64 │ 1 │ + 401. │ 400 │ UInt64 │ 0 │ + 402. │ 401 │ UInt64 │ 1 │ + 403. │ 402 │ UInt64 │ 0 │ + 404. │ 403 │ UInt64 │ 1 │ + 405. │ 404 │ UInt64 │ 0 │ + 406. │ 405 │ UInt64 │ 1 │ + 407. │ 406 │ UInt64 │ 0 │ + 408. │ 407 │ UInt64 │ 1 │ + 409. │ 408 │ UInt64 │ 0 │ + 410. │ 409 │ UInt64 │ 1 │ + 411. │ 410 │ UInt64 │ 0 │ + 412. │ 411 │ UInt64 │ 1 │ + 413. │ 412 │ UInt64 │ 0 │ + 414. │ 413 │ UInt64 │ 1 │ + 415. │ 414 │ UInt64 │ 0 │ + 416. │ 415 │ UInt64 │ 1 │ + 417. │ 416 │ UInt64 │ 0 │ + 418. │ 417 │ UInt64 │ 1 │ + 419. │ 418 │ UInt64 │ 0 │ + 420. │ 419 │ UInt64 │ 1 │ + 421. │ 420 │ UInt64 │ 0 │ + 422. │ 421 │ UInt64 │ 1 │ + 423. │ 422 │ UInt64 │ 0 │ + 424. │ 423 │ UInt64 │ 1 │ + 425. │ 424 │ UInt64 │ 0 │ + 426. │ 425 │ UInt64 │ 1 │ + 427. │ 426 │ UInt64 │ 0 │ + 428. │ 427 │ UInt64 │ 1 │ + 429. │ 428 │ UInt64 │ 0 │ + 430. │ 429 │ UInt64 │ 1 │ + 431. │ 430 │ UInt64 │ 0 │ + 432. │ 431 │ UInt64 │ 1 │ + 433. │ 432 │ UInt64 │ 0 │ + 434. │ 433 │ UInt64 │ 1 │ + 435. │ 434 │ UInt64 │ 0 │ + 436. │ 435 │ UInt64 │ 1 │ + 437. │ 436 │ UInt64 │ 0 │ + 438. │ 437 │ UInt64 │ 1 │ + 439. │ 438 │ UInt64 │ 0 │ + 440. │ 439 │ UInt64 │ 1 │ + 441. │ 440 │ UInt64 │ 0 │ + 442. │ 441 │ UInt64 │ 1 │ + 443. │ 442 │ UInt64 │ 0 │ + 444. │ 443 │ UInt64 │ 1 │ + 445. │ 444 │ UInt64 │ 0 │ + 446. │ 445 │ UInt64 │ 1 │ + 447. │ 446 │ UInt64 │ 0 │ + 448. │ 447 │ UInt64 │ 1 │ + 449. │ 448 │ UInt64 │ 0 │ + 450. │ 449 │ UInt64 │ 1 │ + 451. │ 450 │ UInt64 │ 0 │ + 452. │ 451 │ UInt64 │ 1 │ + 453. │ 452 │ UInt64 │ 0 │ + 454. │ 453 │ UInt64 │ 1 │ + 455. │ 454 │ UInt64 │ 0 │ + 456. │ 455 │ UInt64 │ 1 │ + 457. │ 456 │ UInt64 │ 0 │ + 458. │ 457 │ UInt64 │ 1 │ + 459. │ 458 │ UInt64 │ 0 │ + 460. │ 459 │ UInt64 │ 1 │ + 461. │ 460 │ UInt64 │ 0 │ + 462. │ 461 │ UInt64 │ 1 │ + 463. │ 462 │ UInt64 │ 0 │ + 464. │ 463 │ UInt64 │ 1 │ + 465. │ 464 │ UInt64 │ 0 │ + 466. │ 465 │ UInt64 │ 1 │ + 467. │ 466 │ UInt64 │ 0 │ + 468. │ 467 │ UInt64 │ 1 │ + 469. │ 468 │ UInt64 │ 0 │ + 470. │ 469 │ UInt64 │ 1 │ + 471. │ 470 │ UInt64 │ 0 │ + 472. │ 471 │ UInt64 │ 1 │ + 473. │ 472 │ UInt64 │ 0 │ + 474. │ 473 │ UInt64 │ 1 │ + 475. │ 474 │ UInt64 │ 0 │ + 476. │ 475 │ UInt64 │ 1 │ + 477. │ 476 │ UInt64 │ 0 │ + 478. │ 477 │ UInt64 │ 1 │ + 479. │ 478 │ UInt64 │ 0 │ + 480. │ 479 │ UInt64 │ 1 │ + 481. │ 480 │ UInt64 │ 0 │ + 482. │ 481 │ UInt64 │ 1 │ + 483. │ 482 │ UInt64 │ 0 │ + 484. │ 483 │ UInt64 │ 1 │ + 485. │ 484 │ UInt64 │ 0 │ + 486. │ 485 │ UInt64 │ 1 │ + 487. │ 486 │ UInt64 │ 0 │ + 488. │ 487 │ UInt64 │ 1 │ + 489. │ 488 │ UInt64 │ 0 │ + 490. │ 489 │ UInt64 │ 1 │ + 491. │ 490 │ UInt64 │ 0 │ + 492. │ 491 │ UInt64 │ 1 │ + 493. │ 492 │ UInt64 │ 0 │ + 494. │ 493 │ UInt64 │ 1 │ + 495. │ 494 │ UInt64 │ 0 │ + 496. │ 495 │ UInt64 │ 1 │ + 497. │ 496 │ UInt64 │ 0 │ + 498. │ 497 │ UInt64 │ 1 │ + 499. │ 498 │ UInt64 │ 0 │ + 500. │ 499 │ UInt64 │ 1 │ + 501. │ 500 │ UInt64 │ 0 │ + 502. │ 501 │ UInt64 │ 1 │ + 503. │ 502 │ UInt64 │ 0 │ + 504. │ 503 │ UInt64 │ 1 │ + 505. │ 504 │ UInt64 │ 0 │ + 506. │ 505 │ UInt64 │ 1 │ + 507. │ 506 │ UInt64 │ 0 │ + 508. │ 507 │ UInt64 │ 1 │ + 509. │ 508 │ UInt64 │ 0 │ + 510. │ 509 │ UInt64 │ 1 │ + 511. │ 510 │ UInt64 │ 0 │ + 512. │ 511 │ UInt64 │ 1 │ + 513. │ 512 │ UInt64 │ 0 │ + 514. │ 513 │ UInt64 │ 1 │ + 515. │ 514 │ UInt64 │ 0 │ + 516. │ 515 │ UInt64 │ 1 │ + 517. │ 516 │ UInt64 │ 0 │ + 518. │ 517 │ UInt64 │ 1 │ + 519. │ 518 │ UInt64 │ 0 │ + 520. │ 519 │ UInt64 │ 1 │ + 521. │ 520 │ UInt64 │ 0 │ + 522. │ 521 │ UInt64 │ 1 │ + 523. │ 522 │ UInt64 │ 0 │ + 524. │ 523 │ UInt64 │ 1 │ + 525. │ 524 │ UInt64 │ 0 │ + 526. │ 525 │ UInt64 │ 1 │ + 527. │ 526 │ UInt64 │ 0 │ + 528. │ 527 │ UInt64 │ 1 │ + 529. │ 528 │ UInt64 │ 0 │ + 530. │ 529 │ UInt64 │ 1 │ + 531. │ 530 │ UInt64 │ 0 │ + 532. │ 531 │ UInt64 │ 1 │ + 533. │ 532 │ UInt64 │ 0 │ + 534. │ 533 │ UInt64 │ 1 │ + 535. │ 534 │ UInt64 │ 0 │ + 536. │ 535 │ UInt64 │ 1 │ + 537. │ 536 │ UInt64 │ 0 │ + 538. │ 537 │ UInt64 │ 1 │ + 539. │ 538 │ UInt64 │ 0 │ + 540. │ 539 │ UInt64 │ 1 │ + 541. │ 540 │ UInt64 │ 0 │ + 542. │ 541 │ UInt64 │ 1 │ + 543. │ 542 │ UInt64 │ 0 │ + 544. │ 543 │ UInt64 │ 1 │ + 545. │ 544 │ UInt64 │ 0 │ + 546. │ 545 │ UInt64 │ 1 │ + 547. │ 546 │ UInt64 │ 0 │ + 548. │ 547 │ UInt64 │ 1 │ + 549. │ 548 │ UInt64 │ 0 │ + 550. │ 549 │ UInt64 │ 1 │ + 551. │ 550 │ UInt64 │ 0 │ + 552. │ 551 │ UInt64 │ 1 │ + 553. │ 552 │ UInt64 │ 0 │ + 554. │ 553 │ UInt64 │ 1 │ + 555. │ 554 │ UInt64 │ 0 │ + 556. │ 555 │ UInt64 │ 1 │ + 557. │ 556 │ UInt64 │ 0 │ + 558. │ 557 │ UInt64 │ 1 │ + 559. │ 558 │ UInt64 │ 0 │ + 560. │ 559 │ UInt64 │ 1 │ + 561. │ 560 │ UInt64 │ 0 │ + 562. │ 561 │ UInt64 │ 1 │ + 563. │ 562 │ UInt64 │ 0 │ + 564. │ 563 │ UInt64 │ 1 │ + 565. │ 564 │ UInt64 │ 0 │ + 566. │ 565 │ UInt64 │ 1 │ + 567. │ 566 │ UInt64 │ 0 │ + 568. │ 567 │ UInt64 │ 1 │ + 569. │ 568 │ UInt64 │ 0 │ + 570. │ 569 │ UInt64 │ 1 │ + 571. │ 570 │ UInt64 │ 0 │ + 572. │ 571 │ UInt64 │ 1 │ + 573. │ 572 │ UInt64 │ 0 │ + 574. │ 573 │ UInt64 │ 1 │ + 575. │ 574 │ UInt64 │ 0 │ + 576. │ 575 │ UInt64 │ 1 │ + 577. │ 576 │ UInt64 │ 0 │ + 578. │ 577 │ UInt64 │ 1 │ + 579. │ 578 │ UInt64 │ 0 │ + 580. │ 579 │ UInt64 │ 1 │ + 581. │ 580 │ UInt64 │ 0 │ + 582. │ 581 │ UInt64 │ 1 │ + 583. │ 582 │ UInt64 │ 0 │ + 584. │ 583 │ UInt64 │ 1 │ + 585. │ 584 │ UInt64 │ 0 │ + 586. │ 585 │ UInt64 │ 1 │ + 587. │ 586 │ UInt64 │ 0 │ + 588. │ 587 │ UInt64 │ 1 │ + 589. │ 588 │ UInt64 │ 0 │ + 590. │ 589 │ UInt64 │ 1 │ + 591. │ 590 │ UInt64 │ 0 │ + 592. │ 591 │ UInt64 │ 1 │ + 593. │ 592 │ UInt64 │ 0 │ + 594. │ 593 │ UInt64 │ 1 │ + 595. │ 594 │ UInt64 │ 0 │ + 596. │ 595 │ UInt64 │ 1 │ + 597. │ 596 │ UInt64 │ 0 │ + 598. │ 597 │ UInt64 │ 1 │ + 599. │ 598 │ UInt64 │ 0 │ + 600. │ 599 │ UInt64 │ 1 │ + 601. │ 600 │ UInt64 │ 0 │ + 602. │ 601 │ UInt64 │ 1 │ + 603. │ 602 │ UInt64 │ 0 │ + 604. │ 603 │ UInt64 │ 1 │ + 605. │ 604 │ UInt64 │ 0 │ + 606. │ 605 │ UInt64 │ 1 │ + 607. │ 606 │ UInt64 │ 0 │ + 608. │ 607 │ UInt64 │ 1 │ + 609. │ 608 │ UInt64 │ 0 │ + 610. │ 609 │ UInt64 │ 1 │ + 611. │ 610 │ UInt64 │ 0 │ + 612. │ 611 │ UInt64 │ 1 │ + 613. │ 612 │ UInt64 │ 0 │ + 614. │ 613 │ UInt64 │ 1 │ + 615. │ 614 │ UInt64 │ 0 │ + 616. │ 615 │ UInt64 │ 1 │ + 617. │ 616 │ UInt64 │ 0 │ + 618. │ 617 │ UInt64 │ 1 │ + 619. │ 618 │ UInt64 │ 0 │ + 620. │ 619 │ UInt64 │ 1 │ + 621. │ 620 │ UInt64 │ 0 │ + 622. │ 621 │ UInt64 │ 1 │ + 623. │ 622 │ UInt64 │ 0 │ + 624. │ 623 │ UInt64 │ 1 │ + 625. │ 624 │ UInt64 │ 0 │ + 626. │ 625 │ UInt64 │ 1 │ + 627. │ 626 │ UInt64 │ 0 │ + 628. │ 627 │ UInt64 │ 1 │ + 629. │ 628 │ UInt64 │ 0 │ + 630. │ 629 │ UInt64 │ 1 │ + 631. │ 630 │ UInt64 │ 0 │ + 632. │ 631 │ UInt64 │ 1 │ + 633. │ 632 │ UInt64 │ 0 │ + 634. │ 633 │ UInt64 │ 1 │ + 635. │ 634 │ UInt64 │ 0 │ + 636. │ 635 │ UInt64 │ 1 │ + 637. │ 636 │ UInt64 │ 0 │ + 638. │ 637 │ UInt64 │ 1 │ + 639. │ 638 │ UInt64 │ 0 │ + 640. │ 639 │ UInt64 │ 1 │ + 641. │ 640 │ UInt64 │ 0 │ + 642. │ 641 │ UInt64 │ 1 │ + 643. │ 642 │ UInt64 │ 0 │ + 644. │ 643 │ UInt64 │ 1 │ + 645. │ 644 │ UInt64 │ 0 │ + 646. │ 645 │ UInt64 │ 1 │ + 647. │ 646 │ UInt64 │ 0 │ + 648. │ 647 │ UInt64 │ 1 │ + 649. │ 648 │ UInt64 │ 0 │ + 650. │ 649 │ UInt64 │ 1 │ + 651. │ 650 │ UInt64 │ 0 │ + 652. │ 651 │ UInt64 │ 1 │ + 653. │ 652 │ UInt64 │ 0 │ + 654. │ 653 │ UInt64 │ 1 │ + 655. │ 654 │ UInt64 │ 0 │ + 656. │ 655 │ UInt64 │ 1 │ + 657. │ 656 │ UInt64 │ 0 │ + 658. │ 657 │ UInt64 │ 1 │ + 659. │ 658 │ UInt64 │ 0 │ + 660. │ 659 │ UInt64 │ 1 │ + 661. │ 660 │ UInt64 │ 0 │ + 662. │ 661 │ UInt64 │ 1 │ + 663. │ 662 │ UInt64 │ 0 │ + 664. │ 663 │ UInt64 │ 1 │ + 665. │ 664 │ UInt64 │ 0 │ + 666. │ 665 │ UInt64 │ 1 │ + 667. │ 666 │ UInt64 │ 0 │ + 668. │ 667 │ UInt64 │ 1 │ + 669. │ 668 │ UInt64 │ 0 │ + 670. │ 669 │ UInt64 │ 1 │ + 671. │ 670 │ UInt64 │ 0 │ + 672. │ 671 │ UInt64 │ 1 │ + 673. │ 672 │ UInt64 │ 0 │ + 674. │ 673 │ UInt64 │ 1 │ + 675. │ 674 │ UInt64 │ 0 │ + 676. │ 675 │ UInt64 │ 1 │ + 677. │ 676 │ UInt64 │ 0 │ + 678. │ 677 │ UInt64 │ 1 │ + 679. │ 678 │ UInt64 │ 0 │ + 680. │ 679 │ UInt64 │ 1 │ + 681. │ 680 │ UInt64 │ 0 │ + 682. │ 681 │ UInt64 │ 1 │ + 683. │ 682 │ UInt64 │ 0 │ + 684. │ 683 │ UInt64 │ 1 │ + 685. │ 684 │ UInt64 │ 0 │ + 686. │ 685 │ UInt64 │ 1 │ + 687. │ 686 │ UInt64 │ 0 │ + 688. │ 687 │ UInt64 │ 1 │ + 689. │ 688 │ UInt64 │ 0 │ + 690. │ 689 │ UInt64 │ 1 │ + 691. │ 690 │ UInt64 │ 0 │ + 692. │ 691 │ UInt64 │ 1 │ + 693. │ 692 │ UInt64 │ 0 │ + 694. │ 693 │ UInt64 │ 1 │ + 695. │ 694 │ UInt64 │ 0 │ + 696. │ 695 │ UInt64 │ 1 │ + 697. │ 696 │ UInt64 │ 0 │ + 698. │ 697 │ UInt64 │ 1 │ + 699. │ 698 │ UInt64 │ 0 │ + 700. │ 699 │ UInt64 │ 1 │ + 701. │ 700 │ UInt64 │ 0 │ + 702. │ 701 │ UInt64 │ 1 │ + 703. │ 702 │ UInt64 │ 0 │ + 704. │ 703 │ UInt64 │ 1 │ + 705. │ 704 │ UInt64 │ 0 │ + 706. │ 705 │ UInt64 │ 1 │ + 707. │ 706 │ UInt64 │ 0 │ + 708. │ 707 │ UInt64 │ 1 │ + 709. │ 708 │ UInt64 │ 0 │ + 710. │ 709 │ UInt64 │ 1 │ + 711. │ 710 │ UInt64 │ 0 │ + 712. │ 711 │ UInt64 │ 1 │ + 713. │ 712 │ UInt64 │ 0 │ + 714. │ 713 │ UInt64 │ 1 │ + 715. │ 714 │ UInt64 │ 0 │ + 716. │ 715 │ UInt64 │ 1 │ + 717. │ 716 │ UInt64 │ 0 │ + 718. │ 717 │ UInt64 │ 1 │ + 719. │ 718 │ UInt64 │ 0 │ + 720. │ 719 │ UInt64 │ 1 │ + 721. │ 720 │ UInt64 │ 0 │ + 722. │ 721 │ UInt64 │ 1 │ + 723. │ 722 │ UInt64 │ 0 │ + 724. │ 723 │ UInt64 │ 1 │ + 725. │ 724 │ UInt64 │ 0 │ + 726. │ 725 │ UInt64 │ 1 │ + 727. │ 726 │ UInt64 │ 0 │ + 728. │ 727 │ UInt64 │ 1 │ + 729. │ 728 │ UInt64 │ 0 │ + 730. │ 729 │ UInt64 │ 1 │ + 731. │ 730 │ UInt64 │ 0 │ + 732. │ 731 │ UInt64 │ 1 │ + 733. │ 732 │ UInt64 │ 0 │ + 734. │ 733 │ UInt64 │ 1 │ + 735. │ 734 │ UInt64 │ 0 │ + 736. │ 735 │ UInt64 │ 1 │ + 737. │ 736 │ UInt64 │ 0 │ + 738. │ 737 │ UInt64 │ 1 │ + 739. │ 738 │ UInt64 │ 0 │ + 740. │ 739 │ UInt64 │ 1 │ + 741. │ 740 │ UInt64 │ 0 │ + 742. │ 741 │ UInt64 │ 1 │ + 743. │ 742 │ UInt64 │ 0 │ + 744. │ 743 │ UInt64 │ 1 │ + 745. │ 744 │ UInt64 │ 0 │ + 746. │ 745 │ UInt64 │ 1 │ + 747. │ 746 │ UInt64 │ 0 │ + 748. │ 747 │ UInt64 │ 1 │ + 749. │ 748 │ UInt64 │ 0 │ + 750. │ 749 │ UInt64 │ 1 │ + 751. │ 750 │ UInt64 │ 0 │ + 752. │ 751 │ UInt64 │ 1 │ + 753. │ 752 │ UInt64 │ 0 │ + 754. │ 753 │ UInt64 │ 1 │ + 755. │ 754 │ UInt64 │ 0 │ + 756. │ 755 │ UInt64 │ 1 │ + 757. │ 756 │ UInt64 │ 0 │ + 758. │ 757 │ UInt64 │ 1 │ + 759. │ 758 │ UInt64 │ 0 │ + 760. │ 759 │ UInt64 │ 1 │ + 761. │ 760 │ UInt64 │ 0 │ + 762. │ 761 │ UInt64 │ 1 │ + 763. │ 762 │ UInt64 │ 0 │ + 764. │ 763 │ UInt64 │ 1 │ + 765. │ 764 │ UInt64 │ 0 │ + 766. │ 765 │ UInt64 │ 1 │ + 767. │ 766 │ UInt64 │ 0 │ + 768. │ 767 │ UInt64 │ 1 │ + 769. │ 768 │ UInt64 │ 0 │ + 770. │ 769 │ UInt64 │ 1 │ + 771. │ 770 │ UInt64 │ 0 │ + 772. │ 771 │ UInt64 │ 1 │ + 773. │ 772 │ UInt64 │ 0 │ + 774. │ 773 │ UInt64 │ 1 │ + 775. │ 774 │ UInt64 │ 0 │ + 776. │ 775 │ UInt64 │ 1 │ + 777. │ 776 │ UInt64 │ 0 │ + 778. │ 777 │ UInt64 │ 1 │ + 779. │ 778 │ UInt64 │ 0 │ + 780. │ 779 │ UInt64 │ 1 │ + 781. │ 780 │ UInt64 │ 0 │ + 782. │ 781 │ UInt64 │ 1 │ + 783. │ 782 │ UInt64 │ 0 │ + 784. │ 783 │ UInt64 │ 1 │ + 785. │ 784 │ UInt64 │ 0 │ + 786. │ 785 │ UInt64 │ 1 │ + 787. │ 786 │ UInt64 │ 0 │ + 788. │ 787 │ UInt64 │ 1 │ + 789. │ 788 │ UInt64 │ 0 │ + 790. │ 789 │ UInt64 │ 1 │ + 791. │ 790 │ UInt64 │ 0 │ + 792. │ 791 │ UInt64 │ 1 │ + 793. │ 792 │ UInt64 │ 0 │ + 794. │ 793 │ UInt64 │ 1 │ + 795. │ 794 │ UInt64 │ 0 │ + 796. │ 795 │ UInt64 │ 1 │ + 797. │ 796 │ UInt64 │ 0 │ + 798. │ 797 │ UInt64 │ 1 │ + 799. │ 798 │ UInt64 │ 0 │ + 800. │ 799 │ UInt64 │ 1 │ + 801. │ 800 │ UInt64 │ 0 │ + 802. │ 801 │ UInt64 │ 1 │ + 803. │ 802 │ UInt64 │ 0 │ + 804. │ 803 │ UInt64 │ 1 │ + 805. │ 804 │ UInt64 │ 0 │ + 806. │ 805 │ UInt64 │ 1 │ + 807. │ 806 │ UInt64 │ 0 │ + 808. │ 807 │ UInt64 │ 1 │ + 809. │ 808 │ UInt64 │ 0 │ + 810. │ 809 │ UInt64 │ 1 │ + 811. │ 810 │ UInt64 │ 0 │ + 812. │ 811 │ UInt64 │ 1 │ + 813. │ 812 │ UInt64 │ 0 │ + 814. │ 813 │ UInt64 │ 1 │ + 815. │ 814 │ UInt64 │ 0 │ + 816. │ 815 │ UInt64 │ 1 │ + 817. │ 816 │ UInt64 │ 0 │ + 818. │ 817 │ UInt64 │ 1 │ + 819. │ 818 │ UInt64 │ 0 │ + 820. │ 819 │ UInt64 │ 1 │ + 821. │ 820 │ UInt64 │ 0 │ + 822. │ 821 │ UInt64 │ 1 │ + 823. │ 822 │ UInt64 │ 0 │ + 824. │ 823 │ UInt64 │ 1 │ + 825. │ 824 │ UInt64 │ 0 │ + 826. │ 825 │ UInt64 │ 1 │ + 827. │ 826 │ UInt64 │ 0 │ + 828. │ 827 │ UInt64 │ 1 │ + 829. │ 828 │ UInt64 │ 0 │ + 830. │ 829 │ UInt64 │ 1 │ + 831. │ 830 │ UInt64 │ 0 │ + 832. │ 831 │ UInt64 │ 1 │ + 833. │ 832 │ UInt64 │ 0 │ + 834. │ 833 │ UInt64 │ 1 │ + 835. │ 834 │ UInt64 │ 0 │ + 836. │ 835 │ UInt64 │ 1 │ + 837. │ 836 │ UInt64 │ 0 │ + 838. │ 837 │ UInt64 │ 1 │ + 839. │ 838 │ UInt64 │ 0 │ + 840. │ 839 │ UInt64 │ 1 │ + 841. │ 840 │ UInt64 │ 0 │ + 842. │ 841 │ UInt64 │ 1 │ + 843. │ 842 │ UInt64 │ 0 │ + 844. │ 843 │ UInt64 │ 1 │ + 845. │ 844 │ UInt64 │ 0 │ + 846. │ 845 │ UInt64 │ 1 │ + 847. │ 846 │ UInt64 │ 0 │ + 848. │ 847 │ UInt64 │ 1 │ + 849. │ 848 │ UInt64 │ 0 │ + 850. │ 849 │ UInt64 │ 1 │ + 851. │ 850 │ UInt64 │ 0 │ + 852. │ 851 │ UInt64 │ 1 │ + 853. │ 852 │ UInt64 │ 0 │ + 854. │ 853 │ UInt64 │ 1 │ + 855. │ 854 │ UInt64 │ 0 │ + 856. │ 855 │ UInt64 │ 1 │ + 857. │ 856 │ UInt64 │ 0 │ + 858. │ 857 │ UInt64 │ 1 │ + 859. │ 858 │ UInt64 │ 0 │ + 860. │ 859 │ UInt64 │ 1 │ + 861. │ 860 │ UInt64 │ 0 │ + 862. │ 861 │ UInt64 │ 1 │ + 863. │ 862 │ UInt64 │ 0 │ + 864. │ 863 │ UInt64 │ 1 │ + 865. │ 864 │ UInt64 │ 0 │ + 866. │ 865 │ UInt64 │ 1 │ + 867. │ 866 │ UInt64 │ 0 │ + 868. │ 867 │ UInt64 │ 1 │ + 869. │ 868 │ UInt64 │ 0 │ + 870. │ 869 │ UInt64 │ 1 │ + 871. │ 870 │ UInt64 │ 0 │ + 872. │ 871 │ UInt64 │ 1 │ + 873. │ 872 │ UInt64 │ 0 │ + 874. │ 873 │ UInt64 │ 1 │ + 875. │ 874 │ UInt64 │ 0 │ + 876. │ 875 │ UInt64 │ 1 │ + 877. │ 876 │ UInt64 │ 0 │ + 878. │ 877 │ UInt64 │ 1 │ + 879. │ 878 │ UInt64 │ 0 │ + 880. │ 879 │ UInt64 │ 1 │ + 881. │ 880 │ UInt64 │ 0 │ + 882. │ 881 │ UInt64 │ 1 │ + 883. │ 882 │ UInt64 │ 0 │ + 884. │ 883 │ UInt64 │ 1 │ + 885. │ 884 │ UInt64 │ 0 │ + 886. │ 885 │ UInt64 │ 1 │ + 887. │ 886 │ UInt64 │ 0 │ + 888. │ 887 │ UInt64 │ 1 │ + 889. │ 888 │ UInt64 │ 0 │ + 890. │ 889 │ UInt64 │ 1 │ + 891. │ 890 │ UInt64 │ 0 │ + 892. │ 891 │ UInt64 │ 1 │ + 893. │ 892 │ UInt64 │ 0 │ + 894. │ 893 │ UInt64 │ 1 │ + 895. │ 894 │ UInt64 │ 0 │ + 896. │ 895 │ UInt64 │ 1 │ + 897. │ 896 │ UInt64 │ 0 │ + 898. │ 897 │ UInt64 │ 1 │ + 899. │ 898 │ UInt64 │ 0 │ + 900. │ 899 │ UInt64 │ 1 │ + 901. │ 900 │ UInt64 │ 0 │ + 902. │ 901 │ UInt64 │ 1 │ + 903. │ 902 │ UInt64 │ 0 │ + 904. │ 903 │ UInt64 │ 1 │ + 905. │ 904 │ UInt64 │ 0 │ + 906. │ 905 │ UInt64 │ 1 │ + 907. │ 906 │ UInt64 │ 0 │ + 908. │ 907 │ UInt64 │ 1 │ + 909. │ 908 │ UInt64 │ 0 │ + 910. │ 909 │ UInt64 │ 1 │ + 911. │ 910 │ UInt64 │ 0 │ + 912. │ 911 │ UInt64 │ 1 │ + 913. │ 912 │ UInt64 │ 0 │ + 914. │ 913 │ UInt64 │ 1 │ + 915. │ 914 │ UInt64 │ 0 │ + 916. │ 915 │ UInt64 │ 1 │ + 917. │ 916 │ UInt64 │ 0 │ + 918. │ 917 │ UInt64 │ 1 │ + 919. │ 918 │ UInt64 │ 0 │ + 920. │ 919 │ UInt64 │ 1 │ + 921. │ 920 │ UInt64 │ 0 │ + 922. │ 921 │ UInt64 │ 1 │ + 923. │ 922 │ UInt64 │ 0 │ + 924. │ 923 │ UInt64 │ 1 │ + 925. │ 924 │ UInt64 │ 0 │ + 926. │ 925 │ UInt64 │ 1 │ + 927. │ 926 │ UInt64 │ 0 │ + 928. │ 927 │ UInt64 │ 1 │ + 929. │ 928 │ UInt64 │ 0 │ + 930. │ 929 │ UInt64 │ 1 │ + 931. │ 930 │ UInt64 │ 0 │ + 932. │ 931 │ UInt64 │ 1 │ + 933. │ 932 │ UInt64 │ 0 │ + 934. │ 933 │ UInt64 │ 1 │ + 935. │ 934 │ UInt64 │ 0 │ + 936. │ 935 │ UInt64 │ 1 │ + 937. │ 936 │ UInt64 │ 0 │ + 938. │ 937 │ UInt64 │ 1 │ + 939. │ 938 │ UInt64 │ 0 │ + 940. │ 939 │ UInt64 │ 1 │ + 941. │ 940 │ UInt64 │ 0 │ + 942. │ 941 │ UInt64 │ 1 │ + 943. │ 942 │ UInt64 │ 0 │ + 944. │ 943 │ UInt64 │ 1 │ + 945. │ 944 │ UInt64 │ 0 │ + 946. │ 945 │ UInt64 │ 1 │ + 947. │ 946 │ UInt64 │ 0 │ + 948. │ 947 │ UInt64 │ 1 │ + 949. │ 948 │ UInt64 │ 0 │ + 950. │ 949 │ UInt64 │ 1 │ + 951. │ 950 │ UInt64 │ 0 │ + 952. │ 951 │ UInt64 │ 1 │ + 953. │ 952 │ UInt64 │ 0 │ + 954. │ 953 │ UInt64 │ 1 │ + 955. │ 954 │ UInt64 │ 0 │ + 956. │ 955 │ UInt64 │ 1 │ + 957. │ 956 │ UInt64 │ 0 │ + 958. │ 957 │ UInt64 │ 1 │ + 959. │ 958 │ UInt64 │ 0 │ + 960. │ 959 │ UInt64 │ 1 │ + 961. │ 960 │ UInt64 │ 0 │ + 962. │ 961 │ UInt64 │ 1 │ + 963. │ 962 │ UInt64 │ 0 │ + 964. │ 963 │ UInt64 │ 1 │ + 965. │ 964 │ UInt64 │ 0 │ + 966. │ 965 │ UInt64 │ 1 │ + 967. │ 966 │ UInt64 │ 0 │ + 968. │ 967 │ UInt64 │ 1 │ + 969. │ 968 │ UInt64 │ 0 │ + 970. │ 969 │ UInt64 │ 1 │ + 971. │ 970 │ UInt64 │ 0 │ + 972. │ 971 │ UInt64 │ 1 │ + 973. │ 972 │ UInt64 │ 0 │ + 974. │ 973 │ UInt64 │ 1 │ + 975. │ 974 │ UInt64 │ 0 │ + 976. │ 975 │ UInt64 │ 1 │ + 977. │ 976 │ UInt64 │ 0 │ + 978. │ 977 │ UInt64 │ 1 │ + 979. │ 978 │ UInt64 │ 0 │ + 980. │ 979 │ UInt64 │ 1 │ + 981. │ 980 │ UInt64 │ 0 │ + 982. │ 981 │ UInt64 │ 1 │ + 983. │ 982 │ UInt64 │ 0 │ + 984. │ 983 │ UInt64 │ 1 │ + 985. │ 984 │ UInt64 │ 0 │ + 986. │ 985 │ UInt64 │ 1 │ + 987. │ 986 │ UInt64 │ 0 │ + 988. │ 987 │ UInt64 │ 1 │ + 989. │ 988 │ UInt64 │ 0 │ + 990. │ 989 │ UInt64 │ 1 │ + 991. │ 990 │ UInt64 │ 0 │ + 992. │ 991 │ UInt64 │ 1 │ + 993. │ 992 │ UInt64 │ 0 │ + 994. │ 993 │ UInt64 │ 1 │ + 995. │ 994 │ UInt64 │ 0 │ + 996. │ 995 │ UInt64 │ 1 │ + 997. │ 996 │ UInt64 │ 0 │ + 998. │ 997 │ UInt64 │ 1 │ + 999. │ 998 │ UInt64 │ 0 │ +1000. │ 999 │ UInt64 │ 1 │ + └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ + 100. │ 99 │ UInt64 │ 1 │ + 101. │ 100 │ UInt64 │ 0 │ + 102. │ 101 │ UInt64 │ 1 │ + 103. │ 102 │ UInt64 │ 0 │ + 104. │ 103 │ UInt64 │ 1 │ + 105. │ 104 │ UInt64 │ 0 │ + 106. │ 105 │ UInt64 │ 1 │ + 107. │ 106 │ UInt64 │ 0 │ + 108. │ 107 │ UInt64 │ 1 │ + 109. │ 108 │ UInt64 │ 0 │ + 110. │ 109 │ UInt64 │ 1 │ + 111. │ 110 │ UInt64 │ 0 │ + 112. │ 111 │ UInt64 │ 1 │ + 113. │ 112 │ UInt64 │ 0 │ + 114. │ 113 │ UInt64 │ 1 │ + 115. │ 114 │ UInt64 │ 0 │ + 116. │ 115 │ UInt64 │ 1 │ + 117. │ 116 │ UInt64 │ 0 │ + 118. │ 117 │ UInt64 │ 1 │ + 119. │ 118 │ UInt64 │ 0 │ + 120. │ 119 │ UInt64 │ 1 │ + 121. │ 120 │ UInt64 │ 0 │ + 122. │ 121 │ UInt64 │ 1 │ + 123. │ 122 │ UInt64 │ 0 │ + 124. │ 123 │ UInt64 │ 1 │ + 125. │ 124 │ UInt64 │ 0 │ + 126. │ 125 │ UInt64 │ 1 │ + 127. │ 126 │ UInt64 │ 0 │ + 128. │ 127 │ UInt64 │ 1 │ + 129. │ 128 │ UInt64 │ 0 │ + 130. │ 129 │ UInt64 │ 1 │ + 131. │ 130 │ UInt64 │ 0 │ + 132. │ 131 │ UInt64 │ 1 │ + 133. │ 132 │ UInt64 │ 0 │ + 134. │ 133 │ UInt64 │ 1 │ + 135. │ 134 │ UInt64 │ 0 │ + 136. │ 135 │ UInt64 │ 1 │ + 137. │ 136 │ UInt64 │ 0 │ + 138. │ 137 │ UInt64 │ 1 │ + 139. │ 138 │ UInt64 │ 0 │ + 140. │ 139 │ UInt64 │ 1 │ + 141. │ 140 │ UInt64 │ 0 │ + 142. │ 141 │ UInt64 │ 1 │ + 143. │ 142 │ UInt64 │ 0 │ + 144. │ 143 │ UInt64 │ 1 │ + 145. │ 144 │ UInt64 │ 0 │ + 146. │ 145 │ UInt64 │ 1 │ + 147. │ 146 │ UInt64 │ 0 │ + 148. │ 147 │ UInt64 │ 1 │ + 149. │ 148 │ UInt64 │ 0 │ + 150. │ 149 │ UInt64 │ 1 │ + 151. │ 150 │ UInt64 │ 0 │ + 152. │ 151 │ UInt64 │ 1 │ + 153. │ 152 │ UInt64 │ 0 │ + 154. │ 153 │ UInt64 │ 1 │ + 155. │ 154 │ UInt64 │ 0 │ + 156. │ 155 │ UInt64 │ 1 │ + 157. │ 156 │ UInt64 │ 0 │ + 158. │ 157 │ UInt64 │ 1 │ + 159. │ 158 │ UInt64 │ 0 │ + 160. │ 159 │ UInt64 │ 1 │ + 161. │ 160 │ UInt64 │ 0 │ + 162. │ 161 │ UInt64 │ 1 │ + 163. │ 162 │ UInt64 │ 0 │ + 164. │ 163 │ UInt64 │ 1 │ + 165. │ 164 │ UInt64 │ 0 │ + 166. │ 165 │ UInt64 │ 1 │ + 167. │ 166 │ UInt64 │ 0 │ + 168. │ 167 │ UInt64 │ 1 │ + 169. │ 168 │ UInt64 │ 0 │ + 170. │ 169 │ UInt64 │ 1 │ + 171. │ 170 │ UInt64 │ 0 │ + 172. │ 171 │ UInt64 │ 1 │ + 173. │ 172 │ UInt64 │ 0 │ + 174. │ 173 │ UInt64 │ 1 │ + 175. │ 174 │ UInt64 │ 0 │ + 176. │ 175 │ UInt64 │ 1 │ + 177. │ 176 │ UInt64 │ 0 │ + 178. │ 177 │ UInt64 │ 1 │ + 179. │ 178 │ UInt64 │ 0 │ + 180. │ 179 │ UInt64 │ 1 │ + 181. │ 180 │ UInt64 │ 0 │ + 182. │ 181 │ UInt64 │ 1 │ + 183. │ 182 │ UInt64 │ 0 │ + 184. │ 183 │ UInt64 │ 1 │ + 185. │ 184 │ UInt64 │ 0 │ + 186. │ 185 │ UInt64 │ 1 │ + 187. │ 186 │ UInt64 │ 0 │ + 188. │ 187 │ UInt64 │ 1 │ + 189. │ 188 │ UInt64 │ 0 │ + 190. │ 189 │ UInt64 │ 1 │ + 191. │ 190 │ UInt64 │ 0 │ + 192. │ 191 │ UInt64 │ 1 │ + 193. │ 192 │ UInt64 │ 0 │ + 194. │ 193 │ UInt64 │ 1 │ + 195. │ 194 │ UInt64 │ 0 │ + 196. │ 195 │ UInt64 │ 1 │ + 197. │ 196 │ UInt64 │ 0 │ + 198. │ 197 │ UInt64 │ 1 │ + 199. │ 198 │ UInt64 │ 0 │ + 200. │ 199 │ UInt64 │ 1 │ + 201. │ 200 │ UInt64 │ 0 │ + 202. │ 201 │ UInt64 │ 1 │ + 203. │ 202 │ UInt64 │ 0 │ + 204. │ 203 │ UInt64 │ 1 │ + 205. │ 204 │ UInt64 │ 0 │ + 206. │ 205 │ UInt64 │ 1 │ + 207. │ 206 │ UInt64 │ 0 │ + 208. │ 207 │ UInt64 │ 1 │ + 209. │ 208 │ UInt64 │ 0 │ + 210. │ 209 │ UInt64 │ 1 │ + 211. │ 210 │ UInt64 │ 0 │ + 212. │ 211 │ UInt64 │ 1 │ + 213. │ 212 │ UInt64 │ 0 │ + 214. │ 213 │ UInt64 │ 1 │ + 215. │ 214 │ UInt64 │ 0 │ + 216. │ 215 │ UInt64 │ 1 │ + 217. │ 216 │ UInt64 │ 0 │ + 218. │ 217 │ UInt64 │ 1 │ + 219. │ 218 │ UInt64 │ 0 │ + 220. │ 219 │ UInt64 │ 1 │ + 221. │ 220 │ UInt64 │ 0 │ + 222. │ 221 │ UInt64 │ 1 │ + 223. │ 222 │ UInt64 │ 0 │ + 224. │ 223 │ UInt64 │ 1 │ + 225. │ 224 │ UInt64 │ 0 │ + 226. │ 225 │ UInt64 │ 1 │ + 227. │ 226 │ UInt64 │ 0 │ + 228. │ 227 │ UInt64 │ 1 │ + 229. │ 228 │ UInt64 │ 0 │ + 230. │ 229 │ UInt64 │ 1 │ + 231. │ 230 │ UInt64 │ 0 │ + 232. │ 231 │ UInt64 │ 1 │ + 233. │ 232 │ UInt64 │ 0 │ + 234. │ 233 │ UInt64 │ 1 │ + 235. │ 234 │ UInt64 │ 0 │ + 236. │ 235 │ UInt64 │ 1 │ + 237. │ 236 │ UInt64 │ 0 │ + 238. │ 237 │ UInt64 │ 1 │ + 239. │ 238 │ UInt64 │ 0 │ + 240. │ 239 │ UInt64 │ 1 │ + 241. │ 240 │ UInt64 │ 0 │ + 242. │ 241 │ UInt64 │ 1 │ + 243. │ 242 │ UInt64 │ 0 │ + 244. │ 243 │ UInt64 │ 1 │ + 245. │ 244 │ UInt64 │ 0 │ + 246. │ 245 │ UInt64 │ 1 │ + 247. │ 246 │ UInt64 │ 0 │ + 248. │ 247 │ UInt64 │ 1 │ + 249. │ 248 │ UInt64 │ 0 │ + 250. │ 249 │ UInt64 │ 1 │ + 251. │ 250 │ UInt64 │ 0 │ + 252. │ 251 │ UInt64 │ 1 │ + 253. │ 252 │ UInt64 │ 0 │ + 254. │ 253 │ UInt64 │ 1 │ + 255. │ 254 │ UInt64 │ 0 │ + 256. │ 255 │ UInt64 │ 1 │ + 257. │ 256 │ UInt64 │ 0 │ + 258. │ 257 │ UInt64 │ 1 │ + 259. │ 258 │ UInt64 │ 0 │ + 260. │ 259 │ UInt64 │ 1 │ + 261. │ 260 │ UInt64 │ 0 │ + 262. │ 261 │ UInt64 │ 1 │ + 263. │ 262 │ UInt64 │ 0 │ + 264. │ 263 │ UInt64 │ 1 │ + 265. │ 264 │ UInt64 │ 0 │ + 266. │ 265 │ UInt64 │ 1 │ + 267. │ 266 │ UInt64 │ 0 │ + 268. │ 267 │ UInt64 │ 1 │ + 269. │ 268 │ UInt64 │ 0 │ + 270. │ 269 │ UInt64 │ 1 │ + 271. │ 270 │ UInt64 │ 0 │ + 272. │ 271 │ UInt64 │ 1 │ + 273. │ 272 │ UInt64 │ 0 │ + 274. │ 273 │ UInt64 │ 1 │ + 275. │ 274 │ UInt64 │ 0 │ + 276. │ 275 │ UInt64 │ 1 │ + 277. │ 276 │ UInt64 │ 0 │ + 278. │ 277 │ UInt64 │ 1 │ + 279. │ 278 │ UInt64 │ 0 │ + 280. │ 279 │ UInt64 │ 1 │ + 281. │ 280 │ UInt64 │ 0 │ + 282. │ 281 │ UInt64 │ 1 │ + 283. │ 282 │ UInt64 │ 0 │ + 284. │ 283 │ UInt64 │ 1 │ + 285. │ 284 │ UInt64 │ 0 │ + 286. │ 285 │ UInt64 │ 1 │ + 287. │ 286 │ UInt64 │ 0 │ + 288. │ 287 │ UInt64 │ 1 │ + 289. │ 288 │ UInt64 │ 0 │ + 290. │ 289 │ UInt64 │ 1 │ + 291. │ 290 │ UInt64 │ 0 │ + 292. │ 291 │ UInt64 │ 1 │ + 293. │ 292 │ UInt64 │ 0 │ + 294. │ 293 │ UInt64 │ 1 │ + 295. │ 294 │ UInt64 │ 0 │ + 296. │ 295 │ UInt64 │ 1 │ + 297. │ 296 │ UInt64 │ 0 │ + 298. │ 297 │ UInt64 │ 1 │ + 299. │ 298 │ UInt64 │ 0 │ + 300. │ 299 │ UInt64 │ 1 │ + 301. │ 300 │ UInt64 │ 0 │ + 302. │ 301 │ UInt64 │ 1 │ + 303. │ 302 │ UInt64 │ 0 │ + 304. │ 303 │ UInt64 │ 1 │ + 305. │ 304 │ UInt64 │ 0 │ + 306. │ 305 │ UInt64 │ 1 │ + 307. │ 306 │ UInt64 │ 0 │ + 308. │ 307 │ UInt64 │ 1 │ + 309. │ 308 │ UInt64 │ 0 │ + 310. │ 309 │ UInt64 │ 1 │ + 311. │ 310 │ UInt64 │ 0 │ + 312. │ 311 │ UInt64 │ 1 │ + 313. │ 312 │ UInt64 │ 0 │ + 314. │ 313 │ UInt64 │ 1 │ + 315. │ 314 │ UInt64 │ 0 │ + 316. │ 315 │ UInt64 │ 1 │ + 317. │ 316 │ UInt64 │ 0 │ + 318. │ 317 │ UInt64 │ 1 │ + 319. │ 318 │ UInt64 │ 0 │ + 320. │ 319 │ UInt64 │ 1 │ + 321. │ 320 │ UInt64 │ 0 │ + 322. │ 321 │ UInt64 │ 1 │ + 323. │ 322 │ UInt64 │ 0 │ + 324. │ 323 │ UInt64 │ 1 │ + 325. │ 324 │ UInt64 │ 0 │ + 326. │ 325 │ UInt64 │ 1 │ + 327. │ 326 │ UInt64 │ 0 │ + 328. │ 327 │ UInt64 │ 1 │ + 329. │ 328 │ UInt64 │ 0 │ + 330. │ 329 │ UInt64 │ 1 │ + 331. │ 330 │ UInt64 │ 0 │ + 332. │ 331 │ UInt64 │ 1 │ + 333. │ 332 │ UInt64 │ 0 │ + 334. │ 333 │ UInt64 │ 1 │ + 335. │ 334 │ UInt64 │ 0 │ + 336. │ 335 │ UInt64 │ 1 │ + 337. │ 336 │ UInt64 │ 0 │ + 338. │ 337 │ UInt64 │ 1 │ + 339. │ 338 │ UInt64 │ 0 │ + 340. │ 339 │ UInt64 │ 1 │ + 341. │ 340 │ UInt64 │ 0 │ + 342. │ 341 │ UInt64 │ 1 │ + 343. │ 342 │ UInt64 │ 0 │ + 344. │ 343 │ UInt64 │ 1 │ + 345. │ 344 │ UInt64 │ 0 │ + 346. │ 345 │ UInt64 │ 1 │ + 347. │ 346 │ UInt64 │ 0 │ + 348. │ 347 │ UInt64 │ 1 │ + 349. │ 348 │ UInt64 │ 0 │ + 350. │ 349 │ UInt64 │ 1 │ + 351. │ 350 │ UInt64 │ 0 │ + 352. │ 351 │ UInt64 │ 1 │ + 353. │ 352 │ UInt64 │ 0 │ + 354. │ 353 │ UInt64 │ 1 │ + 355. │ 354 │ UInt64 │ 0 │ + 356. │ 355 │ UInt64 │ 1 │ + 357. │ 356 │ UInt64 │ 0 │ + 358. │ 357 │ UInt64 │ 1 │ + 359. │ 358 │ UInt64 │ 0 │ + 360. │ 359 │ UInt64 │ 1 │ + 361. │ 360 │ UInt64 │ 0 │ + 362. │ 361 │ UInt64 │ 1 │ + 363. │ 362 │ UInt64 │ 0 │ + 364. │ 363 │ UInt64 │ 1 │ + 365. │ 364 │ UInt64 │ 0 │ + 366. │ 365 │ UInt64 │ 1 │ + 367. │ 366 │ UInt64 │ 0 │ + 368. │ 367 │ UInt64 │ 1 │ + 369. │ 368 │ UInt64 │ 0 │ + 370. │ 369 │ UInt64 │ 1 │ + 371. │ 370 │ UInt64 │ 0 │ + 372. │ 371 │ UInt64 │ 1 │ + 373. │ 372 │ UInt64 │ 0 │ + 374. │ 373 │ UInt64 │ 1 │ + 375. │ 374 │ UInt64 │ 0 │ + 376. │ 375 │ UInt64 │ 1 │ + 377. │ 376 │ UInt64 │ 0 │ + 378. │ 377 │ UInt64 │ 1 │ + 379. │ 378 │ UInt64 │ 0 │ + 380. │ 379 │ UInt64 │ 1 │ + 381. │ 380 │ UInt64 │ 0 │ + 382. │ 381 │ UInt64 │ 1 │ + 383. │ 382 │ UInt64 │ 0 │ + 384. │ 383 │ UInt64 │ 1 │ + 385. │ 384 │ UInt64 │ 0 │ + 386. │ 385 │ UInt64 │ 1 │ + 387. │ 386 │ UInt64 │ 0 │ + 388. │ 387 │ UInt64 │ 1 │ + 389. │ 388 │ UInt64 │ 0 │ + 390. │ 389 │ UInt64 │ 1 │ + 391. │ 390 │ UInt64 │ 0 │ + 392. │ 391 │ UInt64 │ 1 │ + 393. │ 392 │ UInt64 │ 0 │ + 394. │ 393 │ UInt64 │ 1 │ + 395. │ 394 │ UInt64 │ 0 │ + 396. │ 395 │ UInt64 │ 1 │ + 397. │ 396 │ UInt64 │ 0 │ + 398. │ 397 │ UInt64 │ 1 │ + 399. │ 398 │ UInt64 │ 0 │ + 400. │ 399 │ UInt64 │ 1 │ + 401. │ 400 │ UInt64 │ 0 │ + 402. │ 401 │ UInt64 │ 1 │ + 403. │ 402 │ UInt64 │ 0 │ + 404. │ 403 │ UInt64 │ 1 │ + 405. │ 404 │ UInt64 │ 0 │ + 406. │ 405 │ UInt64 │ 1 │ + 407. │ 406 │ UInt64 │ 0 │ + 408. │ 407 │ UInt64 │ 1 │ + 409. │ 408 │ UInt64 │ 0 │ + 410. │ 409 │ UInt64 │ 1 │ + 411. │ 410 │ UInt64 │ 0 │ + 412. │ 411 │ UInt64 │ 1 │ + 413. │ 412 │ UInt64 │ 0 │ + 414. │ 413 │ UInt64 │ 1 │ + 415. │ 414 │ UInt64 │ 0 │ + 416. │ 415 │ UInt64 │ 1 │ + 417. │ 416 │ UInt64 │ 0 │ + 418. │ 417 │ UInt64 │ 1 │ + 419. │ 418 │ UInt64 │ 0 │ + 420. │ 419 │ UInt64 │ 1 │ + 421. │ 420 │ UInt64 │ 0 │ + 422. │ 421 │ UInt64 │ 1 │ + 423. │ 422 │ UInt64 │ 0 │ + 424. │ 423 │ UInt64 │ 1 │ + 425. │ 424 │ UInt64 │ 0 │ + 426. │ 425 │ UInt64 │ 1 │ + 427. │ 426 │ UInt64 │ 0 │ + 428. │ 427 │ UInt64 │ 1 │ + 429. │ 428 │ UInt64 │ 0 │ + 430. │ 429 │ UInt64 │ 1 │ + 431. │ 430 │ UInt64 │ 0 │ + 432. │ 431 │ UInt64 │ 1 │ + 433. │ 432 │ UInt64 │ 0 │ + 434. │ 433 │ UInt64 │ 1 │ + 435. │ 434 │ UInt64 │ 0 │ + 436. │ 435 │ UInt64 │ 1 │ + 437. │ 436 │ UInt64 │ 0 │ + 438. │ 437 │ UInt64 │ 1 │ + 439. │ 438 │ UInt64 │ 0 │ + 440. │ 439 │ UInt64 │ 1 │ + 441. │ 440 │ UInt64 │ 0 │ + 442. │ 441 │ UInt64 │ 1 │ + 443. │ 442 │ UInt64 │ 0 │ + 444. │ 443 │ UInt64 │ 1 │ + 445. │ 444 │ UInt64 │ 0 │ + 446. │ 445 │ UInt64 │ 1 │ + 447. │ 446 │ UInt64 │ 0 │ + 448. │ 447 │ UInt64 │ 1 │ + 449. │ 448 │ UInt64 │ 0 │ + 450. │ 449 │ UInt64 │ 1 │ + 451. │ 450 │ UInt64 │ 0 │ + 452. │ 451 │ UInt64 │ 1 │ + 453. │ 452 │ UInt64 │ 0 │ + 454. │ 453 │ UInt64 │ 1 │ + 455. │ 454 │ UInt64 │ 0 │ + 456. │ 455 │ UInt64 │ 1 │ + 457. │ 456 │ UInt64 │ 0 │ + 458. │ 457 │ UInt64 │ 1 │ + 459. │ 458 │ UInt64 │ 0 │ + 460. │ 459 │ UInt64 │ 1 │ + 461. │ 460 │ UInt64 │ 0 │ + 462. │ 461 │ UInt64 │ 1 │ + 463. │ 462 │ UInt64 │ 0 │ + 464. │ 463 │ UInt64 │ 1 │ + 465. │ 464 │ UInt64 │ 0 │ + 466. │ 465 │ UInt64 │ 1 │ + 467. │ 466 │ UInt64 │ 0 │ + 468. │ 467 │ UInt64 │ 1 │ + 469. │ 468 │ UInt64 │ 0 │ + 470. │ 469 │ UInt64 │ 1 │ + 471. │ 470 │ UInt64 │ 0 │ + 472. │ 471 │ UInt64 │ 1 │ + 473. │ 472 │ UInt64 │ 0 │ + 474. │ 473 │ UInt64 │ 1 │ + 475. │ 474 │ UInt64 │ 0 │ + 476. │ 475 │ UInt64 │ 1 │ + 477. │ 476 │ UInt64 │ 0 │ + 478. │ 477 │ UInt64 │ 1 │ + 479. │ 478 │ UInt64 │ 0 │ + 480. │ 479 │ UInt64 │ 1 │ + 481. │ 480 │ UInt64 │ 0 │ + 482. │ 481 │ UInt64 │ 1 │ + 483. │ 482 │ UInt64 │ 0 │ + 484. │ 483 │ UInt64 │ 1 │ + 485. │ 484 │ UInt64 │ 0 │ + 486. │ 485 │ UInt64 │ 1 │ + 487. │ 486 │ UInt64 │ 0 │ + 488. │ 487 │ UInt64 │ 1 │ + 489. │ 488 │ UInt64 │ 0 │ + 490. │ 489 │ UInt64 │ 1 │ + 491. │ 490 │ UInt64 │ 0 │ + 492. │ 491 │ UInt64 │ 1 │ + 493. │ 492 │ UInt64 │ 0 │ + 494. │ 493 │ UInt64 │ 1 │ + 495. │ 494 │ UInt64 │ 0 │ + 496. │ 495 │ UInt64 │ 1 │ + 497. │ 496 │ UInt64 │ 0 │ + 498. │ 497 │ UInt64 │ 1 │ + 499. │ 498 │ UInt64 │ 0 │ + 500. │ 499 │ UInt64 │ 1 │ + 501. │ 500 │ UInt64 │ 0 │ + 502. │ 501 │ UInt64 │ 1 │ + 503. │ 502 │ UInt64 │ 0 │ + 504. │ 503 │ UInt64 │ 1 │ + 505. │ 504 │ UInt64 │ 0 │ + 506. │ 505 │ UInt64 │ 1 │ + 507. │ 506 │ UInt64 │ 0 │ + 508. │ 507 │ UInt64 │ 1 │ + 509. │ 508 │ UInt64 │ 0 │ + 510. │ 509 │ UInt64 │ 1 │ + 511. │ 510 │ UInt64 │ 0 │ + 512. │ 511 │ UInt64 │ 1 │ + 513. │ 512 │ UInt64 │ 0 │ + 514. │ 513 │ UInt64 │ 1 │ + 515. │ 514 │ UInt64 │ 0 │ + 516. │ 515 │ UInt64 │ 1 │ + 517. │ 516 │ UInt64 │ 0 │ + 518. │ 517 │ UInt64 │ 1 │ + 519. │ 518 │ UInt64 │ 0 │ + 520. │ 519 │ UInt64 │ 1 │ + 521. │ 520 │ UInt64 │ 0 │ + 522. │ 521 │ UInt64 │ 1 │ + 523. │ 522 │ UInt64 │ 0 │ + 524. │ 523 │ UInt64 │ 1 │ + 525. │ 524 │ UInt64 │ 0 │ + 526. │ 525 │ UInt64 │ 1 │ + 527. │ 526 │ UInt64 │ 0 │ + 528. │ 527 │ UInt64 │ 1 │ + 529. │ 528 │ UInt64 │ 0 │ + 530. │ 529 │ UInt64 │ 1 │ + 531. │ 530 │ UInt64 │ 0 │ + 532. │ 531 │ UInt64 │ 1 │ + 533. │ 532 │ UInt64 │ 0 │ + 534. │ 533 │ UInt64 │ 1 │ + 535. │ 534 │ UInt64 │ 0 │ + 536. │ 535 │ UInt64 │ 1 │ + 537. │ 536 │ UInt64 │ 0 │ + 538. │ 537 │ UInt64 │ 1 │ + 539. │ 538 │ UInt64 │ 0 │ + 540. │ 539 │ UInt64 │ 1 │ + 541. │ 540 │ UInt64 │ 0 │ + 542. │ 541 │ UInt64 │ 1 │ + 543. │ 542 │ UInt64 │ 0 │ + 544. │ 543 │ UInt64 │ 1 │ + 545. │ 544 │ UInt64 │ 0 │ + 546. │ 545 │ UInt64 │ 1 │ + 547. │ 546 │ UInt64 │ 0 │ + 548. │ 547 │ UInt64 │ 1 │ + 549. │ 548 │ UInt64 │ 0 │ + 550. │ 549 │ UInt64 │ 1 │ + 551. │ 550 │ UInt64 │ 0 │ + 552. │ 551 │ UInt64 │ 1 │ + 553. │ 552 │ UInt64 │ 0 │ + 554. │ 553 │ UInt64 │ 1 │ + 555. │ 554 │ UInt64 │ 0 │ + 556. │ 555 │ UInt64 │ 1 │ + 557. │ 556 │ UInt64 │ 0 │ + 558. │ 557 │ UInt64 │ 1 │ + 559. │ 558 │ UInt64 │ 0 │ + 560. │ 559 │ UInt64 │ 1 │ + 561. │ 560 │ UInt64 │ 0 │ + 562. │ 561 │ UInt64 │ 1 │ + 563. │ 562 │ UInt64 │ 0 │ + 564. │ 563 │ UInt64 │ 1 │ + 565. │ 564 │ UInt64 │ 0 │ + 566. │ 565 │ UInt64 │ 1 │ + 567. │ 566 │ UInt64 │ 0 │ + 568. │ 567 │ UInt64 │ 1 │ + 569. │ 568 │ UInt64 │ 0 │ + 570. │ 569 │ UInt64 │ 1 │ + 571. │ 570 │ UInt64 │ 0 │ + 572. │ 571 │ UInt64 │ 1 │ + 573. │ 572 │ UInt64 │ 0 │ + 574. │ 573 │ UInt64 │ 1 │ + 575. │ 574 │ UInt64 │ 0 │ + 576. │ 575 │ UInt64 │ 1 │ + 577. │ 576 │ UInt64 │ 0 │ + 578. │ 577 │ UInt64 │ 1 │ + 579. │ 578 │ UInt64 │ 0 │ + 580. │ 579 │ UInt64 │ 1 │ + 581. │ 580 │ UInt64 │ 0 │ + 582. │ 581 │ UInt64 │ 1 │ + 583. │ 582 │ UInt64 │ 0 │ + 584. │ 583 │ UInt64 │ 1 │ + 585. │ 584 │ UInt64 │ 0 │ + 586. │ 585 │ UInt64 │ 1 │ + 587. │ 586 │ UInt64 │ 0 │ + 588. │ 587 │ UInt64 │ 1 │ + 589. │ 588 │ UInt64 │ 0 │ + 590. │ 589 │ UInt64 │ 1 │ + 591. │ 590 │ UInt64 │ 0 │ + 592. │ 591 │ UInt64 │ 1 │ + 593. │ 592 │ UInt64 │ 0 │ + 594. │ 593 │ UInt64 │ 1 │ + 595. │ 594 │ UInt64 │ 0 │ + 596. │ 595 │ UInt64 │ 1 │ + 597. │ 596 │ UInt64 │ 0 │ + 598. │ 597 │ UInt64 │ 1 │ + 599. │ 598 │ UInt64 │ 0 │ + 600. │ 599 │ UInt64 │ 1 │ + 601. │ 600 │ UInt64 │ 0 │ + 602. │ 601 │ UInt64 │ 1 │ + 603. │ 602 │ UInt64 │ 0 │ + 604. │ 603 │ UInt64 │ 1 │ + 605. │ 604 │ UInt64 │ 0 │ + 606. │ 605 │ UInt64 │ 1 │ + 607. │ 606 │ UInt64 │ 0 │ + 608. │ 607 │ UInt64 │ 1 │ + 609. │ 608 │ UInt64 │ 0 │ + 610. │ 609 │ UInt64 │ 1 │ + 611. │ 610 │ UInt64 │ 0 │ + 612. │ 611 │ UInt64 │ 1 │ + 613. │ 612 │ UInt64 │ 0 │ + 614. │ 613 │ UInt64 │ 1 │ + 615. │ 614 │ UInt64 │ 0 │ + 616. │ 615 │ UInt64 │ 1 │ + 617. │ 616 │ UInt64 │ 0 │ + 618. │ 617 │ UInt64 │ 1 │ + 619. │ 618 │ UInt64 │ 0 │ + 620. │ 619 │ UInt64 │ 1 │ + 621. │ 620 │ UInt64 │ 0 │ + 622. │ 621 │ UInt64 │ 1 │ + 623. │ 622 │ UInt64 │ 0 │ + 624. │ 623 │ UInt64 │ 1 │ + 625. │ 624 │ UInt64 │ 0 │ + 626. │ 625 │ UInt64 │ 1 │ + 627. │ 626 │ UInt64 │ 0 │ + 628. │ 627 │ UInt64 │ 1 │ + 629. │ 628 │ UInt64 │ 0 │ + 630. │ 629 │ UInt64 │ 1 │ + 631. │ 630 │ UInt64 │ 0 │ + 632. │ 631 │ UInt64 │ 1 │ + 633. │ 632 │ UInt64 │ 0 │ + 634. │ 633 │ UInt64 │ 1 │ + 635. │ 634 │ UInt64 │ 0 │ + 636. │ 635 │ UInt64 │ 1 │ + 637. │ 636 │ UInt64 │ 0 │ + 638. │ 637 │ UInt64 │ 1 │ + 639. │ 638 │ UInt64 │ 0 │ + 640. │ 639 │ UInt64 │ 1 │ + 641. │ 640 │ UInt64 │ 0 │ + 642. │ 641 │ UInt64 │ 1 │ + 643. │ 642 │ UInt64 │ 0 │ + 644. │ 643 │ UInt64 │ 1 │ + 645. │ 644 │ UInt64 │ 0 │ + 646. │ 645 │ UInt64 │ 1 │ + 647. │ 646 │ UInt64 │ 0 │ + 648. │ 647 │ UInt64 │ 1 │ + 649. │ 648 │ UInt64 │ 0 │ + 650. │ 649 │ UInt64 │ 1 │ + 651. │ 650 │ UInt64 │ 0 │ + 652. │ 651 │ UInt64 │ 1 │ + 653. │ 652 │ UInt64 │ 0 │ + 654. │ 653 │ UInt64 │ 1 │ + 655. │ 654 │ UInt64 │ 0 │ + 656. │ 655 │ UInt64 │ 1 │ + 657. │ 656 │ UInt64 │ 0 │ + 658. │ 657 │ UInt64 │ 1 │ + 659. │ 658 │ UInt64 │ 0 │ + 660. │ 659 │ UInt64 │ 1 │ + 661. │ 660 │ UInt64 │ 0 │ + 662. │ 661 │ UInt64 │ 1 │ + 663. │ 662 │ UInt64 │ 0 │ + 664. │ 663 │ UInt64 │ 1 │ + 665. │ 664 │ UInt64 │ 0 │ + 666. │ 665 │ UInt64 │ 1 │ + 667. │ 666 │ UInt64 │ 0 │ + 668. │ 667 │ UInt64 │ 1 │ + 669. │ 668 │ UInt64 │ 0 │ + 670. │ 669 │ UInt64 │ 1 │ + 671. │ 670 │ UInt64 │ 0 │ + 672. │ 671 │ UInt64 │ 1 │ + 673. │ 672 │ UInt64 │ 0 │ + 674. │ 673 │ UInt64 │ 1 │ + 675. │ 674 │ UInt64 │ 0 │ + 676. │ 675 │ UInt64 │ 1 │ + 677. │ 676 │ UInt64 │ 0 │ + 678. │ 677 │ UInt64 │ 1 │ + 679. │ 678 │ UInt64 │ 0 │ + 680. │ 679 │ UInt64 │ 1 │ + 681. │ 680 │ UInt64 │ 0 │ + 682. │ 681 │ UInt64 │ 1 │ + 683. │ 682 │ UInt64 │ 0 │ + 684. │ 683 │ UInt64 │ 1 │ + 685. │ 684 │ UInt64 │ 0 │ + 686. │ 685 │ UInt64 │ 1 │ + 687. │ 686 │ UInt64 │ 0 │ + 688. │ 687 │ UInt64 │ 1 │ + 689. │ 688 │ UInt64 │ 0 │ + 690. │ 689 │ UInt64 │ 1 │ + 691. │ 690 │ UInt64 │ 0 │ + 692. │ 691 │ UInt64 │ 1 │ + 693. │ 692 │ UInt64 │ 0 │ + 694. │ 693 │ UInt64 │ 1 │ + 695. │ 694 │ UInt64 │ 0 │ + 696. │ 695 │ UInt64 │ 1 │ + 697. │ 696 │ UInt64 │ 0 │ + 698. │ 697 │ UInt64 │ 1 │ + 699. │ 698 │ UInt64 │ 0 │ + 700. │ 699 │ UInt64 │ 1 │ + 701. │ 700 │ UInt64 │ 0 │ + 702. │ 701 │ UInt64 │ 1 │ + 703. │ 702 │ UInt64 │ 0 │ + 704. │ 703 │ UInt64 │ 1 │ + 705. │ 704 │ UInt64 │ 0 │ + 706. │ 705 │ UInt64 │ 1 │ + 707. │ 706 │ UInt64 │ 0 │ + 708. │ 707 │ UInt64 │ 1 │ + 709. │ 708 │ UInt64 │ 0 │ + 710. │ 709 │ UInt64 │ 1 │ + 711. │ 710 │ UInt64 │ 0 │ + 712. │ 711 │ UInt64 │ 1 │ + 713. │ 712 │ UInt64 │ 0 │ + 714. │ 713 │ UInt64 │ 1 │ + 715. │ 714 │ UInt64 │ 0 │ + 716. │ 715 │ UInt64 │ 1 │ + 717. │ 716 │ UInt64 │ 0 │ + 718. │ 717 │ UInt64 │ 1 │ + 719. │ 718 │ UInt64 │ 0 │ + 720. │ 719 │ UInt64 │ 1 │ + 721. │ 720 │ UInt64 │ 0 │ + 722. │ 721 │ UInt64 │ 1 │ + 723. │ 722 │ UInt64 │ 0 │ + 724. │ 723 │ UInt64 │ 1 │ + 725. │ 724 │ UInt64 │ 0 │ + 726. │ 725 │ UInt64 │ 1 │ + 727. │ 726 │ UInt64 │ 0 │ + 728. │ 727 │ UInt64 │ 1 │ + 729. │ 728 │ UInt64 │ 0 │ + 730. │ 729 │ UInt64 │ 1 │ + 731. │ 730 │ UInt64 │ 0 │ + 732. │ 731 │ UInt64 │ 1 │ + 733. │ 732 │ UInt64 │ 0 │ + 734. │ 733 │ UInt64 │ 1 │ + 735. │ 734 │ UInt64 │ 0 │ + 736. │ 735 │ UInt64 │ 1 │ + 737. │ 736 │ UInt64 │ 0 │ + 738. │ 737 │ UInt64 │ 1 │ + 739. │ 738 │ UInt64 │ 0 │ + 740. │ 739 │ UInt64 │ 1 │ + 741. │ 740 │ UInt64 │ 0 │ + 742. │ 741 │ UInt64 │ 1 │ + 743. │ 742 │ UInt64 │ 0 │ + 744. │ 743 │ UInt64 │ 1 │ + 745. │ 744 │ UInt64 │ 0 │ + 746. │ 745 │ UInt64 │ 1 │ + 747. │ 746 │ UInt64 │ 0 │ + 748. │ 747 │ UInt64 │ 1 │ + 749. │ 748 │ UInt64 │ 0 │ + 750. │ 749 │ UInt64 │ 1 │ + 751. │ 750 │ UInt64 │ 0 │ + 752. │ 751 │ UInt64 │ 1 │ + 753. │ 752 │ UInt64 │ 0 │ + 754. │ 753 │ UInt64 │ 1 │ + 755. │ 754 │ UInt64 │ 0 │ + 756. │ 755 │ UInt64 │ 1 │ + 757. │ 756 │ UInt64 │ 0 │ + 758. │ 757 │ UInt64 │ 1 │ + 759. │ 758 │ UInt64 │ 0 │ + 760. │ 759 │ UInt64 │ 1 │ + 761. │ 760 │ UInt64 │ 0 │ + 762. │ 761 │ UInt64 │ 1 │ + 763. │ 762 │ UInt64 │ 0 │ + 764. │ 763 │ UInt64 │ 1 │ + 765. │ 764 │ UInt64 │ 0 │ + 766. │ 765 │ UInt64 │ 1 │ + 767. │ 766 │ UInt64 │ 0 │ + 768. │ 767 │ UInt64 │ 1 │ + 769. │ 768 │ UInt64 │ 0 │ + 770. │ 769 │ UInt64 │ 1 │ + 771. │ 770 │ UInt64 │ 0 │ + 772. │ 771 │ UInt64 │ 1 │ + 773. │ 772 │ UInt64 │ 0 │ + 774. │ 773 │ UInt64 │ 1 │ + 775. │ 774 │ UInt64 │ 0 │ + 776. │ 775 │ UInt64 │ 1 │ + 777. │ 776 │ UInt64 │ 0 │ + 778. │ 777 │ UInt64 │ 1 │ + 779. │ 778 │ UInt64 │ 0 │ + 780. │ 779 │ UInt64 │ 1 │ + 781. │ 780 │ UInt64 │ 0 │ + 782. │ 781 │ UInt64 │ 1 │ + 783. │ 782 │ UInt64 │ 0 │ + 784. │ 783 │ UInt64 │ 1 │ + 785. │ 784 │ UInt64 │ 0 │ + 786. │ 785 │ UInt64 │ 1 │ + 787. │ 786 │ UInt64 │ 0 │ + 788. │ 787 │ UInt64 │ 1 │ + 789. │ 788 │ UInt64 │ 0 │ + 790. │ 789 │ UInt64 │ 1 │ + 791. │ 790 │ UInt64 │ 0 │ + 792. │ 791 │ UInt64 │ 1 │ + 793. │ 792 │ UInt64 │ 0 │ + 794. │ 793 │ UInt64 │ 1 │ + 795. │ 794 │ UInt64 │ 0 │ + 796. │ 795 │ UInt64 │ 1 │ + 797. │ 796 │ UInt64 │ 0 │ + 798. │ 797 │ UInt64 │ 1 │ + 799. │ 798 │ UInt64 │ 0 │ + 800. │ 799 │ UInt64 │ 1 │ + 801. │ 800 │ UInt64 │ 0 │ + 802. │ 801 │ UInt64 │ 1 │ + 803. │ 802 │ UInt64 │ 0 │ + 804. │ 803 │ UInt64 │ 1 │ + 805. │ 804 │ UInt64 │ 0 │ + 806. │ 805 │ UInt64 │ 1 │ + 807. │ 806 │ UInt64 │ 0 │ + 808. │ 807 │ UInt64 │ 1 │ + 809. │ 808 │ UInt64 │ 0 │ + 810. │ 809 │ UInt64 │ 1 │ + 811. │ 810 │ UInt64 │ 0 │ + 812. │ 811 │ UInt64 │ 1 │ + 813. │ 812 │ UInt64 │ 0 │ + 814. │ 813 │ UInt64 │ 1 │ + 815. │ 814 │ UInt64 │ 0 │ + 816. │ 815 │ UInt64 │ 1 │ + 817. │ 816 │ UInt64 │ 0 │ + 818. │ 817 │ UInt64 │ 1 │ + 819. │ 818 │ UInt64 │ 0 │ + 820. │ 819 │ UInt64 │ 1 │ + 821. │ 820 │ UInt64 │ 0 │ + 822. │ 821 │ UInt64 │ 1 │ + 823. │ 822 │ UInt64 │ 0 │ + 824. │ 823 │ UInt64 │ 1 │ + 825. │ 824 │ UInt64 │ 0 │ + 826. │ 825 │ UInt64 │ 1 │ + 827. │ 826 │ UInt64 │ 0 │ + 828. │ 827 │ UInt64 │ 1 │ + 829. │ 828 │ UInt64 │ 0 │ + 830. │ 829 │ UInt64 │ 1 │ + 831. │ 830 │ UInt64 │ 0 │ + 832. │ 831 │ UInt64 │ 1 │ + 833. │ 832 │ UInt64 │ 0 │ + 834. │ 833 │ UInt64 │ 1 │ + 835. │ 834 │ UInt64 │ 0 │ + 836. │ 835 │ UInt64 │ 1 │ + 837. │ 836 │ UInt64 │ 0 │ + 838. │ 837 │ UInt64 │ 1 │ + 839. │ 838 │ UInt64 │ 0 │ + 840. │ 839 │ UInt64 │ 1 │ + 841. │ 840 │ UInt64 │ 0 │ + 842. │ 841 │ UInt64 │ 1 │ + 843. │ 842 │ UInt64 │ 0 │ + 844. │ 843 │ UInt64 │ 1 │ + 845. │ 844 │ UInt64 │ 0 │ + 846. │ 845 │ UInt64 │ 1 │ + 847. │ 846 │ UInt64 │ 0 │ + 848. │ 847 │ UInt64 │ 1 │ + 849. │ 848 │ UInt64 │ 0 │ + 850. │ 849 │ UInt64 │ 1 │ + 851. │ 850 │ UInt64 │ 0 │ + 852. │ 851 │ UInt64 │ 1 │ + 853. │ 852 │ UInt64 │ 0 │ + 854. │ 853 │ UInt64 │ 1 │ + 855. │ 854 │ UInt64 │ 0 │ + 856. │ 855 │ UInt64 │ 1 │ + 857. │ 856 │ UInt64 │ 0 │ + 858. │ 857 │ UInt64 │ 1 │ + 859. │ 858 │ UInt64 │ 0 │ + 860. │ 859 │ UInt64 │ 1 │ + 861. │ 860 │ UInt64 │ 0 │ + 862. │ 861 │ UInt64 │ 1 │ + 863. │ 862 │ UInt64 │ 0 │ + 864. │ 863 │ UInt64 │ 1 │ + 865. │ 864 │ UInt64 │ 0 │ + 866. │ 865 │ UInt64 │ 1 │ + 867. │ 866 │ UInt64 │ 0 │ + 868. │ 867 │ UInt64 │ 1 │ + 869. │ 868 │ UInt64 │ 0 │ + 870. │ 869 │ UInt64 │ 1 │ + 871. │ 870 │ UInt64 │ 0 │ + 872. │ 871 │ UInt64 │ 1 │ + 873. │ 872 │ UInt64 │ 0 │ + 874. │ 873 │ UInt64 │ 1 │ + 875. │ 874 │ UInt64 │ 0 │ + 876. │ 875 │ UInt64 │ 1 │ + 877. │ 876 │ UInt64 │ 0 │ + 878. │ 877 │ UInt64 │ 1 │ + 879. │ 878 │ UInt64 │ 0 │ + 880. │ 879 │ UInt64 │ 1 │ + 881. │ 880 │ UInt64 │ 0 │ + 882. │ 881 │ UInt64 │ 1 │ + 883. │ 882 │ UInt64 │ 0 │ + 884. │ 883 │ UInt64 │ 1 │ + 885. │ 884 │ UInt64 │ 0 │ + 886. │ 885 │ UInt64 │ 1 │ + 887. │ 886 │ UInt64 │ 0 │ + 888. │ 887 │ UInt64 │ 1 │ + 889. │ 888 │ UInt64 │ 0 │ + 890. │ 889 │ UInt64 │ 1 │ + 891. │ 890 │ UInt64 │ 0 │ + 892. │ 891 │ UInt64 │ 1 │ + 893. │ 892 │ UInt64 │ 0 │ + 894. │ 893 │ UInt64 │ 1 │ + 895. │ 894 │ UInt64 │ 0 │ + 896. │ 895 │ UInt64 │ 1 │ + 897. │ 896 │ UInt64 │ 0 │ + 898. │ 897 │ UInt64 │ 1 │ + 899. │ 898 │ UInt64 │ 0 │ + 900. │ 899 │ UInt64 │ 1 │ + 901. │ 900 │ UInt64 │ 0 │ + 902. │ 901 │ UInt64 │ 1 │ + 903. │ 902 │ UInt64 │ 0 │ + 904. │ 903 │ UInt64 │ 1 │ + 905. │ 904 │ UInt64 │ 0 │ + 906. │ 905 │ UInt64 │ 1 │ + 907. │ 906 │ UInt64 │ 0 │ + 908. │ 907 │ UInt64 │ 1 │ + 909. │ 908 │ UInt64 │ 0 │ + 910. │ 909 │ UInt64 │ 1 │ + 911. │ 910 │ UInt64 │ 0 │ + 912. │ 911 │ UInt64 │ 1 │ + 913. │ 912 │ UInt64 │ 0 │ + 914. │ 913 │ UInt64 │ 1 │ + 915. │ 914 │ UInt64 │ 0 │ + 916. │ 915 │ UInt64 │ 1 │ + 917. │ 916 │ UInt64 │ 0 │ + 918. │ 917 │ UInt64 │ 1 │ + 919. │ 918 │ UInt64 │ 0 │ + 920. │ 919 │ UInt64 │ 1 │ + 921. │ 920 │ UInt64 │ 0 │ + 922. │ 921 │ UInt64 │ 1 │ + 923. │ 922 │ UInt64 │ 0 │ + 924. │ 923 │ UInt64 │ 1 │ + 925. │ 924 │ UInt64 │ 0 │ + 926. │ 925 │ UInt64 │ 1 │ + 927. │ 926 │ UInt64 │ 0 │ + 928. │ 927 │ UInt64 │ 1 │ + 929. │ 928 │ UInt64 │ 0 │ + 930. │ 929 │ UInt64 │ 1 │ + 931. │ 930 │ UInt64 │ 0 │ + 932. │ 931 │ UInt64 │ 1 │ + 933. │ 932 │ UInt64 │ 0 │ + 934. │ 933 │ UInt64 │ 1 │ + 935. │ 934 │ UInt64 │ 0 │ + 936. │ 935 │ UInt64 │ 1 │ + 937. │ 936 │ UInt64 │ 0 │ + 938. │ 937 │ UInt64 │ 1 │ + 939. │ 938 │ UInt64 │ 0 │ + 940. │ 939 │ UInt64 │ 1 │ + 941. │ 940 │ UInt64 │ 0 │ + 942. │ 941 │ UInt64 │ 1 │ + 943. │ 942 │ UInt64 │ 0 │ + 944. │ 943 │ UInt64 │ 1 │ + 945. │ 944 │ UInt64 │ 0 │ + 946. │ 945 │ UInt64 │ 1 │ + 947. │ 946 │ UInt64 │ 0 │ + 948. │ 947 │ UInt64 │ 1 │ + 949. │ 948 │ UInt64 │ 0 │ + 950. │ 949 │ UInt64 │ 1 │ + 951. │ 950 │ UInt64 │ 0 │ + 952. │ 951 │ UInt64 │ 1 │ + 953. │ 952 │ UInt64 │ 0 │ + 954. │ 953 │ UInt64 │ 1 │ + 955. │ 954 │ UInt64 │ 0 │ + 956. │ 955 │ UInt64 │ 1 │ + 957. │ 956 │ UInt64 │ 0 │ + 958. │ 957 │ UInt64 │ 1 │ + 959. │ 958 │ UInt64 │ 0 │ + 960. │ 959 │ UInt64 │ 1 │ + 961. │ 960 │ UInt64 │ 0 │ + 962. │ 961 │ UInt64 │ 1 │ + 963. │ 962 │ UInt64 │ 0 │ + 964. │ 963 │ UInt64 │ 1 │ + 965. │ 964 │ UInt64 │ 0 │ + 966. │ 965 │ UInt64 │ 1 │ + 967. │ 966 │ UInt64 │ 0 │ + 968. │ 967 │ UInt64 │ 1 │ + 969. │ 968 │ UInt64 │ 0 │ + 970. │ 969 │ UInt64 │ 1 │ + 971. │ 970 │ UInt64 │ 0 │ + 972. │ 971 │ UInt64 │ 1 │ + 973. │ 972 │ UInt64 │ 0 │ + 974. │ 973 │ UInt64 │ 1 │ + 975. │ 974 │ UInt64 │ 0 │ + 976. │ 975 │ UInt64 │ 1 │ + 977. │ 976 │ UInt64 │ 0 │ + 978. │ 977 │ UInt64 │ 1 │ + 979. │ 978 │ UInt64 │ 0 │ + 980. │ 979 │ UInt64 │ 1 │ + 981. │ 980 │ UInt64 │ 0 │ + 982. │ 981 │ UInt64 │ 1 │ + 983. │ 982 │ UInt64 │ 0 │ + 984. │ 983 │ UInt64 │ 1 │ + 985. │ 984 │ UInt64 │ 0 │ + 986. │ 985 │ UInt64 │ 1 │ + 987. │ 986 │ UInt64 │ 0 │ + 988. │ 987 │ UInt64 │ 1 │ + 989. │ 988 │ UInt64 │ 0 │ + 990. │ 989 │ UInt64 │ 1 │ + 991. │ 990 │ UInt64 │ 0 │ + 992. │ 991 │ UInt64 │ 1 │ + 993. │ 992 │ UInt64 │ 0 │ + 994. │ 993 │ UInt64 │ 1 │ + 995. │ 994 │ UInt64 │ 0 │ + 996. │ 995 │ UInt64 │ 1 │ + 997. │ 996 │ UInt64 │ 0 │ + 998. │ 997 │ UInt64 │ 1 │ + 999. │ 998 │ UInt64 │ 0 │ +1000. │ 999 │ UInt64 │ 1 │ + └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ + 100. │ 99 │ UInt64 │ 1 │ + 101. │ 100 │ UInt64 │ 0 │ + 102. │ 101 │ UInt64 │ 1 │ + 103. │ 102 │ UInt64 │ 0 │ + 104. │ 103 │ UInt64 │ 1 │ + 105. │ 104 │ UInt64 │ 0 │ + 106. │ 105 │ UInt64 │ 1 │ + 107. │ 106 │ UInt64 │ 0 │ + 108. │ 107 │ UInt64 │ 1 │ + 109. │ 108 │ UInt64 │ 0 │ + 110. │ 109 │ UInt64 │ 1 │ + 111. │ 110 │ UInt64 │ 0 │ + 112. │ 111 │ UInt64 │ 1 │ + 113. │ 112 │ UInt64 │ 0 │ + 114. │ 113 │ UInt64 │ 1 │ + 115. │ 114 │ UInt64 │ 0 │ + 116. │ 115 │ UInt64 │ 1 │ + 117. │ 116 │ UInt64 │ 0 │ + 118. │ 117 │ UInt64 │ 1 │ + 119. │ 118 │ UInt64 │ 0 │ + 120. │ 119 │ UInt64 │ 1 │ + 121. │ 120 │ UInt64 │ 0 │ + 122. │ 121 │ UInt64 │ 1 │ + 123. │ 122 │ UInt64 │ 0 │ + 124. │ 123 │ UInt64 │ 1 │ + 125. │ 124 │ UInt64 │ 0 │ + 126. │ 125 │ UInt64 │ 1 │ + 127. │ 126 │ UInt64 │ 0 │ + 128. │ 127 │ UInt64 │ 1 │ + 129. │ 128 │ UInt64 │ 0 │ + 130. │ 129 │ UInt64 │ 1 │ + 131. │ 130 │ UInt64 │ 0 │ + 132. │ 131 │ UInt64 │ 1 │ + 133. │ 132 │ UInt64 │ 0 │ + 134. │ 133 │ UInt64 │ 1 │ + 135. │ 134 │ UInt64 │ 0 │ + 136. │ 135 │ UInt64 │ 1 │ + 137. │ 136 │ UInt64 │ 0 │ + 138. │ 137 │ UInt64 │ 1 │ + 139. │ 138 │ UInt64 │ 0 │ + 140. │ 139 │ UInt64 │ 1 │ + 141. │ 140 │ UInt64 │ 0 │ + 142. │ 141 │ UInt64 │ 1 │ + 143. │ 142 │ UInt64 │ 0 │ + 144. │ 143 │ UInt64 │ 1 │ + 145. │ 144 │ UInt64 │ 0 │ + 146. │ 145 │ UInt64 │ 1 │ + 147. │ 146 │ UInt64 │ 0 │ + 148. │ 147 │ UInt64 │ 1 │ + 149. │ 148 │ UInt64 │ 0 │ + 150. │ 149 │ UInt64 │ 1 │ + 151. │ 150 │ UInt64 │ 0 │ + 152. │ 151 │ UInt64 │ 1 │ + 153. │ 152 │ UInt64 │ 0 │ + 154. │ 153 │ UInt64 │ 1 │ + 155. │ 154 │ UInt64 │ 0 │ + 156. │ 155 │ UInt64 │ 1 │ + 157. │ 156 │ UInt64 │ 0 │ + 158. │ 157 │ UInt64 │ 1 │ + 159. │ 158 │ UInt64 │ 0 │ + 160. │ 159 │ UInt64 │ 1 │ + 161. │ 160 │ UInt64 │ 0 │ + 162. │ 161 │ UInt64 │ 1 │ + 163. │ 162 │ UInt64 │ 0 │ + 164. │ 163 │ UInt64 │ 1 │ + 165. │ 164 │ UInt64 │ 0 │ + 166. │ 165 │ UInt64 │ 1 │ + 167. │ 166 │ UInt64 │ 0 │ + 168. │ 167 │ UInt64 │ 1 │ + 169. │ 168 │ UInt64 │ 0 │ + 170. │ 169 │ UInt64 │ 1 │ + 171. │ 170 │ UInt64 │ 0 │ + 172. │ 171 │ UInt64 │ 1 │ + 173. │ 172 │ UInt64 │ 0 │ + 174. │ 173 │ UInt64 │ 1 │ + 175. │ 174 │ UInt64 │ 0 │ + 176. │ 175 │ UInt64 │ 1 │ + 177. │ 176 │ UInt64 │ 0 │ + 178. │ 177 │ UInt64 │ 1 │ + 179. │ 178 │ UInt64 │ 0 │ + 180. │ 179 │ UInt64 │ 1 │ + 181. │ 180 │ UInt64 │ 0 │ + 182. │ 181 │ UInt64 │ 1 │ + 183. │ 182 │ UInt64 │ 0 │ + 184. │ 183 │ UInt64 │ 1 │ + 185. │ 184 │ UInt64 │ 0 │ + 186. │ 185 │ UInt64 │ 1 │ + 187. │ 186 │ UInt64 │ 0 │ + 188. │ 187 │ UInt64 │ 1 │ + 189. │ 188 │ UInt64 │ 0 │ + 190. │ 189 │ UInt64 │ 1 │ + 191. │ 190 │ UInt64 │ 0 │ + 192. │ 191 │ UInt64 │ 1 │ + 193. │ 192 │ UInt64 │ 0 │ + 194. │ 193 │ UInt64 │ 1 │ + 195. │ 194 │ UInt64 │ 0 │ + 196. │ 195 │ UInt64 │ 1 │ + 197. │ 196 │ UInt64 │ 0 │ + 198. │ 197 │ UInt64 │ 1 │ + 199. │ 198 │ UInt64 │ 0 │ + 200. │ 199 │ UInt64 │ 1 │ + 201. │ 200 │ UInt64 │ 0 │ + 202. │ 201 │ UInt64 │ 1 │ + 203. │ 202 │ UInt64 │ 0 │ + 204. │ 203 │ UInt64 │ 1 │ + 205. │ 204 │ UInt64 │ 0 │ + 206. │ 205 │ UInt64 │ 1 │ + 207. │ 206 │ UInt64 │ 0 │ + 208. │ 207 │ UInt64 │ 1 │ + 209. │ 208 │ UInt64 │ 0 │ + 210. │ 209 │ UInt64 │ 1 │ + 211. │ 210 │ UInt64 │ 0 │ + 212. │ 211 │ UInt64 │ 1 │ + 213. │ 212 │ UInt64 │ 0 │ + 214. │ 213 │ UInt64 │ 1 │ + 215. │ 214 │ UInt64 │ 0 │ + 216. │ 215 │ UInt64 │ 1 │ + 217. │ 216 │ UInt64 │ 0 │ + 218. │ 217 │ UInt64 │ 1 │ + 219. │ 218 │ UInt64 │ 0 │ + 220. │ 219 │ UInt64 │ 1 │ + 221. │ 220 │ UInt64 │ 0 │ + 222. │ 221 │ UInt64 │ 1 │ + 223. │ 222 │ UInt64 │ 0 │ + 224. │ 223 │ UInt64 │ 1 │ + 225. │ 224 │ UInt64 │ 0 │ + 226. │ 225 │ UInt64 │ 1 │ + 227. │ 226 │ UInt64 │ 0 │ + 228. │ 227 │ UInt64 │ 1 │ + 229. │ 228 │ UInt64 │ 0 │ + 230. │ 229 │ UInt64 │ 1 │ + 231. │ 230 │ UInt64 │ 0 │ + 232. │ 231 │ UInt64 │ 1 │ + 233. │ 232 │ UInt64 │ 0 │ + 234. │ 233 │ UInt64 │ 1 │ + 235. │ 234 │ UInt64 │ 0 │ + 236. │ 235 │ UInt64 │ 1 │ + 237. │ 236 │ UInt64 │ 0 │ + 238. │ 237 │ UInt64 │ 1 │ + 239. │ 238 │ UInt64 │ 0 │ + 240. │ 239 │ UInt64 │ 1 │ + 241. │ 240 │ UInt64 │ 0 │ + 242. │ 241 │ UInt64 │ 1 │ + 243. │ 242 │ UInt64 │ 0 │ + 244. │ 243 │ UInt64 │ 1 │ + 245. │ 244 │ UInt64 │ 0 │ + 246. │ 245 │ UInt64 │ 1 │ + 247. │ 246 │ UInt64 │ 0 │ + 248. │ 247 │ UInt64 │ 1 │ + 249. │ 248 │ UInt64 │ 0 │ + 250. │ 249 │ UInt64 │ 1 │ + 251. │ 250 │ UInt64 │ 0 │ + 252. │ 251 │ UInt64 │ 1 │ + 253. │ 252 │ UInt64 │ 0 │ + 254. │ 253 │ UInt64 │ 1 │ + 255. │ 254 │ UInt64 │ 0 │ + 256. │ 255 │ UInt64 │ 1 │ + 257. │ 256 │ UInt64 │ 0 │ + 258. │ 257 │ UInt64 │ 1 │ + 259. │ 258 │ UInt64 │ 0 │ + 260. │ 259 │ UInt64 │ 1 │ + 261. │ 260 │ UInt64 │ 0 │ + 262. │ 261 │ UInt64 │ 1 │ + 263. │ 262 │ UInt64 │ 0 │ + 264. │ 263 │ UInt64 │ 1 │ + 265. │ 264 │ UInt64 │ 0 │ + 266. │ 265 │ UInt64 │ 1 │ + 267. │ 266 │ UInt64 │ 0 │ + 268. │ 267 │ UInt64 │ 1 │ + 269. │ 268 │ UInt64 │ 0 │ + 270. │ 269 │ UInt64 │ 1 │ + 271. │ 270 │ UInt64 │ 0 │ + 272. │ 271 │ UInt64 │ 1 │ + 273. │ 272 │ UInt64 │ 0 │ + 274. │ 273 │ UInt64 │ 1 │ + 275. │ 274 │ UInt64 │ 0 │ + 276. │ 275 │ UInt64 │ 1 │ + 277. │ 276 │ UInt64 │ 0 │ + 278. │ 277 │ UInt64 │ 1 │ + 279. │ 278 │ UInt64 │ 0 │ + 280. │ 279 │ UInt64 │ 1 │ + 281. │ 280 │ UInt64 │ 0 │ + 282. │ 281 │ UInt64 │ 1 │ + 283. │ 282 │ UInt64 │ 0 │ + 284. │ 283 │ UInt64 │ 1 │ + 285. │ 284 │ UInt64 │ 0 │ + 286. │ 285 │ UInt64 │ 1 │ + 287. │ 286 │ UInt64 │ 0 │ + 288. │ 287 │ UInt64 │ 1 │ + 289. │ 288 │ UInt64 │ 0 │ + 290. │ 289 │ UInt64 │ 1 │ + 291. │ 290 │ UInt64 │ 0 │ + 292. │ 291 │ UInt64 │ 1 │ + 293. │ 292 │ UInt64 │ 0 │ + 294. │ 293 │ UInt64 │ 1 │ + 295. │ 294 │ UInt64 │ 0 │ + 296. │ 295 │ UInt64 │ 1 │ + 297. │ 296 │ UInt64 │ 0 │ + 298. │ 297 │ UInt64 │ 1 │ + 299. │ 298 │ UInt64 │ 0 │ + 300. │ 299 │ UInt64 │ 1 │ + 301. │ 300 │ UInt64 │ 0 │ + 302. │ 301 │ UInt64 │ 1 │ + 303. │ 302 │ UInt64 │ 0 │ + 304. │ 303 │ UInt64 │ 1 │ + 305. │ 304 │ UInt64 │ 0 │ + 306. │ 305 │ UInt64 │ 1 │ + 307. │ 306 │ UInt64 │ 0 │ + 308. │ 307 │ UInt64 │ 1 │ + 309. │ 308 │ UInt64 │ 0 │ + 310. │ 309 │ UInt64 │ 1 │ + 311. │ 310 │ UInt64 │ 0 │ + 312. │ 311 │ UInt64 │ 1 │ + 313. │ 312 │ UInt64 │ 0 │ + 314. │ 313 │ UInt64 │ 1 │ + 315. │ 314 │ UInt64 │ 0 │ + 316. │ 315 │ UInt64 │ 1 │ + 317. │ 316 │ UInt64 │ 0 │ + 318. │ 317 │ UInt64 │ 1 │ + 319. │ 318 │ UInt64 │ 0 │ + 320. │ 319 │ UInt64 │ 1 │ + 321. │ 320 │ UInt64 │ 0 │ + 322. │ 321 │ UInt64 │ 1 │ + 323. │ 322 │ UInt64 │ 0 │ + 324. │ 323 │ UInt64 │ 1 │ + 325. │ 324 │ UInt64 │ 0 │ + 326. │ 325 │ UInt64 │ 1 │ + 327. │ 326 │ UInt64 │ 0 │ + 328. │ 327 │ UInt64 │ 1 │ + 329. │ 328 │ UInt64 │ 0 │ + 330. │ 329 │ UInt64 │ 1 │ + 331. │ 330 │ UInt64 │ 0 │ + 332. │ 331 │ UInt64 │ 1 │ + 333. │ 332 │ UInt64 │ 0 │ + 334. │ 333 │ UInt64 │ 1 │ + 335. │ 334 │ UInt64 │ 0 │ + 336. │ 335 │ UInt64 │ 1 │ + 337. │ 336 │ UInt64 │ 0 │ + 338. │ 337 │ UInt64 │ 1 │ + 339. │ 338 │ UInt64 │ 0 │ + 340. │ 339 │ UInt64 │ 1 │ + 341. │ 340 │ UInt64 │ 0 │ + 342. │ 341 │ UInt64 │ 1 │ + 343. │ 342 │ UInt64 │ 0 │ + 344. │ 343 │ UInt64 │ 1 │ + 345. │ 344 │ UInt64 │ 0 │ + 346. │ 345 │ UInt64 │ 1 │ + 347. │ 346 │ UInt64 │ 0 │ + 348. │ 347 │ UInt64 │ 1 │ + 349. │ 348 │ UInt64 │ 0 │ + 350. │ 349 │ UInt64 │ 1 │ + 351. │ 350 │ UInt64 │ 0 │ + 352. │ 351 │ UInt64 │ 1 │ + 353. │ 352 │ UInt64 │ 0 │ + 354. │ 353 │ UInt64 │ 1 │ + 355. │ 354 │ UInt64 │ 0 │ + 356. │ 355 │ UInt64 │ 1 │ + 357. │ 356 │ UInt64 │ 0 │ + 358. │ 357 │ UInt64 │ 1 │ + 359. │ 358 │ UInt64 │ 0 │ + 360. │ 359 │ UInt64 │ 1 │ + 361. │ 360 │ UInt64 │ 0 │ + 362. │ 361 │ UInt64 │ 1 │ + 363. │ 362 │ UInt64 │ 0 │ + 364. │ 363 │ UInt64 │ 1 │ + 365. │ 364 │ UInt64 │ 0 │ + 366. │ 365 │ UInt64 │ 1 │ + 367. │ 366 │ UInt64 │ 0 │ + 368. │ 367 │ UInt64 │ 1 │ + 369. │ 368 │ UInt64 │ 0 │ + 370. │ 369 │ UInt64 │ 1 │ + 371. │ 370 │ UInt64 │ 0 │ + 372. │ 371 │ UInt64 │ 1 │ + 373. │ 372 │ UInt64 │ 0 │ + 374. │ 373 │ UInt64 │ 1 │ + 375. │ 374 │ UInt64 │ 0 │ + 376. │ 375 │ UInt64 │ 1 │ + 377. │ 376 │ UInt64 │ 0 │ + 378. │ 377 │ UInt64 │ 1 │ + 379. │ 378 │ UInt64 │ 0 │ + 380. │ 379 │ UInt64 │ 1 │ + 381. │ 380 │ UInt64 │ 0 │ + 382. │ 381 │ UInt64 │ 1 │ + 383. │ 382 │ UInt64 │ 0 │ + 384. │ 383 │ UInt64 │ 1 │ + 385. │ 384 │ UInt64 │ 0 │ + 386. │ 385 │ UInt64 │ 1 │ + 387. │ 386 │ UInt64 │ 0 │ + 388. │ 387 │ UInt64 │ 1 │ + 389. │ 388 │ UInt64 │ 0 │ + 390. │ 389 │ UInt64 │ 1 │ + 391. │ 390 │ UInt64 │ 0 │ + 392. │ 391 │ UInt64 │ 1 │ + 393. │ 392 │ UInt64 │ 0 │ + 394. │ 393 │ UInt64 │ 1 │ + 395. │ 394 │ UInt64 │ 0 │ + 396. │ 395 │ UInt64 │ 1 │ + 397. │ 396 │ UInt64 │ 0 │ + 398. │ 397 │ UInt64 │ 1 │ + 399. │ 398 │ UInt64 │ 0 │ + 400. │ 399 │ UInt64 │ 1 │ + 401. │ 400 │ UInt64 │ 0 │ + 402. │ 401 │ UInt64 │ 1 │ + 403. │ 402 │ UInt64 │ 0 │ + 404. │ 403 │ UInt64 │ 1 │ + 405. │ 404 │ UInt64 │ 0 │ + 406. │ 405 │ UInt64 │ 1 │ + 407. │ 406 │ UInt64 │ 0 │ + 408. │ 407 │ UInt64 │ 1 │ + 409. │ 408 │ UInt64 │ 0 │ + 410. │ 409 │ UInt64 │ 1 │ + 411. │ 410 │ UInt64 │ 0 │ + 412. │ 411 │ UInt64 │ 1 │ + 413. │ 412 │ UInt64 │ 0 │ + 414. │ 413 │ UInt64 │ 1 │ + 415. │ 414 │ UInt64 │ 0 │ + 416. │ 415 │ UInt64 │ 1 │ + 417. │ 416 │ UInt64 │ 0 │ + 418. │ 417 │ UInt64 │ 1 │ + 419. │ 418 │ UInt64 │ 0 │ + 420. │ 419 │ UInt64 │ 1 │ + 421. │ 420 │ UInt64 │ 0 │ + 422. │ 421 │ UInt64 │ 1 │ + 423. │ 422 │ UInt64 │ 0 │ + 424. │ 423 │ UInt64 │ 1 │ + 425. │ 424 │ UInt64 │ 0 │ + 426. │ 425 │ UInt64 │ 1 │ + 427. │ 426 │ UInt64 │ 0 │ + 428. │ 427 │ UInt64 │ 1 │ + 429. │ 428 │ UInt64 │ 0 │ + 430. │ 429 │ UInt64 │ 1 │ + 431. │ 430 │ UInt64 │ 0 │ + 432. │ 431 │ UInt64 │ 1 │ + 433. │ 432 │ UInt64 │ 0 │ + 434. │ 433 │ UInt64 │ 1 │ + 435. │ 434 │ UInt64 │ 0 │ + 436. │ 435 │ UInt64 │ 1 │ + 437. │ 436 │ UInt64 │ 0 │ + 438. │ 437 │ UInt64 │ 1 │ + 439. │ 438 │ UInt64 │ 0 │ + 440. │ 439 │ UInt64 │ 1 │ + 441. │ 440 │ UInt64 │ 0 │ + 442. │ 441 │ UInt64 │ 1 │ + 443. │ 442 │ UInt64 │ 0 │ + 444. │ 443 │ UInt64 │ 1 │ + 445. │ 444 │ UInt64 │ 0 │ + 446. │ 445 │ UInt64 │ 1 │ + 447. │ 446 │ UInt64 │ 0 │ + 448. │ 447 │ UInt64 │ 1 │ + 449. │ 448 │ UInt64 │ 0 │ + 450. │ 449 │ UInt64 │ 1 │ + 451. │ 450 │ UInt64 │ 0 │ + 452. │ 451 │ UInt64 │ 1 │ + 453. │ 452 │ UInt64 │ 0 │ + 454. │ 453 │ UInt64 │ 1 │ + 455. │ 454 │ UInt64 │ 0 │ + 456. │ 455 │ UInt64 │ 1 │ + 457. │ 456 │ UInt64 │ 0 │ + 458. │ 457 │ UInt64 │ 1 │ + 459. │ 458 │ UInt64 │ 0 │ + 460. │ 459 │ UInt64 │ 1 │ + 461. │ 460 │ UInt64 │ 0 │ + 462. │ 461 │ UInt64 │ 1 │ + 463. │ 462 │ UInt64 │ 0 │ + 464. │ 463 │ UInt64 │ 1 │ + 465. │ 464 │ UInt64 │ 0 │ + 466. │ 465 │ UInt64 │ 1 │ + 467. │ 466 │ UInt64 │ 0 │ + 468. │ 467 │ UInt64 │ 1 │ + 469. │ 468 │ UInt64 │ 0 │ + 470. │ 469 │ UInt64 │ 1 │ + 471. │ 470 │ UInt64 │ 0 │ + 472. │ 471 │ UInt64 │ 1 │ + 473. │ 472 │ UInt64 │ 0 │ + 474. │ 473 │ UInt64 │ 1 │ + 475. │ 474 │ UInt64 │ 0 │ + 476. │ 475 │ UInt64 │ 1 │ + 477. │ 476 │ UInt64 │ 0 │ + 478. │ 477 │ UInt64 │ 1 │ + 479. │ 478 │ UInt64 │ 0 │ + 480. │ 479 │ UInt64 │ 1 │ + 481. │ 480 │ UInt64 │ 0 │ + 482. │ 481 │ UInt64 │ 1 │ + 483. │ 482 │ UInt64 │ 0 │ + 484. │ 483 │ UInt64 │ 1 │ + 485. │ 484 │ UInt64 │ 0 │ + 486. │ 485 │ UInt64 │ 1 │ + 487. │ 486 │ UInt64 │ 0 │ + 488. │ 487 │ UInt64 │ 1 │ + 489. │ 488 │ UInt64 │ 0 │ + 490. │ 489 │ UInt64 │ 1 │ + 491. │ 490 │ UInt64 │ 0 │ + 492. │ 491 │ UInt64 │ 1 │ + 493. │ 492 │ UInt64 │ 0 │ + 494. │ 493 │ UInt64 │ 1 │ + 495. │ 494 │ UInt64 │ 0 │ + 496. │ 495 │ UInt64 │ 1 │ + 497. │ 496 │ UInt64 │ 0 │ + 498. │ 497 │ UInt64 │ 1 │ + 499. │ 498 │ UInt64 │ 0 │ + 500. │ 499 │ UInt64 │ 1 │ + 501. │ 500 │ UInt64 │ 0 │ + 502. │ 501 │ UInt64 │ 1 │ + 503. │ 502 │ UInt64 │ 0 │ + 504. │ 503 │ UInt64 │ 1 │ + 505. │ 504 │ UInt64 │ 0 │ + 506. │ 505 │ UInt64 │ 1 │ + 507. │ 506 │ UInt64 │ 0 │ + 508. │ 507 │ UInt64 │ 1 │ + 509. │ 508 │ UInt64 │ 0 │ + 510. │ 509 │ UInt64 │ 1 │ + 511. │ 510 │ UInt64 │ 0 │ + 512. │ 511 │ UInt64 │ 1 │ + 513. │ 512 │ UInt64 │ 0 │ + 514. │ 513 │ UInt64 │ 1 │ + 515. │ 514 │ UInt64 │ 0 │ + 516. │ 515 │ UInt64 │ 1 │ + 517. │ 516 │ UInt64 │ 0 │ + 518. │ 517 │ UInt64 │ 1 │ + 519. │ 518 │ UInt64 │ 0 │ + 520. │ 519 │ UInt64 │ 1 │ + 521. │ 520 │ UInt64 │ 0 │ + 522. │ 521 │ UInt64 │ 1 │ + 523. │ 522 │ UInt64 │ 0 │ + 524. │ 523 │ UInt64 │ 1 │ + 525. │ 524 │ UInt64 │ 0 │ + 526. │ 525 │ UInt64 │ 1 │ + 527. │ 526 │ UInt64 │ 0 │ + 528. │ 527 │ UInt64 │ 1 │ + 529. │ 528 │ UInt64 │ 0 │ + 530. │ 529 │ UInt64 │ 1 │ + 531. │ 530 │ UInt64 │ 0 │ + 532. │ 531 │ UInt64 │ 1 │ + 533. │ 532 │ UInt64 │ 0 │ + 534. │ 533 │ UInt64 │ 1 │ + 535. │ 534 │ UInt64 │ 0 │ + 536. │ 535 │ UInt64 │ 1 │ + 537. │ 536 │ UInt64 │ 0 │ + 538. │ 537 │ UInt64 │ 1 │ + 539. │ 538 │ UInt64 │ 0 │ + 540. │ 539 │ UInt64 │ 1 │ + 541. │ 540 │ UInt64 │ 0 │ + 542. │ 541 │ UInt64 │ 1 │ + 543. │ 542 │ UInt64 │ 0 │ + 544. │ 543 │ UInt64 │ 1 │ + 545. │ 544 │ UInt64 │ 0 │ + 546. │ 545 │ UInt64 │ 1 │ + 547. │ 546 │ UInt64 │ 0 │ + 548. │ 547 │ UInt64 │ 1 │ + 549. │ 548 │ UInt64 │ 0 │ + 550. │ 549 │ UInt64 │ 1 │ + 551. │ 550 │ UInt64 │ 0 │ + 552. │ 551 │ UInt64 │ 1 │ + 553. │ 552 │ UInt64 │ 0 │ + 554. │ 553 │ UInt64 │ 1 │ + 555. │ 554 │ UInt64 │ 0 │ + 556. │ 555 │ UInt64 │ 1 │ + 557. │ 556 │ UInt64 │ 0 │ + 558. │ 557 │ UInt64 │ 1 │ + 559. │ 558 │ UInt64 │ 0 │ + 560. │ 559 │ UInt64 │ 1 │ + 561. │ 560 │ UInt64 │ 0 │ + 562. │ 561 │ UInt64 │ 1 │ + 563. │ 562 │ UInt64 │ 0 │ + 564. │ 563 │ UInt64 │ 1 │ + 565. │ 564 │ UInt64 │ 0 │ + 566. │ 565 │ UInt64 │ 1 │ + 567. │ 566 │ UInt64 │ 0 │ + 568. │ 567 │ UInt64 │ 1 │ + 569. │ 568 │ UInt64 │ 0 │ + 570. │ 569 │ UInt64 │ 1 │ + 571. │ 570 │ UInt64 │ 0 │ + 572. │ 571 │ UInt64 │ 1 │ + 573. │ 572 │ UInt64 │ 0 │ + 574. │ 573 │ UInt64 │ 1 │ + 575. │ 574 │ UInt64 │ 0 │ + 576. │ 575 │ UInt64 │ 1 │ + 577. │ 576 │ UInt64 │ 0 │ + 578. │ 577 │ UInt64 │ 1 │ + 579. │ 578 │ UInt64 │ 0 │ + 580. │ 579 │ UInt64 │ 1 │ + 581. │ 580 │ UInt64 │ 0 │ + 582. │ 581 │ UInt64 │ 1 │ + 583. │ 582 │ UInt64 │ 0 │ + 584. │ 583 │ UInt64 │ 1 │ + 585. │ 584 │ UInt64 │ 0 │ + 586. │ 585 │ UInt64 │ 1 │ + 587. │ 586 │ UInt64 │ 0 │ + 588. │ 587 │ UInt64 │ 1 │ + 589. │ 588 │ UInt64 │ 0 │ + 590. │ 589 │ UInt64 │ 1 │ + 591. │ 590 │ UInt64 │ 0 │ + 592. │ 591 │ UInt64 │ 1 │ + 593. │ 592 │ UInt64 │ 0 │ + 594. │ 593 │ UInt64 │ 1 │ + 595. │ 594 │ UInt64 │ 0 │ + 596. │ 595 │ UInt64 │ 1 │ + 597. │ 596 │ UInt64 │ 0 │ + 598. │ 597 │ UInt64 │ 1 │ + 599. │ 598 │ UInt64 │ 0 │ + 600. │ 599 │ UInt64 │ 1 │ + 601. │ 600 │ UInt64 │ 0 │ + 602. │ 601 │ UInt64 │ 1 │ + 603. │ 602 │ UInt64 │ 0 │ + 604. │ 603 │ UInt64 │ 1 │ + 605. │ 604 │ UInt64 │ 0 │ + 606. │ 605 │ UInt64 │ 1 │ + 607. │ 606 │ UInt64 │ 0 │ + 608. │ 607 │ UInt64 │ 1 │ + 609. │ 608 │ UInt64 │ 0 │ + 610. │ 609 │ UInt64 │ 1 │ + 611. │ 610 │ UInt64 │ 0 │ + 612. │ 611 │ UInt64 │ 1 │ + 613. │ 612 │ UInt64 │ 0 │ + 614. │ 613 │ UInt64 │ 1 │ + 615. │ 614 │ UInt64 │ 0 │ + 616. │ 615 │ UInt64 │ 1 │ + 617. │ 616 │ UInt64 │ 0 │ + 618. │ 617 │ UInt64 │ 1 │ + 619. │ 618 │ UInt64 │ 0 │ + 620. │ 619 │ UInt64 │ 1 │ + 621. │ 620 │ UInt64 │ 0 │ + 622. │ 621 │ UInt64 │ 1 │ + 623. │ 622 │ UInt64 │ 0 │ + 624. │ 623 │ UInt64 │ 1 │ + 625. │ 624 │ UInt64 │ 0 │ + 626. │ 625 │ UInt64 │ 1 │ + 627. │ 626 │ UInt64 │ 0 │ + 628. │ 627 │ UInt64 │ 1 │ + 629. │ 628 │ UInt64 │ 0 │ + 630. │ 629 │ UInt64 │ 1 │ + 631. │ 630 │ UInt64 │ 0 │ + 632. │ 631 │ UInt64 │ 1 │ + 633. │ 632 │ UInt64 │ 0 │ + 634. │ 633 │ UInt64 │ 1 │ + 635. │ 634 │ UInt64 │ 0 │ + 636. │ 635 │ UInt64 │ 1 │ + 637. │ 636 │ UInt64 │ 0 │ + 638. │ 637 │ UInt64 │ 1 │ + 639. │ 638 │ UInt64 │ 0 │ + 640. │ 639 │ UInt64 │ 1 │ + 641. │ 640 │ UInt64 │ 0 │ + 642. │ 641 │ UInt64 │ 1 │ + 643. │ 642 │ UInt64 │ 0 │ + 644. │ 643 │ UInt64 │ 1 │ + 645. │ 644 │ UInt64 │ 0 │ + 646. │ 645 │ UInt64 │ 1 │ + 647. │ 646 │ UInt64 │ 0 │ + 648. │ 647 │ UInt64 │ 1 │ + 649. │ 648 │ UInt64 │ 0 │ + 650. │ 649 │ UInt64 │ 1 │ + 651. │ 650 │ UInt64 │ 0 │ + 652. │ 651 │ UInt64 │ 1 │ + 653. │ 652 │ UInt64 │ 0 │ + 654. │ 653 │ UInt64 │ 1 │ + 655. │ 654 │ UInt64 │ 0 │ + 656. │ 655 │ UInt64 │ 1 │ + 657. │ 656 │ UInt64 │ 0 │ + 658. │ 657 │ UInt64 │ 1 │ + 659. │ 658 │ UInt64 │ 0 │ + 660. │ 659 │ UInt64 │ 1 │ + 661. │ 660 │ UInt64 │ 0 │ + 662. │ 661 │ UInt64 │ 1 │ + 663. │ 662 │ UInt64 │ 0 │ + 664. │ 663 │ UInt64 │ 1 │ + 665. │ 664 │ UInt64 │ 0 │ + 666. │ 665 │ UInt64 │ 1 │ + 667. │ 666 │ UInt64 │ 0 │ + 668. │ 667 │ UInt64 │ 1 │ + 669. │ 668 │ UInt64 │ 0 │ + 670. │ 669 │ UInt64 │ 1 │ + 671. │ 670 │ UInt64 │ 0 │ + 672. │ 671 │ UInt64 │ 1 │ + 673. │ 672 │ UInt64 │ 0 │ + 674. │ 673 │ UInt64 │ 1 │ + 675. │ 674 │ UInt64 │ 0 │ + 676. │ 675 │ UInt64 │ 1 │ + 677. │ 676 │ UInt64 │ 0 │ + 678. │ 677 │ UInt64 │ 1 │ + 679. │ 678 │ UInt64 │ 0 │ + 680. │ 679 │ UInt64 │ 1 │ + 681. │ 680 │ UInt64 │ 0 │ + 682. │ 681 │ UInt64 │ 1 │ + 683. │ 682 │ UInt64 │ 0 │ + 684. │ 683 │ UInt64 │ 1 │ + 685. │ 684 │ UInt64 │ 0 │ + 686. │ 685 │ UInt64 │ 1 │ + 687. │ 686 │ UInt64 │ 0 │ + 688. │ 687 │ UInt64 │ 1 │ + 689. │ 688 │ UInt64 │ 0 │ + 690. │ 689 │ UInt64 │ 1 │ + 691. │ 690 │ UInt64 │ 0 │ + 692. │ 691 │ UInt64 │ 1 │ + 693. │ 692 │ UInt64 │ 0 │ + 694. │ 693 │ UInt64 │ 1 │ + 695. │ 694 │ UInt64 │ 0 │ + 696. │ 695 │ UInt64 │ 1 │ + 697. │ 696 │ UInt64 │ 0 │ + 698. │ 697 │ UInt64 │ 1 │ + 699. │ 698 │ UInt64 │ 0 │ + 700. │ 699 │ UInt64 │ 1 │ + 701. │ 700 │ UInt64 │ 0 │ + 702. │ 701 │ UInt64 │ 1 │ + 703. │ 702 │ UInt64 │ 0 │ + 704. │ 703 │ UInt64 │ 1 │ + 705. │ 704 │ UInt64 │ 0 │ + 706. │ 705 │ UInt64 │ 1 │ + 707. │ 706 │ UInt64 │ 0 │ + 708. │ 707 │ UInt64 │ 1 │ + 709. │ 708 │ UInt64 │ 0 │ + 710. │ 709 │ UInt64 │ 1 │ + 711. │ 710 │ UInt64 │ 0 │ + 712. │ 711 │ UInt64 │ 1 │ + 713. │ 712 │ UInt64 │ 0 │ + 714. │ 713 │ UInt64 │ 1 │ + 715. │ 714 │ UInt64 │ 0 │ + 716. │ 715 │ UInt64 │ 1 │ + 717. │ 716 │ UInt64 │ 0 │ + 718. │ 717 │ UInt64 │ 1 │ + 719. │ 718 │ UInt64 │ 0 │ + 720. │ 719 │ UInt64 │ 1 │ + 721. │ 720 │ UInt64 │ 0 │ + 722. │ 721 │ UInt64 │ 1 │ + 723. │ 722 │ UInt64 │ 0 │ + 724. │ 723 │ UInt64 │ 1 │ + 725. │ 724 │ UInt64 │ 0 │ + 726. │ 725 │ UInt64 │ 1 │ + 727. │ 726 │ UInt64 │ 0 │ + 728. │ 727 │ UInt64 │ 1 │ + 729. │ 728 │ UInt64 │ 0 │ + 730. │ 729 │ UInt64 │ 1 │ + 731. │ 730 │ UInt64 │ 0 │ + 732. │ 731 │ UInt64 │ 1 │ + 733. │ 732 │ UInt64 │ 0 │ + 734. │ 733 │ UInt64 │ 1 │ + 735. │ 734 │ UInt64 │ 0 │ + 736. │ 735 │ UInt64 │ 1 │ + 737. │ 736 │ UInt64 │ 0 │ + 738. │ 737 │ UInt64 │ 1 │ + 739. │ 738 │ UInt64 │ 0 │ + 740. │ 739 │ UInt64 │ 1 │ + 741. │ 740 │ UInt64 │ 0 │ + 742. │ 741 │ UInt64 │ 1 │ + 743. │ 742 │ UInt64 │ 0 │ + 744. │ 743 │ UInt64 │ 1 │ + 745. │ 744 │ UInt64 │ 0 │ + 746. │ 745 │ UInt64 │ 1 │ + 747. │ 746 │ UInt64 │ 0 │ + 748. │ 747 │ UInt64 │ 1 │ + 749. │ 748 │ UInt64 │ 0 │ + 750. │ 749 │ UInt64 │ 1 │ + 751. │ 750 │ UInt64 │ 0 │ + 752. │ 751 │ UInt64 │ 1 │ + 753. │ 752 │ UInt64 │ 0 │ + 754. │ 753 │ UInt64 │ 1 │ + 755. │ 754 │ UInt64 │ 0 │ + 756. │ 755 │ UInt64 │ 1 │ + 757. │ 756 │ UInt64 │ 0 │ + 758. │ 757 │ UInt64 │ 1 │ + 759. │ 758 │ UInt64 │ 0 │ + 760. │ 759 │ UInt64 │ 1 │ + 761. │ 760 │ UInt64 │ 0 │ + 762. │ 761 │ UInt64 │ 1 │ + 763. │ 762 │ UInt64 │ 0 │ + 764. │ 763 │ UInt64 │ 1 │ + 765. │ 764 │ UInt64 │ 0 │ + 766. │ 765 │ UInt64 │ 1 │ + 767. │ 766 │ UInt64 │ 0 │ + 768. │ 767 │ UInt64 │ 1 │ + 769. │ 768 │ UInt64 │ 0 │ + 770. │ 769 │ UInt64 │ 1 │ + 771. │ 770 │ UInt64 │ 0 │ + 772. │ 771 │ UInt64 │ 1 │ + 773. │ 772 │ UInt64 │ 0 │ + 774. │ 773 │ UInt64 │ 1 │ + 775. │ 774 │ UInt64 │ 0 │ + 776. │ 775 │ UInt64 │ 1 │ + 777. │ 776 │ UInt64 │ 0 │ + 778. │ 777 │ UInt64 │ 1 │ + 779. │ 778 │ UInt64 │ 0 │ + 780. │ 779 │ UInt64 │ 1 │ + 781. │ 780 │ UInt64 │ 0 │ + 782. │ 781 │ UInt64 │ 1 │ + 783. │ 782 │ UInt64 │ 0 │ + 784. │ 783 │ UInt64 │ 1 │ + 785. │ 784 │ UInt64 │ 0 │ + 786. │ 785 │ UInt64 │ 1 │ + 787. │ 786 │ UInt64 │ 0 │ + 788. │ 787 │ UInt64 │ 1 │ + 789. │ 788 │ UInt64 │ 0 │ + 790. │ 789 │ UInt64 │ 1 │ + 791. │ 790 │ UInt64 │ 0 │ + 792. │ 791 │ UInt64 │ 1 │ + 793. │ 792 │ UInt64 │ 0 │ + 794. │ 793 │ UInt64 │ 1 │ + 795. │ 794 │ UInt64 │ 0 │ + 796. │ 795 │ UInt64 │ 1 │ + 797. │ 796 │ UInt64 │ 0 │ + 798. │ 797 │ UInt64 │ 1 │ + 799. │ 798 │ UInt64 │ 0 │ + 800. │ 799 │ UInt64 │ 1 │ + 801. │ 800 │ UInt64 │ 0 │ + 802. │ 801 │ UInt64 │ 1 │ + 803. │ 802 │ UInt64 │ 0 │ + 804. │ 803 │ UInt64 │ 1 │ + 805. │ 804 │ UInt64 │ 0 │ + 806. │ 805 │ UInt64 │ 1 │ + 807. │ 806 │ UInt64 │ 0 │ + 808. │ 807 │ UInt64 │ 1 │ + 809. │ 808 │ UInt64 │ 0 │ + 810. │ 809 │ UInt64 │ 1 │ + 811. │ 810 │ UInt64 │ 0 │ + 812. │ 811 │ UInt64 │ 1 │ + 813. │ 812 │ UInt64 │ 0 │ + 814. │ 813 │ UInt64 │ 1 │ + 815. │ 814 │ UInt64 │ 0 │ + 816. │ 815 │ UInt64 │ 1 │ + 817. │ 816 │ UInt64 │ 0 │ + 818. │ 817 │ UInt64 │ 1 │ + 819. │ 818 │ UInt64 │ 0 │ + 820. │ 819 │ UInt64 │ 1 │ + 821. │ 820 │ UInt64 │ 0 │ + 822. │ 821 │ UInt64 │ 1 │ + 823. │ 822 │ UInt64 │ 0 │ + 824. │ 823 │ UInt64 │ 1 │ + 825. │ 824 │ UInt64 │ 0 │ + 826. │ 825 │ UInt64 │ 1 │ + 827. │ 826 │ UInt64 │ 0 │ + 828. │ 827 │ UInt64 │ 1 │ + 829. │ 828 │ UInt64 │ 0 │ + 830. │ 829 │ UInt64 │ 1 │ + 831. │ 830 │ UInt64 │ 0 │ + 832. │ 831 │ UInt64 │ 1 │ + 833. │ 832 │ UInt64 │ 0 │ + 834. │ 833 │ UInt64 │ 1 │ + 835. │ 834 │ UInt64 │ 0 │ + 836. │ 835 │ UInt64 │ 1 │ + 837. │ 836 │ UInt64 │ 0 │ + 838. │ 837 │ UInt64 │ 1 │ + 839. │ 838 │ UInt64 │ 0 │ + 840. │ 839 │ UInt64 │ 1 │ + 841. │ 840 │ UInt64 │ 0 │ + 842. │ 841 │ UInt64 │ 1 │ + 843. │ 842 │ UInt64 │ 0 │ + 844. │ 843 │ UInt64 │ 1 │ + 845. │ 844 │ UInt64 │ 0 │ + 846. │ 845 │ UInt64 │ 1 │ + 847. │ 846 │ UInt64 │ 0 │ + 848. │ 847 │ UInt64 │ 1 │ + 849. │ 848 │ UInt64 │ 0 │ + 850. │ 849 │ UInt64 │ 1 │ + 851. │ 850 │ UInt64 │ 0 │ + 852. │ 851 │ UInt64 │ 1 │ + 853. │ 852 │ UInt64 │ 0 │ + 854. │ 853 │ UInt64 │ 1 │ + 855. │ 854 │ UInt64 │ 0 │ + 856. │ 855 │ UInt64 │ 1 │ + 857. │ 856 │ UInt64 │ 0 │ + 858. │ 857 │ UInt64 │ 1 │ + 859. │ 858 │ UInt64 │ 0 │ + 860. │ 859 │ UInt64 │ 1 │ + 861. │ 860 │ UInt64 │ 0 │ + 862. │ 861 │ UInt64 │ 1 │ + 863. │ 862 │ UInt64 │ 0 │ + 864. │ 863 │ UInt64 │ 1 │ + 865. │ 864 │ UInt64 │ 0 │ + 866. │ 865 │ UInt64 │ 1 │ + 867. │ 866 │ UInt64 │ 0 │ + 868. │ 867 │ UInt64 │ 1 │ + 869. │ 868 │ UInt64 │ 0 │ + 870. │ 869 │ UInt64 │ 1 │ + 871. │ 870 │ UInt64 │ 0 │ + 872. │ 871 │ UInt64 │ 1 │ + 873. │ 872 │ UInt64 │ 0 │ + 874. │ 873 │ UInt64 │ 1 │ + 875. │ 874 │ UInt64 │ 0 │ + 876. │ 875 │ UInt64 │ 1 │ + 877. │ 876 │ UInt64 │ 0 │ + 878. │ 877 │ UInt64 │ 1 │ + 879. │ 878 │ UInt64 │ 0 │ + 880. │ 879 │ UInt64 │ 1 │ + 881. │ 880 │ UInt64 │ 0 │ + 882. │ 881 │ UInt64 │ 1 │ + 883. │ 882 │ UInt64 │ 0 │ + 884. │ 883 │ UInt64 │ 1 │ + 885. │ 884 │ UInt64 │ 0 │ + 886. │ 885 │ UInt64 │ 1 │ + 887. │ 886 │ UInt64 │ 0 │ + 888. │ 887 │ UInt64 │ 1 │ + 889. │ 888 │ UInt64 │ 0 │ + 890. │ 889 │ UInt64 │ 1 │ + 891. │ 890 │ UInt64 │ 0 │ + 892. │ 891 │ UInt64 │ 1 │ + 893. │ 892 │ UInt64 │ 0 │ + 894. │ 893 │ UInt64 │ 1 │ + 895. │ 894 │ UInt64 │ 0 │ + 896. │ 895 │ UInt64 │ 1 │ + 897. │ 896 │ UInt64 │ 0 │ + 898. │ 897 │ UInt64 │ 1 │ + 899. │ 898 │ UInt64 │ 0 │ + 900. │ 899 │ UInt64 │ 1 │ + 901. │ 900 │ UInt64 │ 0 │ + 902. │ 901 │ UInt64 │ 1 │ + 903. │ 902 │ UInt64 │ 0 │ + 904. │ 903 │ UInt64 │ 1 │ + 905. │ 904 │ UInt64 │ 0 │ + 906. │ 905 │ UInt64 │ 1 │ + 907. │ 906 │ UInt64 │ 0 │ + 908. │ 907 │ UInt64 │ 1 │ + 909. │ 908 │ UInt64 │ 0 │ + 910. │ 909 │ UInt64 │ 1 │ + 911. │ 910 │ UInt64 │ 0 │ + 912. │ 911 │ UInt64 │ 1 │ + 913. │ 912 │ UInt64 │ 0 │ + 914. │ 913 │ UInt64 │ 1 │ + 915. │ 914 │ UInt64 │ 0 │ + 916. │ 915 │ UInt64 │ 1 │ + 917. │ 916 │ UInt64 │ 0 │ + 918. │ 917 │ UInt64 │ 1 │ + 919. │ 918 │ UInt64 │ 0 │ + 920. │ 919 │ UInt64 │ 1 │ + 921. │ 920 │ UInt64 │ 0 │ + 922. │ 921 │ UInt64 │ 1 │ + 923. │ 922 │ UInt64 │ 0 │ + 924. │ 923 │ UInt64 │ 1 │ + 925. │ 924 │ UInt64 │ 0 │ + 926. │ 925 │ UInt64 │ 1 │ + 927. │ 926 │ UInt64 │ 0 │ + 928. │ 927 │ UInt64 │ 1 │ + 929. │ 928 │ UInt64 │ 0 │ + 930. │ 929 │ UInt64 │ 1 │ + 931. │ 930 │ UInt64 │ 0 │ + 932. │ 931 │ UInt64 │ 1 │ + 933. │ 932 │ UInt64 │ 0 │ + 934. │ 933 │ UInt64 │ 1 │ + 935. │ 934 │ UInt64 │ 0 │ + 936. │ 935 │ UInt64 │ 1 │ + 937. │ 936 │ UInt64 │ 0 │ + 938. │ 937 │ UInt64 │ 1 │ + 939. │ 938 │ UInt64 │ 0 │ + 940. │ 939 │ UInt64 │ 1 │ + 941. │ 940 │ UInt64 │ 0 │ + 942. │ 941 │ UInt64 │ 1 │ + 943. │ 942 │ UInt64 │ 0 │ + 944. │ 943 │ UInt64 │ 1 │ + 945. │ 944 │ UInt64 │ 0 │ + 946. │ 945 │ UInt64 │ 1 │ + 947. │ 946 │ UInt64 │ 0 │ + 948. │ 947 │ UInt64 │ 1 │ + 949. │ 948 │ UInt64 │ 0 │ + 950. │ 949 │ UInt64 │ 1 │ + 951. │ 950 │ UInt64 │ 0 │ + 952. │ 951 │ UInt64 │ 1 │ + 953. │ 952 │ UInt64 │ 0 │ + 954. │ 953 │ UInt64 │ 1 │ + 955. │ 954 │ UInt64 │ 0 │ + 956. │ 955 │ UInt64 │ 1 │ + 957. │ 956 │ UInt64 │ 0 │ + 958. │ 957 │ UInt64 │ 1 │ + 959. │ 958 │ UInt64 │ 0 │ + 960. │ 959 │ UInt64 │ 1 │ + 961. │ 960 │ UInt64 │ 0 │ + 962. │ 961 │ UInt64 │ 1 │ + 963. │ 962 │ UInt64 │ 0 │ + 964. │ 963 │ UInt64 │ 1 │ + 965. │ 964 │ UInt64 │ 0 │ + 966. │ 965 │ UInt64 │ 1 │ + 967. │ 966 │ UInt64 │ 0 │ + 968. │ 967 │ UInt64 │ 1 │ + 969. │ 968 │ UInt64 │ 0 │ + 970. │ 969 │ UInt64 │ 1 │ + 971. │ 970 │ UInt64 │ 0 │ + 972. │ 971 │ UInt64 │ 1 │ + 973. │ 972 │ UInt64 │ 0 │ + 974. │ 973 │ UInt64 │ 1 │ + 975. │ 974 │ UInt64 │ 0 │ + 976. │ 975 │ UInt64 │ 1 │ + 977. │ 976 │ UInt64 │ 0 │ + 978. │ 977 │ UInt64 │ 1 │ + 979. │ 978 │ UInt64 │ 0 │ + 980. │ 979 │ UInt64 │ 1 │ + 981. │ 980 │ UInt64 │ 0 │ + 982. │ 981 │ UInt64 │ 1 │ + 983. │ 982 │ UInt64 │ 0 │ + 984. │ 983 │ UInt64 │ 1 │ + 985. │ 984 │ UInt64 │ 0 │ + 986. │ 985 │ UInt64 │ 1 │ + 987. │ 986 │ UInt64 │ 0 │ + 988. │ 987 │ UInt64 │ 1 │ + 989. │ 988 │ UInt64 │ 0 │ + 990. │ 989 │ UInt64 │ 1 │ + 991. │ 990 │ UInt64 │ 0 │ + 992. │ 991 │ UInt64 │ 1 │ + 993. │ 992 │ UInt64 │ 0 │ + 994. │ 993 │ UInt64 │ 1 │ + 995. │ 994 │ UInt64 │ 0 │ + 996. │ 995 │ UInt64 │ 1 │ + 997. │ 996 │ UInt64 │ 0 │ + 998. │ 997 │ UInt64 │ 1 │ + 999. │ 998 │ UInt64 │ 0 │ +1000. │ 999 │ UInt64 │ 1 │ + └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ + 100. │ 99 │ UInt64 │ 1 │ + 101. │ 100 │ UInt64 │ 0 │ + 102. │ 101 │ UInt64 │ 1 │ + 103. │ 102 │ UInt64 │ 0 │ + 104. │ 103 │ UInt64 │ 1 │ + 105. │ 104 │ UInt64 │ 0 │ + 106. │ 105 │ UInt64 │ 1 │ + 107. │ 106 │ UInt64 │ 0 │ + 108. │ 107 │ UInt64 │ 1 │ + 109. │ 108 │ UInt64 │ 0 │ + 110. │ 109 │ UInt64 │ 1 │ + 111. │ 110 │ UInt64 │ 0 │ + 112. │ 111 │ UInt64 │ 1 │ + 113. │ 112 │ UInt64 │ 0 │ + 114. │ 113 │ UInt64 │ 1 │ + 115. │ 114 │ UInt64 │ 0 │ + 116. │ 115 │ UInt64 │ 1 │ + 117. │ 116 │ UInt64 │ 0 │ + 118. │ 117 │ UInt64 │ 1 │ + 119. │ 118 │ UInt64 │ 0 │ + 120. │ 119 │ UInt64 │ 1 │ + 121. │ 120 │ UInt64 │ 0 │ + 122. │ 121 │ UInt64 │ 1 │ + 123. │ 122 │ UInt64 │ 0 │ + 124. │ 123 │ UInt64 │ 1 │ + 125. │ 124 │ UInt64 │ 0 │ + 126. │ 125 │ UInt64 │ 1 │ + 127. │ 126 │ UInt64 │ 0 │ + 128. │ 127 │ UInt64 │ 1 │ + 129. │ 128 │ UInt64 │ 0 │ + 130. │ 129 │ UInt64 │ 1 │ + 131. │ 130 │ UInt64 │ 0 │ + 132. │ 131 │ UInt64 │ 1 │ + 133. │ 132 │ UInt64 │ 0 │ + 134. │ 133 │ UInt64 │ 1 │ + 135. │ 134 │ UInt64 │ 0 │ + 136. │ 135 │ UInt64 │ 1 │ + 137. │ 136 │ UInt64 │ 0 │ + 138. │ 137 │ UInt64 │ 1 │ + 139. │ 138 │ UInt64 │ 0 │ + 140. │ 139 │ UInt64 │ 1 │ + 141. │ 140 │ UInt64 │ 0 │ + 142. │ 141 │ UInt64 │ 1 │ + 143. │ 142 │ UInt64 │ 0 │ + 144. │ 143 │ UInt64 │ 1 │ + 145. │ 144 │ UInt64 │ 0 │ + 146. │ 145 │ UInt64 │ 1 │ + 147. │ 146 │ UInt64 │ 0 │ + 148. │ 147 │ UInt64 │ 1 │ + 149. │ 148 │ UInt64 │ 0 │ + 150. │ 149 │ UInt64 │ 1 │ + 151. │ 150 │ UInt64 │ 0 │ + 152. │ 151 │ UInt64 │ 1 │ + 153. │ 152 │ UInt64 │ 0 │ + 154. │ 153 │ UInt64 │ 1 │ + 155. │ 154 │ UInt64 │ 0 │ + 156. │ 155 │ UInt64 │ 1 │ + 157. │ 156 │ UInt64 │ 0 │ + 158. │ 157 │ UInt64 │ 1 │ + 159. │ 158 │ UInt64 │ 0 │ + 160. │ 159 │ UInt64 │ 1 │ + 161. │ 160 │ UInt64 │ 0 │ + 162. │ 161 │ UInt64 │ 1 │ + 163. │ 162 │ UInt64 │ 0 │ + 164. │ 163 │ UInt64 │ 1 │ + 165. │ 164 │ UInt64 │ 0 │ + 166. │ 165 │ UInt64 │ 1 │ + 167. │ 166 │ UInt64 │ 0 │ + 168. │ 167 │ UInt64 │ 1 │ + 169. │ 168 │ UInt64 │ 0 │ + 170. │ 169 │ UInt64 │ 1 │ + 171. │ 170 │ UInt64 │ 0 │ + 172. │ 171 │ UInt64 │ 1 │ + 173. │ 172 │ UInt64 │ 0 │ + 174. │ 173 │ UInt64 │ 1 │ + 175. │ 174 │ UInt64 │ 0 │ + 176. │ 175 │ UInt64 │ 1 │ + 177. │ 176 │ UInt64 │ 0 │ + 178. │ 177 │ UInt64 │ 1 │ + 179. │ 178 │ UInt64 │ 0 │ + 180. │ 179 │ UInt64 │ 1 │ + 181. │ 180 │ UInt64 │ 0 │ + 182. │ 181 │ UInt64 │ 1 │ + 183. │ 182 │ UInt64 │ 0 │ + 184. │ 183 │ UInt64 │ 1 │ + 185. │ 184 │ UInt64 │ 0 │ + 186. │ 185 │ UInt64 │ 1 │ + 187. │ 186 │ UInt64 │ 0 │ + 188. │ 187 │ UInt64 │ 1 │ + 189. │ 188 │ UInt64 │ 0 │ + 190. │ 189 │ UInt64 │ 1 │ + 191. │ 190 │ UInt64 │ 0 │ + 192. │ 191 │ UInt64 │ 1 │ + 193. │ 192 │ UInt64 │ 0 │ + 194. │ 193 │ UInt64 │ 1 │ + 195. │ 194 │ UInt64 │ 0 │ + 196. │ 195 │ UInt64 │ 1 │ + 197. │ 196 │ UInt64 │ 0 │ + 198. │ 197 │ UInt64 │ 1 │ + 199. │ 198 │ UInt64 │ 0 │ + 200. │ 199 │ UInt64 │ 1 │ + 201. │ 200 │ UInt64 │ 0 │ + 202. │ 201 │ UInt64 │ 1 │ + 203. │ 202 │ UInt64 │ 0 │ + 204. │ 203 │ UInt64 │ 1 │ + 205. │ 204 │ UInt64 │ 0 │ + 206. │ 205 │ UInt64 │ 1 │ + 207. │ 206 │ UInt64 │ 0 │ + 208. │ 207 │ UInt64 │ 1 │ + 209. │ 208 │ UInt64 │ 0 │ + 210. │ 209 │ UInt64 │ 1 │ + 211. │ 210 │ UInt64 │ 0 │ + 212. │ 211 │ UInt64 │ 1 │ + 213. │ 212 │ UInt64 │ 0 │ + 214. │ 213 │ UInt64 │ 1 │ + 215. │ 214 │ UInt64 │ 0 │ + 216. │ 215 │ UInt64 │ 1 │ + 217. │ 216 │ UInt64 │ 0 │ + 218. │ 217 │ UInt64 │ 1 │ + 219. │ 218 │ UInt64 │ 0 │ + 220. │ 219 │ UInt64 │ 1 │ + 221. │ 220 │ UInt64 │ 0 │ + 222. │ 221 │ UInt64 │ 1 │ + 223. │ 222 │ UInt64 │ 0 │ + 224. │ 223 │ UInt64 │ 1 │ + 225. │ 224 │ UInt64 │ 0 │ + 226. │ 225 │ UInt64 │ 1 │ + 227. │ 226 │ UInt64 │ 0 │ + 228. │ 227 │ UInt64 │ 1 │ + 229. │ 228 │ UInt64 │ 0 │ + 230. │ 229 │ UInt64 │ 1 │ + 231. │ 230 │ UInt64 │ 0 │ + 232. │ 231 │ UInt64 │ 1 │ + 233. │ 232 │ UInt64 │ 0 │ + 234. │ 233 │ UInt64 │ 1 │ + 235. │ 234 │ UInt64 │ 0 │ + 236. │ 235 │ UInt64 │ 1 │ + 237. │ 236 │ UInt64 │ 0 │ + 238. │ 237 │ UInt64 │ 1 │ + 239. │ 238 │ UInt64 │ 0 │ + 240. │ 239 │ UInt64 │ 1 │ + 241. │ 240 │ UInt64 │ 0 │ + 242. │ 241 │ UInt64 │ 1 │ + 243. │ 242 │ UInt64 │ 0 │ + 244. │ 243 │ UInt64 │ 1 │ + 245. │ 244 │ UInt64 │ 0 │ + 246. │ 245 │ UInt64 │ 1 │ + 247. │ 246 │ UInt64 │ 0 │ + 248. │ 247 │ UInt64 │ 1 │ + 249. │ 248 │ UInt64 │ 0 │ + 250. │ 249 │ UInt64 │ 1 │ + 251. │ 250 │ UInt64 │ 0 │ + 252. │ 251 │ UInt64 │ 1 │ + 253. │ 252 │ UInt64 │ 0 │ + 254. │ 253 │ UInt64 │ 1 │ + 255. │ 254 │ UInt64 │ 0 │ + 256. │ 255 │ UInt64 │ 1 │ + 257. │ 256 │ UInt64 │ 0 │ + 258. │ 257 │ UInt64 │ 1 │ + 259. │ 258 │ UInt64 │ 0 │ + 260. │ 259 │ UInt64 │ 1 │ + 261. │ 260 │ UInt64 │ 0 │ + 262. │ 261 │ UInt64 │ 1 │ + 263. │ 262 │ UInt64 │ 0 │ + 264. │ 263 │ UInt64 │ 1 │ + 265. │ 264 │ UInt64 │ 0 │ + 266. │ 265 │ UInt64 │ 1 │ + 267. │ 266 │ UInt64 │ 0 │ + 268. │ 267 │ UInt64 │ 1 │ + 269. │ 268 │ UInt64 │ 0 │ + 270. │ 269 │ UInt64 │ 1 │ + 271. │ 270 │ UInt64 │ 0 │ + 272. │ 271 │ UInt64 │ 1 │ + 273. │ 272 │ UInt64 │ 0 │ + 274. │ 273 │ UInt64 │ 1 │ + 275. │ 274 │ UInt64 │ 0 │ + 276. │ 275 │ UInt64 │ 1 │ + 277. │ 276 │ UInt64 │ 0 │ + 278. │ 277 │ UInt64 │ 1 │ + 279. │ 278 │ UInt64 │ 0 │ + 280. │ 279 │ UInt64 │ 1 │ + 281. │ 280 │ UInt64 │ 0 │ + 282. │ 281 │ UInt64 │ 1 │ + 283. │ 282 │ UInt64 │ 0 │ + 284. │ 283 │ UInt64 │ 1 │ + 285. │ 284 │ UInt64 │ 0 │ + 286. │ 285 │ UInt64 │ 1 │ + 287. │ 286 │ UInt64 │ 0 │ + 288. │ 287 │ UInt64 │ 1 │ + 289. │ 288 │ UInt64 │ 0 │ + 290. │ 289 │ UInt64 │ 1 │ + 291. │ 290 │ UInt64 │ 0 │ + 292. │ 291 │ UInt64 │ 1 │ + 293. │ 292 │ UInt64 │ 0 │ + 294. │ 293 │ UInt64 │ 1 │ + 295. │ 294 │ UInt64 │ 0 │ + 296. │ 295 │ UInt64 │ 1 │ + 297. │ 296 │ UInt64 │ 0 │ + 298. │ 297 │ UInt64 │ 1 │ + 299. │ 298 │ UInt64 │ 0 │ + 300. │ 299 │ UInt64 │ 1 │ + 301. │ 300 │ UInt64 │ 0 │ + 302. │ 301 │ UInt64 │ 1 │ + 303. │ 302 │ UInt64 │ 0 │ + 304. │ 303 │ UInt64 │ 1 │ + 305. │ 304 │ UInt64 │ 0 │ + 306. │ 305 │ UInt64 │ 1 │ + 307. │ 306 │ UInt64 │ 0 │ + 308. │ 307 │ UInt64 │ 1 │ + 309. │ 308 │ UInt64 │ 0 │ + 310. │ 309 │ UInt64 │ 1 │ + 311. │ 310 │ UInt64 │ 0 │ + 312. │ 311 │ UInt64 │ 1 │ + 313. │ 312 │ UInt64 │ 0 │ + 314. │ 313 │ UInt64 │ 1 │ + 315. │ 314 │ UInt64 │ 0 │ + 316. │ 315 │ UInt64 │ 1 │ + 317. │ 316 │ UInt64 │ 0 │ + 318. │ 317 │ UInt64 │ 1 │ + 319. │ 318 │ UInt64 │ 0 │ + 320. │ 319 │ UInt64 │ 1 │ + 321. │ 320 │ UInt64 │ 0 │ + 322. │ 321 │ UInt64 │ 1 │ + 323. │ 322 │ UInt64 │ 0 │ + 324. │ 323 │ UInt64 │ 1 │ + 325. │ 324 │ UInt64 │ 0 │ + 326. │ 325 │ UInt64 │ 1 │ + 327. │ 326 │ UInt64 │ 0 │ + 328. │ 327 │ UInt64 │ 1 │ + 329. │ 328 │ UInt64 │ 0 │ + 330. │ 329 │ UInt64 │ 1 │ + 331. │ 330 │ UInt64 │ 0 │ + 332. │ 331 │ UInt64 │ 1 │ + 333. │ 332 │ UInt64 │ 0 │ + 334. │ 333 │ UInt64 │ 1 │ + 335. │ 334 │ UInt64 │ 0 │ + 336. │ 335 │ UInt64 │ 1 │ + 337. │ 336 │ UInt64 │ 0 │ + 338. │ 337 │ UInt64 │ 1 │ + 339. │ 338 │ UInt64 │ 0 │ + 340. │ 339 │ UInt64 │ 1 │ + 341. │ 340 │ UInt64 │ 0 │ + 342. │ 341 │ UInt64 │ 1 │ + 343. │ 342 │ UInt64 │ 0 │ + 344. │ 343 │ UInt64 │ 1 │ + 345. │ 344 │ UInt64 │ 0 │ + 346. │ 345 │ UInt64 │ 1 │ + 347. │ 346 │ UInt64 │ 0 │ + 348. │ 347 │ UInt64 │ 1 │ + 349. │ 348 │ UInt64 │ 0 │ + 350. │ 349 │ UInt64 │ 1 │ + 351. │ 350 │ UInt64 │ 0 │ + 352. │ 351 │ UInt64 │ 1 │ + 353. │ 352 │ UInt64 │ 0 │ + 354. │ 353 │ UInt64 │ 1 │ + 355. │ 354 │ UInt64 │ 0 │ + 356. │ 355 │ UInt64 │ 1 │ + 357. │ 356 │ UInt64 │ 0 │ + 358. │ 357 │ UInt64 │ 1 │ + 359. │ 358 │ UInt64 │ 0 │ + 360. │ 359 │ UInt64 │ 1 │ + 361. │ 360 │ UInt64 │ 0 │ + 362. │ 361 │ UInt64 │ 1 │ + 363. │ 362 │ UInt64 │ 0 │ + 364. │ 363 │ UInt64 │ 1 │ + 365. │ 364 │ UInt64 │ 0 │ + 366. │ 365 │ UInt64 │ 1 │ + 367. │ 366 │ UInt64 │ 0 │ + 368. │ 367 │ UInt64 │ 1 │ + 369. │ 368 │ UInt64 │ 0 │ + 370. │ 369 │ UInt64 │ 1 │ + 371. │ 370 │ UInt64 │ 0 │ + 372. │ 371 │ UInt64 │ 1 │ + 373. │ 372 │ UInt64 │ 0 │ + 374. │ 373 │ UInt64 │ 1 │ + 375. │ 374 │ UInt64 │ 0 │ + 376. │ 375 │ UInt64 │ 1 │ + 377. │ 376 │ UInt64 │ 0 │ + 378. │ 377 │ UInt64 │ 1 │ + 379. │ 378 │ UInt64 │ 0 │ + 380. │ 379 │ UInt64 │ 1 │ + 381. │ 380 │ UInt64 │ 0 │ + 382. │ 381 │ UInt64 │ 1 │ + 383. │ 382 │ UInt64 │ 0 │ + 384. │ 383 │ UInt64 │ 1 │ + 385. │ 384 │ UInt64 │ 0 │ + 386. │ 385 │ UInt64 │ 1 │ + 387. │ 386 │ UInt64 │ 0 │ + 388. │ 387 │ UInt64 │ 1 │ + 389. │ 388 │ UInt64 │ 0 │ + 390. │ 389 │ UInt64 │ 1 │ + 391. │ 390 │ UInt64 │ 0 │ + 392. │ 391 │ UInt64 │ 1 │ + 393. │ 392 │ UInt64 │ 0 │ + 394. │ 393 │ UInt64 │ 1 │ + 395. │ 394 │ UInt64 │ 0 │ + 396. │ 395 │ UInt64 │ 1 │ + 397. │ 396 │ UInt64 │ 0 │ + 398. │ 397 │ UInt64 │ 1 │ + 399. │ 398 │ UInt64 │ 0 │ + 400. │ 399 │ UInt64 │ 1 │ + 401. │ 400 │ UInt64 │ 0 │ + 402. │ 401 │ UInt64 │ 1 │ + 403. │ 402 │ UInt64 │ 0 │ + 404. │ 403 │ UInt64 │ 1 │ + 405. │ 404 │ UInt64 │ 0 │ + 406. │ 405 │ UInt64 │ 1 │ + 407. │ 406 │ UInt64 │ 0 │ + 408. │ 407 │ UInt64 │ 1 │ + 409. │ 408 │ UInt64 │ 0 │ + 410. │ 409 │ UInt64 │ 1 │ + 411. │ 410 │ UInt64 │ 0 │ + 412. │ 411 │ UInt64 │ 1 │ + 413. │ 412 │ UInt64 │ 0 │ + 414. │ 413 │ UInt64 │ 1 │ + 415. │ 414 │ UInt64 │ 0 │ + 416. │ 415 │ UInt64 │ 1 │ + 417. │ 416 │ UInt64 │ 0 │ + 418. │ 417 │ UInt64 │ 1 │ + 419. │ 418 │ UInt64 │ 0 │ + 420. │ 419 │ UInt64 │ 1 │ + 421. │ 420 │ UInt64 │ 0 │ + 422. │ 421 │ UInt64 │ 1 │ + 423. │ 422 │ UInt64 │ 0 │ + 424. │ 423 │ UInt64 │ 1 │ + 425. │ 424 │ UInt64 │ 0 │ + 426. │ 425 │ UInt64 │ 1 │ + 427. │ 426 │ UInt64 │ 0 │ + 428. │ 427 │ UInt64 │ 1 │ + 429. │ 428 │ UInt64 │ 0 │ + 430. │ 429 │ UInt64 │ 1 │ + 431. │ 430 │ UInt64 │ 0 │ + 432. │ 431 │ UInt64 │ 1 │ + 433. │ 432 │ UInt64 │ 0 │ + 434. │ 433 │ UInt64 │ 1 │ + 435. │ 434 │ UInt64 │ 0 │ + 436. │ 435 │ UInt64 │ 1 │ + 437. │ 436 │ UInt64 │ 0 │ + 438. │ 437 │ UInt64 │ 1 │ + 439. │ 438 │ UInt64 │ 0 │ + 440. │ 439 │ UInt64 │ 1 │ + 441. │ 440 │ UInt64 │ 0 │ + 442. │ 441 │ UInt64 │ 1 │ + 443. │ 442 │ UInt64 │ 0 │ + 444. │ 443 │ UInt64 │ 1 │ + 445. │ 444 │ UInt64 │ 0 │ + 446. │ 445 │ UInt64 │ 1 │ + 447. │ 446 │ UInt64 │ 0 │ + 448. │ 447 │ UInt64 │ 1 │ + 449. │ 448 │ UInt64 │ 0 │ + 450. │ 449 │ UInt64 │ 1 │ + 451. │ 450 │ UInt64 │ 0 │ + 452. │ 451 │ UInt64 │ 1 │ + 453. │ 452 │ UInt64 │ 0 │ + 454. │ 453 │ UInt64 │ 1 │ + 455. │ 454 │ UInt64 │ 0 │ + 456. │ 455 │ UInt64 │ 1 │ + 457. │ 456 │ UInt64 │ 0 │ + 458. │ 457 │ UInt64 │ 1 │ + 459. │ 458 │ UInt64 │ 0 │ + 460. │ 459 │ UInt64 │ 1 │ + 461. │ 460 │ UInt64 │ 0 │ + 462. │ 461 │ UInt64 │ 1 │ + 463. │ 462 │ UInt64 │ 0 │ + 464. │ 463 │ UInt64 │ 1 │ + 465. │ 464 │ UInt64 │ 0 │ + 466. │ 465 │ UInt64 │ 1 │ + 467. │ 466 │ UInt64 │ 0 │ + 468. │ 467 │ UInt64 │ 1 │ + 469. │ 468 │ UInt64 │ 0 │ + 470. │ 469 │ UInt64 │ 1 │ + 471. │ 470 │ UInt64 │ 0 │ + 472. │ 471 │ UInt64 │ 1 │ + 473. │ 472 │ UInt64 │ 0 │ + 474. │ 473 │ UInt64 │ 1 │ + 475. │ 474 │ UInt64 │ 0 │ + 476. │ 475 │ UInt64 │ 1 │ + 477. │ 476 │ UInt64 │ 0 │ + 478. │ 477 │ UInt64 │ 1 │ + 479. │ 478 │ UInt64 │ 0 │ + 480. │ 479 │ UInt64 │ 1 │ + 481. │ 480 │ UInt64 │ 0 │ + 482. │ 481 │ UInt64 │ 1 │ + 483. │ 482 │ UInt64 │ 0 │ + 484. │ 483 │ UInt64 │ 1 │ + 485. │ 484 │ UInt64 │ 0 │ + 486. │ 485 │ UInt64 │ 1 │ + 487. │ 486 │ UInt64 │ 0 │ + 488. │ 487 │ UInt64 │ 1 │ + 489. │ 488 │ UInt64 │ 0 │ + 490. │ 489 │ UInt64 │ 1 │ + 491. │ 490 │ UInt64 │ 0 │ + 492. │ 491 │ UInt64 │ 1 │ + 493. │ 492 │ UInt64 │ 0 │ + 494. │ 493 │ UInt64 │ 1 │ + 495. │ 494 │ UInt64 │ 0 │ + 496. │ 495 │ UInt64 │ 1 │ + 497. │ 496 │ UInt64 │ 0 │ + 498. │ 497 │ UInt64 │ 1 │ + 499. │ 498 │ UInt64 │ 0 │ + 500. │ 499 │ UInt64 │ 1 │ + 501. │ 500 │ UInt64 │ 0 │ + 502. │ 501 │ UInt64 │ 1 │ + 503. │ 502 │ UInt64 │ 0 │ + 504. │ 503 │ UInt64 │ 1 │ + 505. │ 504 │ UInt64 │ 0 │ + 506. │ 505 │ UInt64 │ 1 │ + 507. │ 506 │ UInt64 │ 0 │ + 508. │ 507 │ UInt64 │ 1 │ + 509. │ 508 │ UInt64 │ 0 │ + 510. │ 509 │ UInt64 │ 1 │ + 511. │ 510 │ UInt64 │ 0 │ + 512. │ 511 │ UInt64 │ 1 │ + 513. │ 512 │ UInt64 │ 0 │ + 514. │ 513 │ UInt64 │ 1 │ + 515. │ 514 │ UInt64 │ 0 │ + 516. │ 515 │ UInt64 │ 1 │ + 517. │ 516 │ UInt64 │ 0 │ + 518. │ 517 │ UInt64 │ 1 │ + 519. │ 518 │ UInt64 │ 0 │ + 520. │ 519 │ UInt64 │ 1 │ + 521. │ 520 │ UInt64 │ 0 │ + 522. │ 521 │ UInt64 │ 1 │ + 523. │ 522 │ UInt64 │ 0 │ + 524. │ 523 │ UInt64 │ 1 │ + 525. │ 524 │ UInt64 │ 0 │ + 526. │ 525 │ UInt64 │ 1 │ + 527. │ 526 │ UInt64 │ 0 │ + 528. │ 527 │ UInt64 │ 1 │ + 529. │ 528 │ UInt64 │ 0 │ + 530. │ 529 │ UInt64 │ 1 │ + 531. │ 530 │ UInt64 │ 0 │ + 532. │ 531 │ UInt64 │ 1 │ + 533. │ 532 │ UInt64 │ 0 │ + 534. │ 533 │ UInt64 │ 1 │ + 535. │ 534 │ UInt64 │ 0 │ + 536. │ 535 │ UInt64 │ 1 │ + 537. │ 536 │ UInt64 │ 0 │ + 538. │ 537 │ UInt64 │ 1 │ + 539. │ 538 │ UInt64 │ 0 │ + 540. │ 539 │ UInt64 │ 1 │ + 541. │ 540 │ UInt64 │ 0 │ + 542. │ 541 │ UInt64 │ 1 │ + 543. │ 542 │ UInt64 │ 0 │ + 544. │ 543 │ UInt64 │ 1 │ + 545. │ 544 │ UInt64 │ 0 │ + 546. │ 545 │ UInt64 │ 1 │ + 547. │ 546 │ UInt64 │ 0 │ + 548. │ 547 │ UInt64 │ 1 │ + 549. │ 548 │ UInt64 │ 0 │ + 550. │ 549 │ UInt64 │ 1 │ + 551. │ 550 │ UInt64 │ 0 │ + 552. │ 551 │ UInt64 │ 1 │ + 553. │ 552 │ UInt64 │ 0 │ + 554. │ 553 │ UInt64 │ 1 │ + 555. │ 554 │ UInt64 │ 0 │ + 556. │ 555 │ UInt64 │ 1 │ + 557. │ 556 │ UInt64 │ 0 │ + 558. │ 557 │ UInt64 │ 1 │ + 559. │ 558 │ UInt64 │ 0 │ + 560. │ 559 │ UInt64 │ 1 │ + 561. │ 560 │ UInt64 │ 0 │ + 562. │ 561 │ UInt64 │ 1 │ + 563. │ 562 │ UInt64 │ 0 │ + 564. │ 563 │ UInt64 │ 1 │ + 565. │ 564 │ UInt64 │ 0 │ + 566. │ 565 │ UInt64 │ 1 │ + 567. │ 566 │ UInt64 │ 0 │ + 568. │ 567 │ UInt64 │ 1 │ + 569. │ 568 │ UInt64 │ 0 │ + 570. │ 569 │ UInt64 │ 1 │ + 571. │ 570 │ UInt64 │ 0 │ + 572. │ 571 │ UInt64 │ 1 │ + 573. │ 572 │ UInt64 │ 0 │ + 574. │ 573 │ UInt64 │ 1 │ + 575. │ 574 │ UInt64 │ 0 │ + 576. │ 575 │ UInt64 │ 1 │ + 577. │ 576 │ UInt64 │ 0 │ + 578. │ 577 │ UInt64 │ 1 │ + 579. │ 578 │ UInt64 │ 0 │ + 580. │ 579 │ UInt64 │ 1 │ + 581. │ 580 │ UInt64 │ 0 │ + 582. │ 581 │ UInt64 │ 1 │ + 583. │ 582 │ UInt64 │ 0 │ + 584. │ 583 │ UInt64 │ 1 │ + 585. │ 584 │ UInt64 │ 0 │ + 586. │ 585 │ UInt64 │ 1 │ + 587. │ 586 │ UInt64 │ 0 │ + 588. │ 587 │ UInt64 │ 1 │ + 589. │ 588 │ UInt64 │ 0 │ + 590. │ 589 │ UInt64 │ 1 │ + 591. │ 590 │ UInt64 │ 0 │ + 592. │ 591 │ UInt64 │ 1 │ + 593. │ 592 │ UInt64 │ 0 │ + 594. │ 593 │ UInt64 │ 1 │ + 595. │ 594 │ UInt64 │ 0 │ + 596. │ 595 │ UInt64 │ 1 │ + 597. │ 596 │ UInt64 │ 0 │ + 598. │ 597 │ UInt64 │ 1 │ + 599. │ 598 │ UInt64 │ 0 │ + 600. │ 599 │ UInt64 │ 1 │ + 601. │ 600 │ UInt64 │ 0 │ + 602. │ 601 │ UInt64 │ 1 │ + 603. │ 602 │ UInt64 │ 0 │ + 604. │ 603 │ UInt64 │ 1 │ + 605. │ 604 │ UInt64 │ 0 │ + 606. │ 605 │ UInt64 │ 1 │ + 607. │ 606 │ UInt64 │ 0 │ + 608. │ 607 │ UInt64 │ 1 │ + 609. │ 608 │ UInt64 │ 0 │ + 610. │ 609 │ UInt64 │ 1 │ + 611. │ 610 │ UInt64 │ 0 │ + 612. │ 611 │ UInt64 │ 1 │ + 613. │ 612 │ UInt64 │ 0 │ + 614. │ 613 │ UInt64 │ 1 │ + 615. │ 614 │ UInt64 │ 0 │ + 616. │ 615 │ UInt64 │ 1 │ + 617. │ 616 │ UInt64 │ 0 │ + 618. │ 617 │ UInt64 │ 1 │ + 619. │ 618 │ UInt64 │ 0 │ + 620. │ 619 │ UInt64 │ 1 │ + 621. │ 620 │ UInt64 │ 0 │ + 622. │ 621 │ UInt64 │ 1 │ + 623. │ 622 │ UInt64 │ 0 │ + 624. │ 623 │ UInt64 │ 1 │ + 625. │ 624 │ UInt64 │ 0 │ + 626. │ 625 │ UInt64 │ 1 │ + 627. │ 626 │ UInt64 │ 0 │ + 628. │ 627 │ UInt64 │ 1 │ + 629. │ 628 │ UInt64 │ 0 │ + 630. │ 629 │ UInt64 │ 1 │ + 631. │ 630 │ UInt64 │ 0 │ + 632. │ 631 │ UInt64 │ 1 │ + 633. │ 632 │ UInt64 │ 0 │ + 634. │ 633 │ UInt64 │ 1 │ + 635. │ 634 │ UInt64 │ 0 │ + 636. │ 635 │ UInt64 │ 1 │ + 637. │ 636 │ UInt64 │ 0 │ + 638. │ 637 │ UInt64 │ 1 │ + 639. │ 638 │ UInt64 │ 0 │ + 640. │ 639 │ UInt64 │ 1 │ + 641. │ 640 │ UInt64 │ 0 │ + 642. │ 641 │ UInt64 │ 1 │ + 643. │ 642 │ UInt64 │ 0 │ + 644. │ 643 │ UInt64 │ 1 │ + 645. │ 644 │ UInt64 │ 0 │ + 646. │ 645 │ UInt64 │ 1 │ + 647. │ 646 │ UInt64 │ 0 │ + 648. │ 647 │ UInt64 │ 1 │ + 649. │ 648 │ UInt64 │ 0 │ + 650. │ 649 │ UInt64 │ 1 │ + 651. │ 650 │ UInt64 │ 0 │ + 652. │ 651 │ UInt64 │ 1 │ + 653. │ 652 │ UInt64 │ 0 │ + 654. │ 653 │ UInt64 │ 1 │ + 655. │ 654 │ UInt64 │ 0 │ + 656. │ 655 │ UInt64 │ 1 │ + 657. │ 656 │ UInt64 │ 0 │ + 658. │ 657 │ UInt64 │ 1 │ + 659. │ 658 │ UInt64 │ 0 │ + 660. │ 659 │ UInt64 │ 1 │ + 661. │ 660 │ UInt64 │ 0 │ + 662. │ 661 │ UInt64 │ 1 │ + 663. │ 662 │ UInt64 │ 0 │ + 664. │ 663 │ UInt64 │ 1 │ + 665. │ 664 │ UInt64 │ 0 │ + 666. │ 665 │ UInt64 │ 1 │ + 667. │ 666 │ UInt64 │ 0 │ + 668. │ 667 │ UInt64 │ 1 │ + 669. │ 668 │ UInt64 │ 0 │ + 670. │ 669 │ UInt64 │ 1 │ + 671. │ 670 │ UInt64 │ 0 │ + 672. │ 671 │ UInt64 │ 1 │ + 673. │ 672 │ UInt64 │ 0 │ + 674. │ 673 │ UInt64 │ 1 │ + 675. │ 674 │ UInt64 │ 0 │ + 676. │ 675 │ UInt64 │ 1 │ + 677. │ 676 │ UInt64 │ 0 │ + 678. │ 677 │ UInt64 │ 1 │ + 679. │ 678 │ UInt64 │ 0 │ + 680. │ 679 │ UInt64 │ 1 │ + 681. │ 680 │ UInt64 │ 0 │ + 682. │ 681 │ UInt64 │ 1 │ + 683. │ 682 │ UInt64 │ 0 │ + 684. │ 683 │ UInt64 │ 1 │ + 685. │ 684 │ UInt64 │ 0 │ + 686. │ 685 │ UInt64 │ 1 │ + 687. │ 686 │ UInt64 │ 0 │ + 688. │ 687 │ UInt64 │ 1 │ + 689. │ 688 │ UInt64 │ 0 │ + 690. │ 689 │ UInt64 │ 1 │ + 691. │ 690 │ UInt64 │ 0 │ + 692. │ 691 │ UInt64 │ 1 │ + 693. │ 692 │ UInt64 │ 0 │ + 694. │ 693 │ UInt64 │ 1 │ + 695. │ 694 │ UInt64 │ 0 │ + 696. │ 695 │ UInt64 │ 1 │ + 697. │ 696 │ UInt64 │ 0 │ + 698. │ 697 │ UInt64 │ 1 │ + 699. │ 698 │ UInt64 │ 0 │ + 700. │ 699 │ UInt64 │ 1 │ + 701. │ 700 │ UInt64 │ 0 │ + 702. │ 701 │ UInt64 │ 1 │ + 703. │ 702 │ UInt64 │ 0 │ + 704. │ 703 │ UInt64 │ 1 │ + 705. │ 704 │ UInt64 │ 0 │ + 706. │ 705 │ UInt64 │ 1 │ + 707. │ 706 │ UInt64 │ 0 │ + 708. │ 707 │ UInt64 │ 1 │ + 709. │ 708 │ UInt64 │ 0 │ + 710. │ 709 │ UInt64 │ 1 │ + 711. │ 710 │ UInt64 │ 0 │ + 712. │ 711 │ UInt64 │ 1 │ + 713. │ 712 │ UInt64 │ 0 │ + 714. │ 713 │ UInt64 │ 1 │ + 715. │ 714 │ UInt64 │ 0 │ + 716. │ 715 │ UInt64 │ 1 │ + 717. │ 716 │ UInt64 │ 0 │ + 718. │ 717 │ UInt64 │ 1 │ + 719. │ 718 │ UInt64 │ 0 │ + 720. │ 719 │ UInt64 │ 1 │ + 721. │ 720 │ UInt64 │ 0 │ + 722. │ 721 │ UInt64 │ 1 │ + 723. │ 722 │ UInt64 │ 0 │ + 724. │ 723 │ UInt64 │ 1 │ + 725. │ 724 │ UInt64 │ 0 │ + 726. │ 725 │ UInt64 │ 1 │ + 727. │ 726 │ UInt64 │ 0 │ + 728. │ 727 │ UInt64 │ 1 │ + 729. │ 728 │ UInt64 │ 0 │ + 730. │ 729 │ UInt64 │ 1 │ + 731. │ 730 │ UInt64 │ 0 │ + 732. │ 731 │ UInt64 │ 1 │ + 733. │ 732 │ UInt64 │ 0 │ + 734. │ 733 │ UInt64 │ 1 │ + 735. │ 734 │ UInt64 │ 0 │ + 736. │ 735 │ UInt64 │ 1 │ + 737. │ 736 │ UInt64 │ 0 │ + 738. │ 737 │ UInt64 │ 1 │ + 739. │ 738 │ UInt64 │ 0 │ + 740. │ 739 │ UInt64 │ 1 │ + 741. │ 740 │ UInt64 │ 0 │ + 742. │ 741 │ UInt64 │ 1 │ + 743. │ 742 │ UInt64 │ 0 │ + 744. │ 743 │ UInt64 │ 1 │ + 745. │ 744 │ UInt64 │ 0 │ + 746. │ 745 │ UInt64 │ 1 │ + 747. │ 746 │ UInt64 │ 0 │ + 748. │ 747 │ UInt64 │ 1 │ + 749. │ 748 │ UInt64 │ 0 │ + 750. │ 749 │ UInt64 │ 1 │ + 751. │ 750 │ UInt64 │ 0 │ + 752. │ 751 │ UInt64 │ 1 │ + 753. │ 752 │ UInt64 │ 0 │ + 754. │ 753 │ UInt64 │ 1 │ + 755. │ 754 │ UInt64 │ 0 │ + 756. │ 755 │ UInt64 │ 1 │ + 757. │ 756 │ UInt64 │ 0 │ + 758. │ 757 │ UInt64 │ 1 │ + 759. │ 758 │ UInt64 │ 0 │ + 760. │ 759 │ UInt64 │ 1 │ + 761. │ 760 │ UInt64 │ 0 │ + 762. │ 761 │ UInt64 │ 1 │ + 763. │ 762 │ UInt64 │ 0 │ + 764. │ 763 │ UInt64 │ 1 │ + 765. │ 764 │ UInt64 │ 0 │ + 766. │ 765 │ UInt64 │ 1 │ + 767. │ 766 │ UInt64 │ 0 │ + 768. │ 767 │ UInt64 │ 1 │ + 769. │ 768 │ UInt64 │ 0 │ + 770. │ 769 │ UInt64 │ 1 │ + 771. │ 770 │ UInt64 │ 0 │ + 772. │ 771 │ UInt64 │ 1 │ + 773. │ 772 │ UInt64 │ 0 │ + 774. │ 773 │ UInt64 │ 1 │ + 775. │ 774 │ UInt64 │ 0 │ + 776. │ 775 │ UInt64 │ 1 │ + 777. │ 776 │ UInt64 │ 0 │ + 778. │ 777 │ UInt64 │ 1 │ + 779. │ 778 │ UInt64 │ 0 │ + 780. │ 779 │ UInt64 │ 1 │ + 781. │ 780 │ UInt64 │ 0 │ + 782. │ 781 │ UInt64 │ 1 │ + 783. │ 782 │ UInt64 │ 0 │ + 784. │ 783 │ UInt64 │ 1 │ + 785. │ 784 │ UInt64 │ 0 │ + 786. │ 785 │ UInt64 │ 1 │ + 787. │ 786 │ UInt64 │ 0 │ + 788. │ 787 │ UInt64 │ 1 │ + 789. │ 788 │ UInt64 │ 0 │ + 790. │ 789 │ UInt64 │ 1 │ + 791. │ 790 │ UInt64 │ 0 │ + 792. │ 791 │ UInt64 │ 1 │ + 793. │ 792 │ UInt64 │ 0 │ + 794. │ 793 │ UInt64 │ 1 │ + 795. │ 794 │ UInt64 │ 0 │ + 796. │ 795 │ UInt64 │ 1 │ + 797. │ 796 │ UInt64 │ 0 │ + 798. │ 797 │ UInt64 │ 1 │ + 799. │ 798 │ UInt64 │ 0 │ + 800. │ 799 │ UInt64 │ 1 │ + 801. │ 800 │ UInt64 │ 0 │ + 802. │ 801 │ UInt64 │ 1 │ + 803. │ 802 │ UInt64 │ 0 │ + 804. │ 803 │ UInt64 │ 1 │ + 805. │ 804 │ UInt64 │ 0 │ + 806. │ 805 │ UInt64 │ 1 │ + 807. │ 806 │ UInt64 │ 0 │ + 808. │ 807 │ UInt64 │ 1 │ + 809. │ 808 │ UInt64 │ 0 │ + 810. │ 809 │ UInt64 │ 1 │ + 811. │ 810 │ UInt64 │ 0 │ + 812. │ 811 │ UInt64 │ 1 │ + 813. │ 812 │ UInt64 │ 0 │ + 814. │ 813 │ UInt64 │ 1 │ + 815. │ 814 │ UInt64 │ 0 │ + 816. │ 815 │ UInt64 │ 1 │ + 817. │ 816 │ UInt64 │ 0 │ + 818. │ 817 │ UInt64 │ 1 │ + 819. │ 818 │ UInt64 │ 0 │ + 820. │ 819 │ UInt64 │ 1 │ + 821. │ 820 │ UInt64 │ 0 │ + 822. │ 821 │ UInt64 │ 1 │ + 823. │ 822 │ UInt64 │ 0 │ + 824. │ 823 │ UInt64 │ 1 │ + 825. │ 824 │ UInt64 │ 0 │ + 826. │ 825 │ UInt64 │ 1 │ + 827. │ 826 │ UInt64 │ 0 │ + 828. │ 827 │ UInt64 │ 1 │ + 829. │ 828 │ UInt64 │ 0 │ + 830. │ 829 │ UInt64 │ 1 │ + 831. │ 830 │ UInt64 │ 0 │ + 832. │ 831 │ UInt64 │ 1 │ + 833. │ 832 │ UInt64 │ 0 │ + 834. │ 833 │ UInt64 │ 1 │ + 835. │ 834 │ UInt64 │ 0 │ + 836. │ 835 │ UInt64 │ 1 │ + 837. │ 836 │ UInt64 │ 0 │ + 838. │ 837 │ UInt64 │ 1 │ + 839. │ 838 │ UInt64 │ 0 │ + 840. │ 839 │ UInt64 │ 1 │ + 841. │ 840 │ UInt64 │ 0 │ + 842. │ 841 │ UInt64 │ 1 │ + 843. │ 842 │ UInt64 │ 0 │ + 844. │ 843 │ UInt64 │ 1 │ + 845. │ 844 │ UInt64 │ 0 │ + 846. │ 845 │ UInt64 │ 1 │ + 847. │ 846 │ UInt64 │ 0 │ + 848. │ 847 │ UInt64 │ 1 │ + 849. │ 848 │ UInt64 │ 0 │ + 850. │ 849 │ UInt64 │ 1 │ + 851. │ 850 │ UInt64 │ 0 │ + 852. │ 851 │ UInt64 │ 1 │ + 853. │ 852 │ UInt64 │ 0 │ + 854. │ 853 │ UInt64 │ 1 │ + 855. │ 854 │ UInt64 │ 0 │ + 856. │ 855 │ UInt64 │ 1 │ + 857. │ 856 │ UInt64 │ 0 │ + 858. │ 857 │ UInt64 │ 1 │ + 859. │ 858 │ UInt64 │ 0 │ + 860. │ 859 │ UInt64 │ 1 │ + 861. │ 860 │ UInt64 │ 0 │ + 862. │ 861 │ UInt64 │ 1 │ + 863. │ 862 │ UInt64 │ 0 │ + 864. │ 863 │ UInt64 │ 1 │ + 865. │ 864 │ UInt64 │ 0 │ + 866. │ 865 │ UInt64 │ 1 │ + 867. │ 866 │ UInt64 │ 0 │ + 868. │ 867 │ UInt64 │ 1 │ + 869. │ 868 │ UInt64 │ 0 │ + 870. │ 869 │ UInt64 │ 1 │ + 871. │ 870 │ UInt64 │ 0 │ + 872. │ 871 │ UInt64 │ 1 │ + 873. │ 872 │ UInt64 │ 0 │ + 874. │ 873 │ UInt64 │ 1 │ + 875. │ 874 │ UInt64 │ 0 │ + 876. │ 875 │ UInt64 │ 1 │ + 877. │ 876 │ UInt64 │ 0 │ + 878. │ 877 │ UInt64 │ 1 │ + 879. │ 878 │ UInt64 │ 0 │ + 880. │ 879 │ UInt64 │ 1 │ + 881. │ 880 │ UInt64 │ 0 │ + 882. │ 881 │ UInt64 │ 1 │ + 883. │ 882 │ UInt64 │ 0 │ + 884. │ 883 │ UInt64 │ 1 │ + 885. │ 884 │ UInt64 │ 0 │ + 886. │ 885 │ UInt64 │ 1 │ + 887. │ 886 │ UInt64 │ 0 │ + 888. │ 887 │ UInt64 │ 1 │ + 889. │ 888 │ UInt64 │ 0 │ + 890. │ 889 │ UInt64 │ 1 │ + 891. │ 890 │ UInt64 │ 0 │ + 892. │ 891 │ UInt64 │ 1 │ + 893. │ 892 │ UInt64 │ 0 │ + 894. │ 893 │ UInt64 │ 1 │ + 895. │ 894 │ UInt64 │ 0 │ + 896. │ 895 │ UInt64 │ 1 │ + 897. │ 896 │ UInt64 │ 0 │ + 898. │ 897 │ UInt64 │ 1 │ + 899. │ 898 │ UInt64 │ 0 │ + 900. │ 899 │ UInt64 │ 1 │ + 901. │ 900 │ UInt64 │ 0 │ + 902. │ 901 │ UInt64 │ 1 │ + 903. │ 902 │ UInt64 │ 0 │ + 904. │ 903 │ UInt64 │ 1 │ + 905. │ 904 │ UInt64 │ 0 │ + 906. │ 905 │ UInt64 │ 1 │ + 907. │ 906 │ UInt64 │ 0 │ + 908. │ 907 │ UInt64 │ 1 │ + 909. │ 908 │ UInt64 │ 0 │ + 910. │ 909 │ UInt64 │ 1 │ + 911. │ 910 │ UInt64 │ 0 │ + 912. │ 911 │ UInt64 │ 1 │ + 913. │ 912 │ UInt64 │ 0 │ + 914. │ 913 │ UInt64 │ 1 │ + 915. │ 914 │ UInt64 │ 0 │ + 916. │ 915 │ UInt64 │ 1 │ + 917. │ 916 │ UInt64 │ 0 │ + 918. │ 917 │ UInt64 │ 1 │ + 919. │ 918 │ UInt64 │ 0 │ + 920. │ 919 │ UInt64 │ 1 │ + 921. │ 920 │ UInt64 │ 0 │ + 922. │ 921 │ UInt64 │ 1 │ + 923. │ 922 │ UInt64 │ 0 │ + 924. │ 923 │ UInt64 │ 1 │ + 925. │ 924 │ UInt64 │ 0 │ + 926. │ 925 │ UInt64 │ 1 │ + 927. │ 926 │ UInt64 │ 0 │ + 928. │ 927 │ UInt64 │ 1 │ + 929. │ 928 │ UInt64 │ 0 │ + 930. │ 929 │ UInt64 │ 1 │ + 931. │ 930 │ UInt64 │ 0 │ + 932. │ 931 │ UInt64 │ 1 │ + 933. │ 932 │ UInt64 │ 0 │ + 934. │ 933 │ UInt64 │ 1 │ + 935. │ 934 │ UInt64 │ 0 │ + 936. │ 935 │ UInt64 │ 1 │ + 937. │ 936 │ UInt64 │ 0 │ + 938. │ 937 │ UInt64 │ 1 │ + 939. │ 938 │ UInt64 │ 0 │ + 940. │ 939 │ UInt64 │ 1 │ + 941. │ 940 │ UInt64 │ 0 │ + 942. │ 941 │ UInt64 │ 1 │ + 943. │ 942 │ UInt64 │ 0 │ + 944. │ 943 │ UInt64 │ 1 │ + 945. │ 944 │ UInt64 │ 0 │ + 946. │ 945 │ UInt64 │ 1 │ + 947. │ 946 │ UInt64 │ 0 │ + 948. │ 947 │ UInt64 │ 1 │ + 949. │ 948 │ UInt64 │ 0 │ + 950. │ 949 │ UInt64 │ 1 │ + 951. │ 950 │ UInt64 │ 0 │ + 952. │ 951 │ UInt64 │ 1 │ + 953. │ 952 │ UInt64 │ 0 │ + 954. │ 953 │ UInt64 │ 1 │ + 955. │ 954 │ UInt64 │ 0 │ + 956. │ 955 │ UInt64 │ 1 │ + 957. │ 956 │ UInt64 │ 0 │ + 958. │ 957 │ UInt64 │ 1 │ + 959. │ 958 │ UInt64 │ 0 │ + 960. │ 959 │ UInt64 │ 1 │ + 961. │ 960 │ UInt64 │ 0 │ + 962. │ 961 │ UInt64 │ 1 │ + 963. │ 962 │ UInt64 │ 0 │ + 964. │ 963 │ UInt64 │ 1 │ + 965. │ 964 │ UInt64 │ 0 │ + 966. │ 965 │ UInt64 │ 1 │ + 967. │ 966 │ UInt64 │ 0 │ + 968. │ 967 │ UInt64 │ 1 │ + 969. │ 968 │ UInt64 │ 0 │ + 970. │ 969 │ UInt64 │ 1 │ + 971. │ 970 │ UInt64 │ 0 │ + 972. │ 971 │ UInt64 │ 1 │ + 973. │ 972 │ UInt64 │ 0 │ + 974. │ 973 │ UInt64 │ 1 │ + 975. │ 974 │ UInt64 │ 0 │ + 976. │ 975 │ UInt64 │ 1 │ + 977. │ 976 │ UInt64 │ 0 │ + 978. │ 977 │ UInt64 │ 1 │ + 979. │ 978 │ UInt64 │ 0 │ + 980. │ 979 │ UInt64 │ 1 │ + 981. │ 980 │ UInt64 │ 0 │ + 982. │ 981 │ UInt64 │ 1 │ + 983. │ 982 │ UInt64 │ 0 │ + 984. │ 983 │ UInt64 │ 1 │ + 985. │ 984 │ UInt64 │ 0 │ + 986. │ 985 │ UInt64 │ 1 │ + 987. │ 986 │ UInt64 │ 0 │ + 988. │ 987 │ UInt64 │ 1 │ + 989. │ 988 │ UInt64 │ 0 │ + 990. │ 989 │ UInt64 │ 1 │ + 991. │ 990 │ UInt64 │ 0 │ + 992. │ 991 │ UInt64 │ 1 │ + 993. │ 992 │ UInt64 │ 0 │ + 994. │ 993 │ UInt64 │ 1 │ + 995. │ 994 │ UInt64 │ 0 │ + 996. │ 995 │ UInt64 │ 1 │ + 997. │ 996 │ UInt64 │ 0 │ + 998. │ 997 │ UInt64 │ 1 │ + 999. │ 998 │ UInt64 │ 0 │ +1000. │ 999 │ UInt64 │ 1 │ + └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ + number toTypeName(number) mod(number, 2) + + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 + 100. 99 UInt64 1 + 101. 100 UInt64 0 + 102. 101 UInt64 1 + 103. 102 UInt64 0 + 104. 103 UInt64 1 + 105. 104 UInt64 0 + 106. 105 UInt64 1 + 107. 106 UInt64 0 + 108. 107 UInt64 1 + 109. 108 UInt64 0 + 110. 109 UInt64 1 + 111. 110 UInt64 0 + 112. 111 UInt64 1 + 113. 112 UInt64 0 + 114. 113 UInt64 1 + 115. 114 UInt64 0 + 116. 115 UInt64 1 + 117. 116 UInt64 0 + 118. 117 UInt64 1 + 119. 118 UInt64 0 + 120. 119 UInt64 1 + 121. 120 UInt64 0 + 122. 121 UInt64 1 + 123. 122 UInt64 0 + 124. 123 UInt64 1 + 125. 124 UInt64 0 + 126. 125 UInt64 1 + 127. 126 UInt64 0 + 128. 127 UInt64 1 + 129. 128 UInt64 0 + 130. 129 UInt64 1 + 131. 130 UInt64 0 + 132. 131 UInt64 1 + 133. 132 UInt64 0 + 134. 133 UInt64 1 + 135. 134 UInt64 0 + 136. 135 UInt64 1 + 137. 136 UInt64 0 + 138. 137 UInt64 1 + 139. 138 UInt64 0 + 140. 139 UInt64 1 + 141. 140 UInt64 0 + 142. 141 UInt64 1 + 143. 142 UInt64 0 + 144. 143 UInt64 1 + 145. 144 UInt64 0 + 146. 145 UInt64 1 + 147. 146 UInt64 0 + 148. 147 UInt64 1 + 149. 148 UInt64 0 + 150. 149 UInt64 1 + 151. 150 UInt64 0 + 152. 151 UInt64 1 + 153. 152 UInt64 0 + 154. 153 UInt64 1 + 155. 154 UInt64 0 + 156. 155 UInt64 1 + 157. 156 UInt64 0 + 158. 157 UInt64 1 + 159. 158 UInt64 0 + 160. 159 UInt64 1 + 161. 160 UInt64 0 + 162. 161 UInt64 1 + 163. 162 UInt64 0 + 164. 163 UInt64 1 + 165. 164 UInt64 0 + 166. 165 UInt64 1 + 167. 166 UInt64 0 + 168. 167 UInt64 1 + 169. 168 UInt64 0 + 170. 169 UInt64 1 + 171. 170 UInt64 0 + 172. 171 UInt64 1 + 173. 172 UInt64 0 + 174. 173 UInt64 1 + 175. 174 UInt64 0 + 176. 175 UInt64 1 + 177. 176 UInt64 0 + 178. 177 UInt64 1 + 179. 178 UInt64 0 + 180. 179 UInt64 1 + 181. 180 UInt64 0 + 182. 181 UInt64 1 + 183. 182 UInt64 0 + 184. 183 UInt64 1 + 185. 184 UInt64 0 + 186. 185 UInt64 1 + 187. 186 UInt64 0 + 188. 187 UInt64 1 + 189. 188 UInt64 0 + 190. 189 UInt64 1 + 191. 190 UInt64 0 + 192. 191 UInt64 1 + 193. 192 UInt64 0 + 194. 193 UInt64 1 + 195. 194 UInt64 0 + 196. 195 UInt64 1 + 197. 196 UInt64 0 + 198. 197 UInt64 1 + 199. 198 UInt64 0 + 200. 199 UInt64 1 + 201. 200 UInt64 0 + 202. 201 UInt64 1 + 203. 202 UInt64 0 + 204. 203 UInt64 1 + 205. 204 UInt64 0 + 206. 205 UInt64 1 + 207. 206 UInt64 0 + 208. 207 UInt64 1 + 209. 208 UInt64 0 + 210. 209 UInt64 1 + 211. 210 UInt64 0 + 212. 211 UInt64 1 + 213. 212 UInt64 0 + 214. 213 UInt64 1 + 215. 214 UInt64 0 + 216. 215 UInt64 1 + 217. 216 UInt64 0 + 218. 217 UInt64 1 + 219. 218 UInt64 0 + 220. 219 UInt64 1 + 221. 220 UInt64 0 + 222. 221 UInt64 1 + 223. 222 UInt64 0 + 224. 223 UInt64 1 + 225. 224 UInt64 0 + 226. 225 UInt64 1 + 227. 226 UInt64 0 + 228. 227 UInt64 1 + 229. 228 UInt64 0 + 230. 229 UInt64 1 + 231. 230 UInt64 0 + 232. 231 UInt64 1 + 233. 232 UInt64 0 + 234. 233 UInt64 1 + 235. 234 UInt64 0 + 236. 235 UInt64 1 + 237. 236 UInt64 0 + 238. 237 UInt64 1 + 239. 238 UInt64 0 + 240. 239 UInt64 1 + 241. 240 UInt64 0 + 242. 241 UInt64 1 + 243. 242 UInt64 0 + 244. 243 UInt64 1 + 245. 244 UInt64 0 + 246. 245 UInt64 1 + 247. 246 UInt64 0 + 248. 247 UInt64 1 + 249. 248 UInt64 0 + 250. 249 UInt64 1 + 251. 250 UInt64 0 + 252. 251 UInt64 1 + 253. 252 UInt64 0 + 254. 253 UInt64 1 + 255. 254 UInt64 0 + 256. 255 UInt64 1 + 257. 256 UInt64 0 + 258. 257 UInt64 1 + 259. 258 UInt64 0 + 260. 259 UInt64 1 + 261. 260 UInt64 0 + 262. 261 UInt64 1 + 263. 262 UInt64 0 + 264. 263 UInt64 1 + 265. 264 UInt64 0 + 266. 265 UInt64 1 + 267. 266 UInt64 0 + 268. 267 UInt64 1 + 269. 268 UInt64 0 + 270. 269 UInt64 1 + 271. 270 UInt64 0 + 272. 271 UInt64 1 + 273. 272 UInt64 0 + 274. 273 UInt64 1 + 275. 274 UInt64 0 + 276. 275 UInt64 1 + 277. 276 UInt64 0 + 278. 277 UInt64 1 + 279. 278 UInt64 0 + 280. 279 UInt64 1 + 281. 280 UInt64 0 + 282. 281 UInt64 1 + 283. 282 UInt64 0 + 284. 283 UInt64 1 + 285. 284 UInt64 0 + 286. 285 UInt64 1 + 287. 286 UInt64 0 + 288. 287 UInt64 1 + 289. 288 UInt64 0 + 290. 289 UInt64 1 + 291. 290 UInt64 0 + 292. 291 UInt64 1 + 293. 292 UInt64 0 + 294. 293 UInt64 1 + 295. 294 UInt64 0 + 296. 295 UInt64 1 + 297. 296 UInt64 0 + 298. 297 UInt64 1 + 299. 298 UInt64 0 + 300. 299 UInt64 1 + 301. 300 UInt64 0 + 302. 301 UInt64 1 + 303. 302 UInt64 0 + 304. 303 UInt64 1 + 305. 304 UInt64 0 + 306. 305 UInt64 1 + 307. 306 UInt64 0 + 308. 307 UInt64 1 + 309. 308 UInt64 0 + 310. 309 UInt64 1 + 311. 310 UInt64 0 + 312. 311 UInt64 1 + 313. 312 UInt64 0 + 314. 313 UInt64 1 + 315. 314 UInt64 0 + 316. 315 UInt64 1 + 317. 316 UInt64 0 + 318. 317 UInt64 1 + 319. 318 UInt64 0 + 320. 319 UInt64 1 + 321. 320 UInt64 0 + 322. 321 UInt64 1 + 323. 322 UInt64 0 + 324. 323 UInt64 1 + 325. 324 UInt64 0 + 326. 325 UInt64 1 + 327. 326 UInt64 0 + 328. 327 UInt64 1 + 329. 328 UInt64 0 + 330. 329 UInt64 1 + 331. 330 UInt64 0 + 332. 331 UInt64 1 + 333. 332 UInt64 0 + 334. 333 UInt64 1 + 335. 334 UInt64 0 + 336. 335 UInt64 1 + 337. 336 UInt64 0 + 338. 337 UInt64 1 + 339. 338 UInt64 0 + 340. 339 UInt64 1 + 341. 340 UInt64 0 + 342. 341 UInt64 1 + 343. 342 UInt64 0 + 344. 343 UInt64 1 + 345. 344 UInt64 0 + 346. 345 UInt64 1 + 347. 346 UInt64 0 + 348. 347 UInt64 1 + 349. 348 UInt64 0 + 350. 349 UInt64 1 + 351. 350 UInt64 0 + 352. 351 UInt64 1 + 353. 352 UInt64 0 + 354. 353 UInt64 1 + 355. 354 UInt64 0 + 356. 355 UInt64 1 + 357. 356 UInt64 0 + 358. 357 UInt64 1 + 359. 358 UInt64 0 + 360. 359 UInt64 1 + 361. 360 UInt64 0 + 362. 361 UInt64 1 + 363. 362 UInt64 0 + 364. 363 UInt64 1 + 365. 364 UInt64 0 + 366. 365 UInt64 1 + 367. 366 UInt64 0 + 368. 367 UInt64 1 + 369. 368 UInt64 0 + 370. 369 UInt64 1 + 371. 370 UInt64 0 + 372. 371 UInt64 1 + 373. 372 UInt64 0 + 374. 373 UInt64 1 + 375. 374 UInt64 0 + 376. 375 UInt64 1 + 377. 376 UInt64 0 + 378. 377 UInt64 1 + 379. 378 UInt64 0 + 380. 379 UInt64 1 + 381. 380 UInt64 0 + 382. 381 UInt64 1 + 383. 382 UInt64 0 + 384. 383 UInt64 1 + 385. 384 UInt64 0 + 386. 385 UInt64 1 + 387. 386 UInt64 0 + 388. 387 UInt64 1 + 389. 388 UInt64 0 + 390. 389 UInt64 1 + 391. 390 UInt64 0 + 392. 391 UInt64 1 + 393. 392 UInt64 0 + 394. 393 UInt64 1 + 395. 394 UInt64 0 + 396. 395 UInt64 1 + 397. 396 UInt64 0 + 398. 397 UInt64 1 + 399. 398 UInt64 0 + 400. 399 UInt64 1 + 401. 400 UInt64 0 + 402. 401 UInt64 1 + 403. 402 UInt64 0 + 404. 403 UInt64 1 + 405. 404 UInt64 0 + 406. 405 UInt64 1 + 407. 406 UInt64 0 + 408. 407 UInt64 1 + 409. 408 UInt64 0 + 410. 409 UInt64 1 + 411. 410 UInt64 0 + 412. 411 UInt64 1 + 413. 412 UInt64 0 + 414. 413 UInt64 1 + 415. 414 UInt64 0 + 416. 415 UInt64 1 + 417. 416 UInt64 0 + 418. 417 UInt64 1 + 419. 418 UInt64 0 + 420. 419 UInt64 1 + 421. 420 UInt64 0 + 422. 421 UInt64 1 + 423. 422 UInt64 0 + 424. 423 UInt64 1 + 425. 424 UInt64 0 + 426. 425 UInt64 1 + 427. 426 UInt64 0 + 428. 427 UInt64 1 + 429. 428 UInt64 0 + 430. 429 UInt64 1 + 431. 430 UInt64 0 + 432. 431 UInt64 1 + 433. 432 UInt64 0 + 434. 433 UInt64 1 + 435. 434 UInt64 0 + 436. 435 UInt64 1 + 437. 436 UInt64 0 + 438. 437 UInt64 1 + 439. 438 UInt64 0 + 440. 439 UInt64 1 + 441. 440 UInt64 0 + 442. 441 UInt64 1 + 443. 442 UInt64 0 + 444. 443 UInt64 1 + 445. 444 UInt64 0 + 446. 445 UInt64 1 + 447. 446 UInt64 0 + 448. 447 UInt64 1 + 449. 448 UInt64 0 + 450. 449 UInt64 1 + 451. 450 UInt64 0 + 452. 451 UInt64 1 + 453. 452 UInt64 0 + 454. 453 UInt64 1 + 455. 454 UInt64 0 + 456. 455 UInt64 1 + 457. 456 UInt64 0 + 458. 457 UInt64 1 + 459. 458 UInt64 0 + 460. 459 UInt64 1 + 461. 460 UInt64 0 + 462. 461 UInt64 1 + 463. 462 UInt64 0 + 464. 463 UInt64 1 + 465. 464 UInt64 0 + 466. 465 UInt64 1 + 467. 466 UInt64 0 + 468. 467 UInt64 1 + 469. 468 UInt64 0 + 470. 469 UInt64 1 + 471. 470 UInt64 0 + 472. 471 UInt64 1 + 473. 472 UInt64 0 + 474. 473 UInt64 1 + 475. 474 UInt64 0 + 476. 475 UInt64 1 + 477. 476 UInt64 0 + 478. 477 UInt64 1 + 479. 478 UInt64 0 + 480. 479 UInt64 1 + 481. 480 UInt64 0 + 482. 481 UInt64 1 + 483. 482 UInt64 0 + 484. 483 UInt64 1 + 485. 484 UInt64 0 + 486. 485 UInt64 1 + 487. 486 UInt64 0 + 488. 487 UInt64 1 + 489. 488 UInt64 0 + 490. 489 UInt64 1 + 491. 490 UInt64 0 + 492. 491 UInt64 1 + 493. 492 UInt64 0 + 494. 493 UInt64 1 + 495. 494 UInt64 0 + 496. 495 UInt64 1 + 497. 496 UInt64 0 + 498. 497 UInt64 1 + 499. 498 UInt64 0 + 500. 499 UInt64 1 + 501. 500 UInt64 0 + 502. 501 UInt64 1 + 503. 502 UInt64 0 + 504. 503 UInt64 1 + 505. 504 UInt64 0 + 506. 505 UInt64 1 + 507. 506 UInt64 0 + 508. 507 UInt64 1 + 509. 508 UInt64 0 + 510. 509 UInt64 1 + 511. 510 UInt64 0 + 512. 511 UInt64 1 + 513. 512 UInt64 0 + 514. 513 UInt64 1 + 515. 514 UInt64 0 + 516. 515 UInt64 1 + 517. 516 UInt64 0 + 518. 517 UInt64 1 + 519. 518 UInt64 0 + 520. 519 UInt64 1 + 521. 520 UInt64 0 + 522. 521 UInt64 1 + 523. 522 UInt64 0 + 524. 523 UInt64 1 + 525. 524 UInt64 0 + 526. 525 UInt64 1 + 527. 526 UInt64 0 + 528. 527 UInt64 1 + 529. 528 UInt64 0 + 530. 529 UInt64 1 + 531. 530 UInt64 0 + 532. 531 UInt64 1 + 533. 532 UInt64 0 + 534. 533 UInt64 1 + 535. 534 UInt64 0 + 536. 535 UInt64 1 + 537. 536 UInt64 0 + 538. 537 UInt64 1 + 539. 538 UInt64 0 + 540. 539 UInt64 1 + 541. 540 UInt64 0 + 542. 541 UInt64 1 + 543. 542 UInt64 0 + 544. 543 UInt64 1 + 545. 544 UInt64 0 + 546. 545 UInt64 1 + 547. 546 UInt64 0 + 548. 547 UInt64 1 + 549. 548 UInt64 0 + 550. 549 UInt64 1 + 551. 550 UInt64 0 + 552. 551 UInt64 1 + 553. 552 UInt64 0 + 554. 553 UInt64 1 + 555. 554 UInt64 0 + 556. 555 UInt64 1 + 557. 556 UInt64 0 + 558. 557 UInt64 1 + 559. 558 UInt64 0 + 560. 559 UInt64 1 + 561. 560 UInt64 0 + 562. 561 UInt64 1 + 563. 562 UInt64 0 + 564. 563 UInt64 1 + 565. 564 UInt64 0 + 566. 565 UInt64 1 + 567. 566 UInt64 0 + 568. 567 UInt64 1 + 569. 568 UInt64 0 + 570. 569 UInt64 1 + 571. 570 UInt64 0 + 572. 571 UInt64 1 + 573. 572 UInt64 0 + 574. 573 UInt64 1 + 575. 574 UInt64 0 + 576. 575 UInt64 1 + 577. 576 UInt64 0 + 578. 577 UInt64 1 + 579. 578 UInt64 0 + 580. 579 UInt64 1 + 581. 580 UInt64 0 + 582. 581 UInt64 1 + 583. 582 UInt64 0 + 584. 583 UInt64 1 + 585. 584 UInt64 0 + 586. 585 UInt64 1 + 587. 586 UInt64 0 + 588. 587 UInt64 1 + 589. 588 UInt64 0 + 590. 589 UInt64 1 + 591. 590 UInt64 0 + 592. 591 UInt64 1 + 593. 592 UInt64 0 + 594. 593 UInt64 1 + 595. 594 UInt64 0 + 596. 595 UInt64 1 + 597. 596 UInt64 0 + 598. 597 UInt64 1 + 599. 598 UInt64 0 + 600. 599 UInt64 1 + 601. 600 UInt64 0 + 602. 601 UInt64 1 + 603. 602 UInt64 0 + 604. 603 UInt64 1 + 605. 604 UInt64 0 + 606. 605 UInt64 1 + 607. 606 UInt64 0 + 608. 607 UInt64 1 + 609. 608 UInt64 0 + 610. 609 UInt64 1 + 611. 610 UInt64 0 + 612. 611 UInt64 1 + 613. 612 UInt64 0 + 614. 613 UInt64 1 + 615. 614 UInt64 0 + 616. 615 UInt64 1 + 617. 616 UInt64 0 + 618. 617 UInt64 1 + 619. 618 UInt64 0 + 620. 619 UInt64 1 + 621. 620 UInt64 0 + 622. 621 UInt64 1 + 623. 622 UInt64 0 + 624. 623 UInt64 1 + 625. 624 UInt64 0 + 626. 625 UInt64 1 + 627. 626 UInt64 0 + 628. 627 UInt64 1 + 629. 628 UInt64 0 + 630. 629 UInt64 1 + 631. 630 UInt64 0 + 632. 631 UInt64 1 + 633. 632 UInt64 0 + 634. 633 UInt64 1 + 635. 634 UInt64 0 + 636. 635 UInt64 1 + 637. 636 UInt64 0 + 638. 637 UInt64 1 + 639. 638 UInt64 0 + 640. 639 UInt64 1 + 641. 640 UInt64 0 + 642. 641 UInt64 1 + 643. 642 UInt64 0 + 644. 643 UInt64 1 + 645. 644 UInt64 0 + 646. 645 UInt64 1 + 647. 646 UInt64 0 + 648. 647 UInt64 1 + 649. 648 UInt64 0 + 650. 649 UInt64 1 + 651. 650 UInt64 0 + 652. 651 UInt64 1 + 653. 652 UInt64 0 + 654. 653 UInt64 1 + 655. 654 UInt64 0 + 656. 655 UInt64 1 + 657. 656 UInt64 0 + 658. 657 UInt64 1 + 659. 658 UInt64 0 + 660. 659 UInt64 1 + 661. 660 UInt64 0 + 662. 661 UInt64 1 + 663. 662 UInt64 0 + 664. 663 UInt64 1 + 665. 664 UInt64 0 + 666. 665 UInt64 1 + 667. 666 UInt64 0 + 668. 667 UInt64 1 + 669. 668 UInt64 0 + 670. 669 UInt64 1 + 671. 670 UInt64 0 + 672. 671 UInt64 1 + 673. 672 UInt64 0 + 674. 673 UInt64 1 + 675. 674 UInt64 0 + 676. 675 UInt64 1 + 677. 676 UInt64 0 + 678. 677 UInt64 1 + 679. 678 UInt64 0 + 680. 679 UInt64 1 + 681. 680 UInt64 0 + 682. 681 UInt64 1 + 683. 682 UInt64 0 + 684. 683 UInt64 1 + 685. 684 UInt64 0 + 686. 685 UInt64 1 + 687. 686 UInt64 0 + 688. 687 UInt64 1 + 689. 688 UInt64 0 + 690. 689 UInt64 1 + 691. 690 UInt64 0 + 692. 691 UInt64 1 + 693. 692 UInt64 0 + 694. 693 UInt64 1 + 695. 694 UInt64 0 + 696. 695 UInt64 1 + 697. 696 UInt64 0 + 698. 697 UInt64 1 + 699. 698 UInt64 0 + 700. 699 UInt64 1 + 701. 700 UInt64 0 + 702. 701 UInt64 1 + 703. 702 UInt64 0 + 704. 703 UInt64 1 + 705. 704 UInt64 0 + 706. 705 UInt64 1 + 707. 706 UInt64 0 + 708. 707 UInt64 1 + 709. 708 UInt64 0 + 710. 709 UInt64 1 + 711. 710 UInt64 0 + 712. 711 UInt64 1 + 713. 712 UInt64 0 + 714. 713 UInt64 1 + 715. 714 UInt64 0 + 716. 715 UInt64 1 + 717. 716 UInt64 0 + 718. 717 UInt64 1 + 719. 718 UInt64 0 + 720. 719 UInt64 1 + 721. 720 UInt64 0 + 722. 721 UInt64 1 + 723. 722 UInt64 0 + 724. 723 UInt64 1 + 725. 724 UInt64 0 + 726. 725 UInt64 1 + 727. 726 UInt64 0 + 728. 727 UInt64 1 + 729. 728 UInt64 0 + 730. 729 UInt64 1 + 731. 730 UInt64 0 + 732. 731 UInt64 1 + 733. 732 UInt64 0 + 734. 733 UInt64 1 + 735. 734 UInt64 0 + 736. 735 UInt64 1 + 737. 736 UInt64 0 + 738. 737 UInt64 1 + 739. 738 UInt64 0 + 740. 739 UInt64 1 + 741. 740 UInt64 0 + 742. 741 UInt64 1 + 743. 742 UInt64 0 + 744. 743 UInt64 1 + 745. 744 UInt64 0 + 746. 745 UInt64 1 + 747. 746 UInt64 0 + 748. 747 UInt64 1 + 749. 748 UInt64 0 + 750. 749 UInt64 1 + 751. 750 UInt64 0 + 752. 751 UInt64 1 + 753. 752 UInt64 0 + 754. 753 UInt64 1 + 755. 754 UInt64 0 + 756. 755 UInt64 1 + 757. 756 UInt64 0 + 758. 757 UInt64 1 + 759. 758 UInt64 0 + 760. 759 UInt64 1 + 761. 760 UInt64 0 + 762. 761 UInt64 1 + 763. 762 UInt64 0 + 764. 763 UInt64 1 + 765. 764 UInt64 0 + 766. 765 UInt64 1 + 767. 766 UInt64 0 + 768. 767 UInt64 1 + 769. 768 UInt64 0 + 770. 769 UInt64 1 + 771. 770 UInt64 0 + 772. 771 UInt64 1 + 773. 772 UInt64 0 + 774. 773 UInt64 1 + 775. 774 UInt64 0 + 776. 775 UInt64 1 + 777. 776 UInt64 0 + 778. 777 UInt64 1 + 779. 778 UInt64 0 + 780. 779 UInt64 1 + 781. 780 UInt64 0 + 782. 781 UInt64 1 + 783. 782 UInt64 0 + 784. 783 UInt64 1 + 785. 784 UInt64 0 + 786. 785 UInt64 1 + 787. 786 UInt64 0 + 788. 787 UInt64 1 + 789. 788 UInt64 0 + 790. 789 UInt64 1 + 791. 790 UInt64 0 + 792. 791 UInt64 1 + 793. 792 UInt64 0 + 794. 793 UInt64 1 + 795. 794 UInt64 0 + 796. 795 UInt64 1 + 797. 796 UInt64 0 + 798. 797 UInt64 1 + 799. 798 UInt64 0 + 800. 799 UInt64 1 + 801. 800 UInt64 0 + 802. 801 UInt64 1 + 803. 802 UInt64 0 + 804. 803 UInt64 1 + 805. 804 UInt64 0 + 806. 805 UInt64 1 + 807. 806 UInt64 0 + 808. 807 UInt64 1 + 809. 808 UInt64 0 + 810. 809 UInt64 1 + 811. 810 UInt64 0 + 812. 811 UInt64 1 + 813. 812 UInt64 0 + 814. 813 UInt64 1 + 815. 814 UInt64 0 + 816. 815 UInt64 1 + 817. 816 UInt64 0 + 818. 817 UInt64 1 + 819. 818 UInt64 0 + 820. 819 UInt64 1 + 821. 820 UInt64 0 + 822. 821 UInt64 1 + 823. 822 UInt64 0 + 824. 823 UInt64 1 + 825. 824 UInt64 0 + 826. 825 UInt64 1 + 827. 826 UInt64 0 + 828. 827 UInt64 1 + 829. 828 UInt64 0 + 830. 829 UInt64 1 + 831. 830 UInt64 0 + 832. 831 UInt64 1 + 833. 832 UInt64 0 + 834. 833 UInt64 1 + 835. 834 UInt64 0 + 836. 835 UInt64 1 + 837. 836 UInt64 0 + 838. 837 UInt64 1 + 839. 838 UInt64 0 + 840. 839 UInt64 1 + 841. 840 UInt64 0 + 842. 841 UInt64 1 + 843. 842 UInt64 0 + 844. 843 UInt64 1 + 845. 844 UInt64 0 + 846. 845 UInt64 1 + 847. 846 UInt64 0 + 848. 847 UInt64 1 + 849. 848 UInt64 0 + 850. 849 UInt64 1 + 851. 850 UInt64 0 + 852. 851 UInt64 1 + 853. 852 UInt64 0 + 854. 853 UInt64 1 + 855. 854 UInt64 0 + 856. 855 UInt64 1 + 857. 856 UInt64 0 + 858. 857 UInt64 1 + 859. 858 UInt64 0 + 860. 859 UInt64 1 + 861. 860 UInt64 0 + 862. 861 UInt64 1 + 863. 862 UInt64 0 + 864. 863 UInt64 1 + 865. 864 UInt64 0 + 866. 865 UInt64 1 + 867. 866 UInt64 0 + 868. 867 UInt64 1 + 869. 868 UInt64 0 + 870. 869 UInt64 1 + 871. 870 UInt64 0 + 872. 871 UInt64 1 + 873. 872 UInt64 0 + 874. 873 UInt64 1 + 875. 874 UInt64 0 + 876. 875 UInt64 1 + 877. 876 UInt64 0 + 878. 877 UInt64 1 + 879. 878 UInt64 0 + 880. 879 UInt64 1 + 881. 880 UInt64 0 + 882. 881 UInt64 1 + 883. 882 UInt64 0 + 884. 883 UInt64 1 + 885. 884 UInt64 0 + 886. 885 UInt64 1 + 887. 886 UInt64 0 + 888. 887 UInt64 1 + 889. 888 UInt64 0 + 890. 889 UInt64 1 + 891. 890 UInt64 0 + 892. 891 UInt64 1 + 893. 892 UInt64 0 + 894. 893 UInt64 1 + 895. 894 UInt64 0 + 896. 895 UInt64 1 + 897. 896 UInt64 0 + 898. 897 UInt64 1 + 899. 898 UInt64 0 + 900. 899 UInt64 1 + 901. 900 UInt64 0 + 902. 901 UInt64 1 + 903. 902 UInt64 0 + 904. 903 UInt64 1 + 905. 904 UInt64 0 + 906. 905 UInt64 1 + 907. 906 UInt64 0 + 908. 907 UInt64 1 + 909. 908 UInt64 0 + 910. 909 UInt64 1 + 911. 910 UInt64 0 + 912. 911 UInt64 1 + 913. 912 UInt64 0 + 914. 913 UInt64 1 + 915. 914 UInt64 0 + 916. 915 UInt64 1 + 917. 916 UInt64 0 + 918. 917 UInt64 1 + 919. 918 UInt64 0 + 920. 919 UInt64 1 + 921. 920 UInt64 0 + 922. 921 UInt64 1 + 923. 922 UInt64 0 + 924. 923 UInt64 1 + 925. 924 UInt64 0 + 926. 925 UInt64 1 + 927. 926 UInt64 0 + 928. 927 UInt64 1 + 929. 928 UInt64 0 + 930. 929 UInt64 1 + 931. 930 UInt64 0 + 932. 931 UInt64 1 + 933. 932 UInt64 0 + 934. 933 UInt64 1 + 935. 934 UInt64 0 + 936. 935 UInt64 1 + 937. 936 UInt64 0 + 938. 937 UInt64 1 + 939. 938 UInt64 0 + 940. 939 UInt64 1 + 941. 940 UInt64 0 + 942. 941 UInt64 1 + 943. 942 UInt64 0 + 944. 943 UInt64 1 + 945. 944 UInt64 0 + 946. 945 UInt64 1 + 947. 946 UInt64 0 + 948. 947 UInt64 1 + 949. 948 UInt64 0 + 950. 949 UInt64 1 + 951. 950 UInt64 0 + 952. 951 UInt64 1 + 953. 952 UInt64 0 + 954. 953 UInt64 1 + 955. 954 UInt64 0 + 956. 955 UInt64 1 + 957. 956 UInt64 0 + 958. 957 UInt64 1 + 959. 958 UInt64 0 + 960. 959 UInt64 1 + 961. 960 UInt64 0 + 962. 961 UInt64 1 + 963. 962 UInt64 0 + 964. 963 UInt64 1 + 965. 964 UInt64 0 + 966. 965 UInt64 1 + 967. 966 UInt64 0 + 968. 967 UInt64 1 + 969. 968 UInt64 0 + 970. 969 UInt64 1 + 971. 970 UInt64 0 + 972. 971 UInt64 1 + 973. 972 UInt64 0 + 974. 973 UInt64 1 + 975. 974 UInt64 0 + 976. 975 UInt64 1 + 977. 976 UInt64 0 + 978. 977 UInt64 1 + 979. 978 UInt64 0 + 980. 979 UInt64 1 + 981. 980 UInt64 0 + 982. 981 UInt64 1 + 983. 982 UInt64 0 + 984. 983 UInt64 1 + 985. 984 UInt64 0 + 986. 985 UInt64 1 + 987. 986 UInt64 0 + 988. 987 UInt64 1 + 989. 988 UInt64 0 + 990. 989 UInt64 1 + 991. 990 UInt64 0 + 992. 991 UInt64 1 + 993. 992 UInt64 0 + 994. 993 UInt64 1 + 995. 994 UInt64 0 + 996. 995 UInt64 1 + 997. 996 UInt64 0 + 998. 997 UInt64 1 + 999. 998 UInt64 0 +1000. 999 UInt64 1 + + number toTypeName(number) mod(number, 2) + + number toTypeName(number) mod(number, 2) + + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 + 100. 99 UInt64 1 + 101. 100 UInt64 0 + 102. 101 UInt64 1 + 103. 102 UInt64 0 + 104. 103 UInt64 1 + 105. 104 UInt64 0 + 106. 105 UInt64 1 + 107. 106 UInt64 0 + 108. 107 UInt64 1 + 109. 108 UInt64 0 + 110. 109 UInt64 1 + 111. 110 UInt64 0 + 112. 111 UInt64 1 + 113. 112 UInt64 0 + 114. 113 UInt64 1 + 115. 114 UInt64 0 + 116. 115 UInt64 1 + 117. 116 UInt64 0 + 118. 117 UInt64 1 + 119. 118 UInt64 0 + 120. 119 UInt64 1 + 121. 120 UInt64 0 + 122. 121 UInt64 1 + 123. 122 UInt64 0 + 124. 123 UInt64 1 + 125. 124 UInt64 0 + 126. 125 UInt64 1 + 127. 126 UInt64 0 + 128. 127 UInt64 1 + 129. 128 UInt64 0 + 130. 129 UInt64 1 + 131. 130 UInt64 0 + 132. 131 UInt64 1 + 133. 132 UInt64 0 + 134. 133 UInt64 1 + 135. 134 UInt64 0 + 136. 135 UInt64 1 + 137. 136 UInt64 0 + 138. 137 UInt64 1 + 139. 138 UInt64 0 + 140. 139 UInt64 1 + 141. 140 UInt64 0 + 142. 141 UInt64 1 + 143. 142 UInt64 0 + 144. 143 UInt64 1 + 145. 144 UInt64 0 + 146. 145 UInt64 1 + 147. 146 UInt64 0 + 148. 147 UInt64 1 + 149. 148 UInt64 0 + 150. 149 UInt64 1 + 151. 150 UInt64 0 + 152. 151 UInt64 1 + 153. 152 UInt64 0 + 154. 153 UInt64 1 + 155. 154 UInt64 0 + 156. 155 UInt64 1 + 157. 156 UInt64 0 + 158. 157 UInt64 1 + 159. 158 UInt64 0 + 160. 159 UInt64 1 + 161. 160 UInt64 0 + 162. 161 UInt64 1 + 163. 162 UInt64 0 + 164. 163 UInt64 1 + 165. 164 UInt64 0 + 166. 165 UInt64 1 + 167. 166 UInt64 0 + 168. 167 UInt64 1 + 169. 168 UInt64 0 + 170. 169 UInt64 1 + 171. 170 UInt64 0 + 172. 171 UInt64 1 + 173. 172 UInt64 0 + 174. 173 UInt64 1 + 175. 174 UInt64 0 + 176. 175 UInt64 1 + 177. 176 UInt64 0 + 178. 177 UInt64 1 + 179. 178 UInt64 0 + 180. 179 UInt64 1 + 181. 180 UInt64 0 + 182. 181 UInt64 1 + 183. 182 UInt64 0 + 184. 183 UInt64 1 + 185. 184 UInt64 0 + 186. 185 UInt64 1 + 187. 186 UInt64 0 + 188. 187 UInt64 1 + 189. 188 UInt64 0 + 190. 189 UInt64 1 + 191. 190 UInt64 0 + 192. 191 UInt64 1 + 193. 192 UInt64 0 + 194. 193 UInt64 1 + 195. 194 UInt64 0 + 196. 195 UInt64 1 + 197. 196 UInt64 0 + 198. 197 UInt64 1 + 199. 198 UInt64 0 + 200. 199 UInt64 1 + 201. 200 UInt64 0 + 202. 201 UInt64 1 + 203. 202 UInt64 0 + 204. 203 UInt64 1 + 205. 204 UInt64 0 + 206. 205 UInt64 1 + 207. 206 UInt64 0 + 208. 207 UInt64 1 + 209. 208 UInt64 0 + 210. 209 UInt64 1 + 211. 210 UInt64 0 + 212. 211 UInt64 1 + 213. 212 UInt64 0 + 214. 213 UInt64 1 + 215. 214 UInt64 0 + 216. 215 UInt64 1 + 217. 216 UInt64 0 + 218. 217 UInt64 1 + 219. 218 UInt64 0 + 220. 219 UInt64 1 + 221. 220 UInt64 0 + 222. 221 UInt64 1 + 223. 222 UInt64 0 + 224. 223 UInt64 1 + 225. 224 UInt64 0 + 226. 225 UInt64 1 + 227. 226 UInt64 0 + 228. 227 UInt64 1 + 229. 228 UInt64 0 + 230. 229 UInt64 1 + 231. 230 UInt64 0 + 232. 231 UInt64 1 + 233. 232 UInt64 0 + 234. 233 UInt64 1 + 235. 234 UInt64 0 + 236. 235 UInt64 1 + 237. 236 UInt64 0 + 238. 237 UInt64 1 + 239. 238 UInt64 0 + 240. 239 UInt64 1 + 241. 240 UInt64 0 + 242. 241 UInt64 1 + 243. 242 UInt64 0 + 244. 243 UInt64 1 + 245. 244 UInt64 0 + 246. 245 UInt64 1 + 247. 246 UInt64 0 + 248. 247 UInt64 1 + 249. 248 UInt64 0 + 250. 249 UInt64 1 + 251. 250 UInt64 0 + 252. 251 UInt64 1 + 253. 252 UInt64 0 + 254. 253 UInt64 1 + 255. 254 UInt64 0 + 256. 255 UInt64 1 + 257. 256 UInt64 0 + 258. 257 UInt64 1 + 259. 258 UInt64 0 + 260. 259 UInt64 1 + 261. 260 UInt64 0 + 262. 261 UInt64 1 + 263. 262 UInt64 0 + 264. 263 UInt64 1 + 265. 264 UInt64 0 + 266. 265 UInt64 1 + 267. 266 UInt64 0 + 268. 267 UInt64 1 + 269. 268 UInt64 0 + 270. 269 UInt64 1 + 271. 270 UInt64 0 + 272. 271 UInt64 1 + 273. 272 UInt64 0 + 274. 273 UInt64 1 + 275. 274 UInt64 0 + 276. 275 UInt64 1 + 277. 276 UInt64 0 + 278. 277 UInt64 1 + 279. 278 UInt64 0 + 280. 279 UInt64 1 + 281. 280 UInt64 0 + 282. 281 UInt64 1 + 283. 282 UInt64 0 + 284. 283 UInt64 1 + 285. 284 UInt64 0 + 286. 285 UInt64 1 + 287. 286 UInt64 0 + 288. 287 UInt64 1 + 289. 288 UInt64 0 + 290. 289 UInt64 1 + 291. 290 UInt64 0 + 292. 291 UInt64 1 + 293. 292 UInt64 0 + 294. 293 UInt64 1 + 295. 294 UInt64 0 + 296. 295 UInt64 1 + 297. 296 UInt64 0 + 298. 297 UInt64 1 + 299. 298 UInt64 0 + 300. 299 UInt64 1 + 301. 300 UInt64 0 + 302. 301 UInt64 1 + 303. 302 UInt64 0 + 304. 303 UInt64 1 + 305. 304 UInt64 0 + 306. 305 UInt64 1 + 307. 306 UInt64 0 + 308. 307 UInt64 1 + 309. 308 UInt64 0 + 310. 309 UInt64 1 + 311. 310 UInt64 0 + 312. 311 UInt64 1 + 313. 312 UInt64 0 + 314. 313 UInt64 1 + 315. 314 UInt64 0 + 316. 315 UInt64 1 + 317. 316 UInt64 0 + 318. 317 UInt64 1 + 319. 318 UInt64 0 + 320. 319 UInt64 1 + 321. 320 UInt64 0 + 322. 321 UInt64 1 + 323. 322 UInt64 0 + 324. 323 UInt64 1 + 325. 324 UInt64 0 + 326. 325 UInt64 1 + 327. 326 UInt64 0 + 328. 327 UInt64 1 + 329. 328 UInt64 0 + 330. 329 UInt64 1 + 331. 330 UInt64 0 + 332. 331 UInt64 1 + 333. 332 UInt64 0 + 334. 333 UInt64 1 + 335. 334 UInt64 0 + 336. 335 UInt64 1 + 337. 336 UInt64 0 + 338. 337 UInt64 1 + 339. 338 UInt64 0 + 340. 339 UInt64 1 + 341. 340 UInt64 0 + 342. 341 UInt64 1 + 343. 342 UInt64 0 + 344. 343 UInt64 1 + 345. 344 UInt64 0 + 346. 345 UInt64 1 + 347. 346 UInt64 0 + 348. 347 UInt64 1 + 349. 348 UInt64 0 + 350. 349 UInt64 1 + 351. 350 UInt64 0 + 352. 351 UInt64 1 + 353. 352 UInt64 0 + 354. 353 UInt64 1 + 355. 354 UInt64 0 + 356. 355 UInt64 1 + 357. 356 UInt64 0 + 358. 357 UInt64 1 + 359. 358 UInt64 0 + 360. 359 UInt64 1 + 361. 360 UInt64 0 + 362. 361 UInt64 1 + 363. 362 UInt64 0 + 364. 363 UInt64 1 + 365. 364 UInt64 0 + 366. 365 UInt64 1 + 367. 366 UInt64 0 + 368. 367 UInt64 1 + 369. 368 UInt64 0 + 370. 369 UInt64 1 + 371. 370 UInt64 0 + 372. 371 UInt64 1 + 373. 372 UInt64 0 + 374. 373 UInt64 1 + 375. 374 UInt64 0 + 376. 375 UInt64 1 + 377. 376 UInt64 0 + 378. 377 UInt64 1 + 379. 378 UInt64 0 + 380. 379 UInt64 1 + 381. 380 UInt64 0 + 382. 381 UInt64 1 + 383. 382 UInt64 0 + 384. 383 UInt64 1 + 385. 384 UInt64 0 + 386. 385 UInt64 1 + 387. 386 UInt64 0 + 388. 387 UInt64 1 + 389. 388 UInt64 0 + 390. 389 UInt64 1 + 391. 390 UInt64 0 + 392. 391 UInt64 1 + 393. 392 UInt64 0 + 394. 393 UInt64 1 + 395. 394 UInt64 0 + 396. 395 UInt64 1 + 397. 396 UInt64 0 + 398. 397 UInt64 1 + 399. 398 UInt64 0 + 400. 399 UInt64 1 + 401. 400 UInt64 0 + 402. 401 UInt64 1 + 403. 402 UInt64 0 + 404. 403 UInt64 1 + 405. 404 UInt64 0 + 406. 405 UInt64 1 + 407. 406 UInt64 0 + 408. 407 UInt64 1 + 409. 408 UInt64 0 + 410. 409 UInt64 1 + 411. 410 UInt64 0 + 412. 411 UInt64 1 + 413. 412 UInt64 0 + 414. 413 UInt64 1 + 415. 414 UInt64 0 + 416. 415 UInt64 1 + 417. 416 UInt64 0 + 418. 417 UInt64 1 + 419. 418 UInt64 0 + 420. 419 UInt64 1 + 421. 420 UInt64 0 + 422. 421 UInt64 1 + 423. 422 UInt64 0 + 424. 423 UInt64 1 + 425. 424 UInt64 0 + 426. 425 UInt64 1 + 427. 426 UInt64 0 + 428. 427 UInt64 1 + 429. 428 UInt64 0 + 430. 429 UInt64 1 + 431. 430 UInt64 0 + 432. 431 UInt64 1 + 433. 432 UInt64 0 + 434. 433 UInt64 1 + 435. 434 UInt64 0 + 436. 435 UInt64 1 + 437. 436 UInt64 0 + 438. 437 UInt64 1 + 439. 438 UInt64 0 + 440. 439 UInt64 1 + 441. 440 UInt64 0 + 442. 441 UInt64 1 + 443. 442 UInt64 0 + 444. 443 UInt64 1 + 445. 444 UInt64 0 + 446. 445 UInt64 1 + 447. 446 UInt64 0 + 448. 447 UInt64 1 + 449. 448 UInt64 0 + 450. 449 UInt64 1 + 451. 450 UInt64 0 + 452. 451 UInt64 1 + 453. 452 UInt64 0 + 454. 453 UInt64 1 + 455. 454 UInt64 0 + 456. 455 UInt64 1 + 457. 456 UInt64 0 + 458. 457 UInt64 1 + 459. 458 UInt64 0 + 460. 459 UInt64 1 + 461. 460 UInt64 0 + 462. 461 UInt64 1 + 463. 462 UInt64 0 + 464. 463 UInt64 1 + 465. 464 UInt64 0 + 466. 465 UInt64 1 + 467. 466 UInt64 0 + 468. 467 UInt64 1 + 469. 468 UInt64 0 + 470. 469 UInt64 1 + 471. 470 UInt64 0 + 472. 471 UInt64 1 + 473. 472 UInt64 0 + 474. 473 UInt64 1 + 475. 474 UInt64 0 + 476. 475 UInt64 1 + 477. 476 UInt64 0 + 478. 477 UInt64 1 + 479. 478 UInt64 0 + 480. 479 UInt64 1 + 481. 480 UInt64 0 + 482. 481 UInt64 1 + 483. 482 UInt64 0 + 484. 483 UInt64 1 + 485. 484 UInt64 0 + 486. 485 UInt64 1 + 487. 486 UInt64 0 + 488. 487 UInt64 1 + 489. 488 UInt64 0 + 490. 489 UInt64 1 + 491. 490 UInt64 0 + 492. 491 UInt64 1 + 493. 492 UInt64 0 + 494. 493 UInt64 1 + 495. 494 UInt64 0 + 496. 495 UInt64 1 + 497. 496 UInt64 0 + 498. 497 UInt64 1 + 499. 498 UInt64 0 + 500. 499 UInt64 1 + 501. 500 UInt64 0 + 502. 501 UInt64 1 + 503. 502 UInt64 0 + 504. 503 UInt64 1 + 505. 504 UInt64 0 + 506. 505 UInt64 1 + 507. 506 UInt64 0 + 508. 507 UInt64 1 + 509. 508 UInt64 0 + 510. 509 UInt64 1 + 511. 510 UInt64 0 + 512. 511 UInt64 1 + 513. 512 UInt64 0 + 514. 513 UInt64 1 + 515. 514 UInt64 0 + 516. 515 UInt64 1 + 517. 516 UInt64 0 + 518. 517 UInt64 1 + 519. 518 UInt64 0 + 520. 519 UInt64 1 + 521. 520 UInt64 0 + 522. 521 UInt64 1 + 523. 522 UInt64 0 + 524. 523 UInt64 1 + 525. 524 UInt64 0 + 526. 525 UInt64 1 + 527. 526 UInt64 0 + 528. 527 UInt64 1 + 529. 528 UInt64 0 + 530. 529 UInt64 1 + 531. 530 UInt64 0 + 532. 531 UInt64 1 + 533. 532 UInt64 0 + 534. 533 UInt64 1 + 535. 534 UInt64 0 + 536. 535 UInt64 1 + 537. 536 UInt64 0 + 538. 537 UInt64 1 + 539. 538 UInt64 0 + 540. 539 UInt64 1 + 541. 540 UInt64 0 + 542. 541 UInt64 1 + 543. 542 UInt64 0 + 544. 543 UInt64 1 + 545. 544 UInt64 0 + 546. 545 UInt64 1 + 547. 546 UInt64 0 + 548. 547 UInt64 1 + 549. 548 UInt64 0 + 550. 549 UInt64 1 + 551. 550 UInt64 0 + 552. 551 UInt64 1 + 553. 552 UInt64 0 + 554. 553 UInt64 1 + 555. 554 UInt64 0 + 556. 555 UInt64 1 + 557. 556 UInt64 0 + 558. 557 UInt64 1 + 559. 558 UInt64 0 + 560. 559 UInt64 1 + 561. 560 UInt64 0 + 562. 561 UInt64 1 + 563. 562 UInt64 0 + 564. 563 UInt64 1 + 565. 564 UInt64 0 + 566. 565 UInt64 1 + 567. 566 UInt64 0 + 568. 567 UInt64 1 + 569. 568 UInt64 0 + 570. 569 UInt64 1 + 571. 570 UInt64 0 + 572. 571 UInt64 1 + 573. 572 UInt64 0 + 574. 573 UInt64 1 + 575. 574 UInt64 0 + 576. 575 UInt64 1 + 577. 576 UInt64 0 + 578. 577 UInt64 1 + 579. 578 UInt64 0 + 580. 579 UInt64 1 + 581. 580 UInt64 0 + 582. 581 UInt64 1 + 583. 582 UInt64 0 + 584. 583 UInt64 1 + 585. 584 UInt64 0 + 586. 585 UInt64 1 + 587. 586 UInt64 0 + 588. 587 UInt64 1 + 589. 588 UInt64 0 + 590. 589 UInt64 1 + 591. 590 UInt64 0 + 592. 591 UInt64 1 + 593. 592 UInt64 0 + 594. 593 UInt64 1 + 595. 594 UInt64 0 + 596. 595 UInt64 1 + 597. 596 UInt64 0 + 598. 597 UInt64 1 + 599. 598 UInt64 0 + 600. 599 UInt64 1 + 601. 600 UInt64 0 + 602. 601 UInt64 1 + 603. 602 UInt64 0 + 604. 603 UInt64 1 + 605. 604 UInt64 0 + 606. 605 UInt64 1 + 607. 606 UInt64 0 + 608. 607 UInt64 1 + 609. 608 UInt64 0 + 610. 609 UInt64 1 + 611. 610 UInt64 0 + 612. 611 UInt64 1 + 613. 612 UInt64 0 + 614. 613 UInt64 1 + 615. 614 UInt64 0 + 616. 615 UInt64 1 + 617. 616 UInt64 0 + 618. 617 UInt64 1 + 619. 618 UInt64 0 + 620. 619 UInt64 1 + 621. 620 UInt64 0 + 622. 621 UInt64 1 + 623. 622 UInt64 0 + 624. 623 UInt64 1 + 625. 624 UInt64 0 + 626. 625 UInt64 1 + 627. 626 UInt64 0 + 628. 627 UInt64 1 + 629. 628 UInt64 0 + 630. 629 UInt64 1 + 631. 630 UInt64 0 + 632. 631 UInt64 1 + 633. 632 UInt64 0 + 634. 633 UInt64 1 + 635. 634 UInt64 0 + 636. 635 UInt64 1 + 637. 636 UInt64 0 + 638. 637 UInt64 1 + 639. 638 UInt64 0 + 640. 639 UInt64 1 + 641. 640 UInt64 0 + 642. 641 UInt64 1 + 643. 642 UInt64 0 + 644. 643 UInt64 1 + 645. 644 UInt64 0 + 646. 645 UInt64 1 + 647. 646 UInt64 0 + 648. 647 UInt64 1 + 649. 648 UInt64 0 + 650. 649 UInt64 1 + 651. 650 UInt64 0 + 652. 651 UInt64 1 + 653. 652 UInt64 0 + 654. 653 UInt64 1 + 655. 654 UInt64 0 + 656. 655 UInt64 1 + 657. 656 UInt64 0 + 658. 657 UInt64 1 + 659. 658 UInt64 0 + 660. 659 UInt64 1 + 661. 660 UInt64 0 + 662. 661 UInt64 1 + 663. 662 UInt64 0 + 664. 663 UInt64 1 + 665. 664 UInt64 0 + 666. 665 UInt64 1 + 667. 666 UInt64 0 + 668. 667 UInt64 1 + 669. 668 UInt64 0 + 670. 669 UInt64 1 + 671. 670 UInt64 0 + 672. 671 UInt64 1 + 673. 672 UInt64 0 + 674. 673 UInt64 1 + 675. 674 UInt64 0 + 676. 675 UInt64 1 + 677. 676 UInt64 0 + 678. 677 UInt64 1 + 679. 678 UInt64 0 + 680. 679 UInt64 1 + 681. 680 UInt64 0 + 682. 681 UInt64 1 + 683. 682 UInt64 0 + 684. 683 UInt64 1 + 685. 684 UInt64 0 + 686. 685 UInt64 1 + 687. 686 UInt64 0 + 688. 687 UInt64 1 + 689. 688 UInt64 0 + 690. 689 UInt64 1 + 691. 690 UInt64 0 + 692. 691 UInt64 1 + 693. 692 UInt64 0 + 694. 693 UInt64 1 + 695. 694 UInt64 0 + 696. 695 UInt64 1 + 697. 696 UInt64 0 + 698. 697 UInt64 1 + 699. 698 UInt64 0 + 700. 699 UInt64 1 + 701. 700 UInt64 0 + 702. 701 UInt64 1 + 703. 702 UInt64 0 + 704. 703 UInt64 1 + 705. 704 UInt64 0 + 706. 705 UInt64 1 + 707. 706 UInt64 0 + 708. 707 UInt64 1 + 709. 708 UInt64 0 + 710. 709 UInt64 1 + 711. 710 UInt64 0 + 712. 711 UInt64 1 + 713. 712 UInt64 0 + 714. 713 UInt64 1 + 715. 714 UInt64 0 + 716. 715 UInt64 1 + 717. 716 UInt64 0 + 718. 717 UInt64 1 + 719. 718 UInt64 0 + 720. 719 UInt64 1 + 721. 720 UInt64 0 + 722. 721 UInt64 1 + 723. 722 UInt64 0 + 724. 723 UInt64 1 + 725. 724 UInt64 0 + 726. 725 UInt64 1 + 727. 726 UInt64 0 + 728. 727 UInt64 1 + 729. 728 UInt64 0 + 730. 729 UInt64 1 + 731. 730 UInt64 0 + 732. 731 UInt64 1 + 733. 732 UInt64 0 + 734. 733 UInt64 1 + 735. 734 UInt64 0 + 736. 735 UInt64 1 + 737. 736 UInt64 0 + 738. 737 UInt64 1 + 739. 738 UInt64 0 + 740. 739 UInt64 1 + 741. 740 UInt64 0 + 742. 741 UInt64 1 + 743. 742 UInt64 0 + 744. 743 UInt64 1 + 745. 744 UInt64 0 + 746. 745 UInt64 1 + 747. 746 UInt64 0 + 748. 747 UInt64 1 + 749. 748 UInt64 0 + 750. 749 UInt64 1 + 751. 750 UInt64 0 + 752. 751 UInt64 1 + 753. 752 UInt64 0 + 754. 753 UInt64 1 + 755. 754 UInt64 0 + 756. 755 UInt64 1 + 757. 756 UInt64 0 + 758. 757 UInt64 1 + 759. 758 UInt64 0 + 760. 759 UInt64 1 + 761. 760 UInt64 0 + 762. 761 UInt64 1 + 763. 762 UInt64 0 + 764. 763 UInt64 1 + 765. 764 UInt64 0 + 766. 765 UInt64 1 + 767. 766 UInt64 0 + 768. 767 UInt64 1 + 769. 768 UInt64 0 + 770. 769 UInt64 1 + 771. 770 UInt64 0 + 772. 771 UInt64 1 + 773. 772 UInt64 0 + 774. 773 UInt64 1 + 775. 774 UInt64 0 + 776. 775 UInt64 1 + 777. 776 UInt64 0 + 778. 777 UInt64 1 + 779. 778 UInt64 0 + 780. 779 UInt64 1 + 781. 780 UInt64 0 + 782. 781 UInt64 1 + 783. 782 UInt64 0 + 784. 783 UInt64 1 + 785. 784 UInt64 0 + 786. 785 UInt64 1 + 787. 786 UInt64 0 + 788. 787 UInt64 1 + 789. 788 UInt64 0 + 790. 789 UInt64 1 + 791. 790 UInt64 0 + 792. 791 UInt64 1 + 793. 792 UInt64 0 + 794. 793 UInt64 1 + 795. 794 UInt64 0 + 796. 795 UInt64 1 + 797. 796 UInt64 0 + 798. 797 UInt64 1 + 799. 798 UInt64 0 + 800. 799 UInt64 1 + 801. 800 UInt64 0 + 802. 801 UInt64 1 + 803. 802 UInt64 0 + 804. 803 UInt64 1 + 805. 804 UInt64 0 + 806. 805 UInt64 1 + 807. 806 UInt64 0 + 808. 807 UInt64 1 + 809. 808 UInt64 0 + 810. 809 UInt64 1 + 811. 810 UInt64 0 + 812. 811 UInt64 1 + 813. 812 UInt64 0 + 814. 813 UInt64 1 + 815. 814 UInt64 0 + 816. 815 UInt64 1 + 817. 816 UInt64 0 + 818. 817 UInt64 1 + 819. 818 UInt64 0 + 820. 819 UInt64 1 + 821. 820 UInt64 0 + 822. 821 UInt64 1 + 823. 822 UInt64 0 + 824. 823 UInt64 1 + 825. 824 UInt64 0 + 826. 825 UInt64 1 + 827. 826 UInt64 0 + 828. 827 UInt64 1 + 829. 828 UInt64 0 + 830. 829 UInt64 1 + 831. 830 UInt64 0 + 832. 831 UInt64 1 + 833. 832 UInt64 0 + 834. 833 UInt64 1 + 835. 834 UInt64 0 + 836. 835 UInt64 1 + 837. 836 UInt64 0 + 838. 837 UInt64 1 + 839. 838 UInt64 0 + 840. 839 UInt64 1 + 841. 840 UInt64 0 + 842. 841 UInt64 1 + 843. 842 UInt64 0 + 844. 843 UInt64 1 + 845. 844 UInt64 0 + 846. 845 UInt64 1 + 847. 846 UInt64 0 + 848. 847 UInt64 1 + 849. 848 UInt64 0 + 850. 849 UInt64 1 + 851. 850 UInt64 0 + 852. 851 UInt64 1 + 853. 852 UInt64 0 + 854. 853 UInt64 1 + 855. 854 UInt64 0 + 856. 855 UInt64 1 + 857. 856 UInt64 0 + 858. 857 UInt64 1 + 859. 858 UInt64 0 + 860. 859 UInt64 1 + 861. 860 UInt64 0 + 862. 861 UInt64 1 + 863. 862 UInt64 0 + 864. 863 UInt64 1 + 865. 864 UInt64 0 + 866. 865 UInt64 1 + 867. 866 UInt64 0 + 868. 867 UInt64 1 + 869. 868 UInt64 0 + 870. 869 UInt64 1 + 871. 870 UInt64 0 + 872. 871 UInt64 1 + 873. 872 UInt64 0 + 874. 873 UInt64 1 + 875. 874 UInt64 0 + 876. 875 UInt64 1 + 877. 876 UInt64 0 + 878. 877 UInt64 1 + 879. 878 UInt64 0 + 880. 879 UInt64 1 + 881. 880 UInt64 0 + 882. 881 UInt64 1 + 883. 882 UInt64 0 + 884. 883 UInt64 1 + 885. 884 UInt64 0 + 886. 885 UInt64 1 + 887. 886 UInt64 0 + 888. 887 UInt64 1 + 889. 888 UInt64 0 + 890. 889 UInt64 1 + 891. 890 UInt64 0 + 892. 891 UInt64 1 + 893. 892 UInt64 0 + 894. 893 UInt64 1 + 895. 894 UInt64 0 + 896. 895 UInt64 1 + 897. 896 UInt64 0 + 898. 897 UInt64 1 + 899. 898 UInt64 0 + 900. 899 UInt64 1 + 901. 900 UInt64 0 + 902. 901 UInt64 1 + 903. 902 UInt64 0 + 904. 903 UInt64 1 + 905. 904 UInt64 0 + 906. 905 UInt64 1 + 907. 906 UInt64 0 + 908. 907 UInt64 1 + 909. 908 UInt64 0 + 910. 909 UInt64 1 + 911. 910 UInt64 0 + 912. 911 UInt64 1 + 913. 912 UInt64 0 + 914. 913 UInt64 1 + 915. 914 UInt64 0 + 916. 915 UInt64 1 + 917. 916 UInt64 0 + 918. 917 UInt64 1 + 919. 918 UInt64 0 + 920. 919 UInt64 1 + 921. 920 UInt64 0 + 922. 921 UInt64 1 + 923. 922 UInt64 0 + 924. 923 UInt64 1 + 925. 924 UInt64 0 + 926. 925 UInt64 1 + 927. 926 UInt64 0 + 928. 927 UInt64 1 + 929. 928 UInt64 0 + 930. 929 UInt64 1 + 931. 930 UInt64 0 + 932. 931 UInt64 1 + 933. 932 UInt64 0 + 934. 933 UInt64 1 + 935. 934 UInt64 0 + 936. 935 UInt64 1 + 937. 936 UInt64 0 + 938. 937 UInt64 1 + 939. 938 UInt64 0 + 940. 939 UInt64 1 + 941. 940 UInt64 0 + 942. 941 UInt64 1 + 943. 942 UInt64 0 + 944. 943 UInt64 1 + 945. 944 UInt64 0 + 946. 945 UInt64 1 + 947. 946 UInt64 0 + 948. 947 UInt64 1 + 949. 948 UInt64 0 + 950. 949 UInt64 1 + 951. 950 UInt64 0 + 952. 951 UInt64 1 + 953. 952 UInt64 0 + 954. 953 UInt64 1 + 955. 954 UInt64 0 + 956. 955 UInt64 1 + 957. 956 UInt64 0 + 958. 957 UInt64 1 + 959. 958 UInt64 0 + 960. 959 UInt64 1 + 961. 960 UInt64 0 + 962. 961 UInt64 1 + 963. 962 UInt64 0 + 964. 963 UInt64 1 + 965. 964 UInt64 0 + 966. 965 UInt64 1 + 967. 966 UInt64 0 + 968. 967 UInt64 1 + 969. 968 UInt64 0 + 970. 969 UInt64 1 + 971. 970 UInt64 0 + 972. 971 UInt64 1 + 973. 972 UInt64 0 + 974. 973 UInt64 1 + 975. 974 UInt64 0 + 976. 975 UInt64 1 + 977. 976 UInt64 0 + 978. 977 UInt64 1 + 979. 978 UInt64 0 + 980. 979 UInt64 1 + 981. 980 UInt64 0 + 982. 981 UInt64 1 + 983. 982 UInt64 0 + 984. 983 UInt64 1 + 985. 984 UInt64 0 + 986. 985 UInt64 1 + 987. 986 UInt64 0 + 988. 987 UInt64 1 + 989. 988 UInt64 0 + 990. 989 UInt64 1 + 991. 990 UInt64 0 + 992. 991 UInt64 1 + 993. 992 UInt64 0 + 994. 993 UInt64 1 + 995. 994 UInt64 0 + 996. 995 UInt64 1 + 997. 996 UInt64 0 + 998. 997 UInt64 1 + 999. 998 UInt64 0 +1000. 999 UInt64 1 + + number toTypeName(number) mod(number, 2) + + number toTypeName(number) mod(number, 2) + + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 + 100. 99 UInt64 1 + 101. 100 UInt64 0 + 102. 101 UInt64 1 + 103. 102 UInt64 0 + 104. 103 UInt64 1 + 105. 104 UInt64 0 + 106. 105 UInt64 1 + 107. 106 UInt64 0 + 108. 107 UInt64 1 + 109. 108 UInt64 0 + 110. 109 UInt64 1 + 111. 110 UInt64 0 + 112. 111 UInt64 1 + 113. 112 UInt64 0 + 114. 113 UInt64 1 + 115. 114 UInt64 0 + 116. 115 UInt64 1 + 117. 116 UInt64 0 + 118. 117 UInt64 1 + 119. 118 UInt64 0 + 120. 119 UInt64 1 + 121. 120 UInt64 0 + 122. 121 UInt64 1 + 123. 122 UInt64 0 + 124. 123 UInt64 1 + 125. 124 UInt64 0 + 126. 125 UInt64 1 + 127. 126 UInt64 0 + 128. 127 UInt64 1 + 129. 128 UInt64 0 + 130. 129 UInt64 1 + 131. 130 UInt64 0 + 132. 131 UInt64 1 + 133. 132 UInt64 0 + 134. 133 UInt64 1 + 135. 134 UInt64 0 + 136. 135 UInt64 1 + 137. 136 UInt64 0 + 138. 137 UInt64 1 + 139. 138 UInt64 0 + 140. 139 UInt64 1 + 141. 140 UInt64 0 + 142. 141 UInt64 1 + 143. 142 UInt64 0 + 144. 143 UInt64 1 + 145. 144 UInt64 0 + 146. 145 UInt64 1 + 147. 146 UInt64 0 + 148. 147 UInt64 1 + 149. 148 UInt64 0 + 150. 149 UInt64 1 + 151. 150 UInt64 0 + 152. 151 UInt64 1 + 153. 152 UInt64 0 + 154. 153 UInt64 1 + 155. 154 UInt64 0 + 156. 155 UInt64 1 + 157. 156 UInt64 0 + 158. 157 UInt64 1 + 159. 158 UInt64 0 + 160. 159 UInt64 1 + 161. 160 UInt64 0 + 162. 161 UInt64 1 + 163. 162 UInt64 0 + 164. 163 UInt64 1 + 165. 164 UInt64 0 + 166. 165 UInt64 1 + 167. 166 UInt64 0 + 168. 167 UInt64 1 + 169. 168 UInt64 0 + 170. 169 UInt64 1 + 171. 170 UInt64 0 + 172. 171 UInt64 1 + 173. 172 UInt64 0 + 174. 173 UInt64 1 + 175. 174 UInt64 0 + 176. 175 UInt64 1 + 177. 176 UInt64 0 + 178. 177 UInt64 1 + 179. 178 UInt64 0 + 180. 179 UInt64 1 + 181. 180 UInt64 0 + 182. 181 UInt64 1 + 183. 182 UInt64 0 + 184. 183 UInt64 1 + 185. 184 UInt64 0 + 186. 185 UInt64 1 + 187. 186 UInt64 0 + 188. 187 UInt64 1 + 189. 188 UInt64 0 + 190. 189 UInt64 1 + 191. 190 UInt64 0 + 192. 191 UInt64 1 + 193. 192 UInt64 0 + 194. 193 UInt64 1 + 195. 194 UInt64 0 + 196. 195 UInt64 1 + 197. 196 UInt64 0 + 198. 197 UInt64 1 + 199. 198 UInt64 0 + 200. 199 UInt64 1 + 201. 200 UInt64 0 + 202. 201 UInt64 1 + 203. 202 UInt64 0 + 204. 203 UInt64 1 + 205. 204 UInt64 0 + 206. 205 UInt64 1 + 207. 206 UInt64 0 + 208. 207 UInt64 1 + 209. 208 UInt64 0 + 210. 209 UInt64 1 + 211. 210 UInt64 0 + 212. 211 UInt64 1 + 213. 212 UInt64 0 + 214. 213 UInt64 1 + 215. 214 UInt64 0 + 216. 215 UInt64 1 + 217. 216 UInt64 0 + 218. 217 UInt64 1 + 219. 218 UInt64 0 + 220. 219 UInt64 1 + 221. 220 UInt64 0 + 222. 221 UInt64 1 + 223. 222 UInt64 0 + 224. 223 UInt64 1 + 225. 224 UInt64 0 + 226. 225 UInt64 1 + 227. 226 UInt64 0 + 228. 227 UInt64 1 + 229. 228 UInt64 0 + 230. 229 UInt64 1 + 231. 230 UInt64 0 + 232. 231 UInt64 1 + 233. 232 UInt64 0 + 234. 233 UInt64 1 + 235. 234 UInt64 0 + 236. 235 UInt64 1 + 237. 236 UInt64 0 + 238. 237 UInt64 1 + 239. 238 UInt64 0 + 240. 239 UInt64 1 + 241. 240 UInt64 0 + 242. 241 UInt64 1 + 243. 242 UInt64 0 + 244. 243 UInt64 1 + 245. 244 UInt64 0 + 246. 245 UInt64 1 + 247. 246 UInt64 0 + 248. 247 UInt64 1 + 249. 248 UInt64 0 + 250. 249 UInt64 1 + 251. 250 UInt64 0 + 252. 251 UInt64 1 + 253. 252 UInt64 0 + 254. 253 UInt64 1 + 255. 254 UInt64 0 + 256. 255 UInt64 1 + 257. 256 UInt64 0 + 258. 257 UInt64 1 + 259. 258 UInt64 0 + 260. 259 UInt64 1 + 261. 260 UInt64 0 + 262. 261 UInt64 1 + 263. 262 UInt64 0 + 264. 263 UInt64 1 + 265. 264 UInt64 0 + 266. 265 UInt64 1 + 267. 266 UInt64 0 + 268. 267 UInt64 1 + 269. 268 UInt64 0 + 270. 269 UInt64 1 + 271. 270 UInt64 0 + 272. 271 UInt64 1 + 273. 272 UInt64 0 + 274. 273 UInt64 1 + 275. 274 UInt64 0 + 276. 275 UInt64 1 + 277. 276 UInt64 0 + 278. 277 UInt64 1 + 279. 278 UInt64 0 + 280. 279 UInt64 1 + 281. 280 UInt64 0 + 282. 281 UInt64 1 + 283. 282 UInt64 0 + 284. 283 UInt64 1 + 285. 284 UInt64 0 + 286. 285 UInt64 1 + 287. 286 UInt64 0 + 288. 287 UInt64 1 + 289. 288 UInt64 0 + 290. 289 UInt64 1 + 291. 290 UInt64 0 + 292. 291 UInt64 1 + 293. 292 UInt64 0 + 294. 293 UInt64 1 + 295. 294 UInt64 0 + 296. 295 UInt64 1 + 297. 296 UInt64 0 + 298. 297 UInt64 1 + 299. 298 UInt64 0 + 300. 299 UInt64 1 + 301. 300 UInt64 0 + 302. 301 UInt64 1 + 303. 302 UInt64 0 + 304. 303 UInt64 1 + 305. 304 UInt64 0 + 306. 305 UInt64 1 + 307. 306 UInt64 0 + 308. 307 UInt64 1 + 309. 308 UInt64 0 + 310. 309 UInt64 1 + 311. 310 UInt64 0 + 312. 311 UInt64 1 + 313. 312 UInt64 0 + 314. 313 UInt64 1 + 315. 314 UInt64 0 + 316. 315 UInt64 1 + 317. 316 UInt64 0 + 318. 317 UInt64 1 + 319. 318 UInt64 0 + 320. 319 UInt64 1 + 321. 320 UInt64 0 + 322. 321 UInt64 1 + 323. 322 UInt64 0 + 324. 323 UInt64 1 + 325. 324 UInt64 0 + 326. 325 UInt64 1 + 327. 326 UInt64 0 + 328. 327 UInt64 1 + 329. 328 UInt64 0 + 330. 329 UInt64 1 + 331. 330 UInt64 0 + 332. 331 UInt64 1 + 333. 332 UInt64 0 + 334. 333 UInt64 1 + 335. 334 UInt64 0 + 336. 335 UInt64 1 + 337. 336 UInt64 0 + 338. 337 UInt64 1 + 339. 338 UInt64 0 + 340. 339 UInt64 1 + 341. 340 UInt64 0 + 342. 341 UInt64 1 + 343. 342 UInt64 0 + 344. 343 UInt64 1 + 345. 344 UInt64 0 + 346. 345 UInt64 1 + 347. 346 UInt64 0 + 348. 347 UInt64 1 + 349. 348 UInt64 0 + 350. 349 UInt64 1 + 351. 350 UInt64 0 + 352. 351 UInt64 1 + 353. 352 UInt64 0 + 354. 353 UInt64 1 + 355. 354 UInt64 0 + 356. 355 UInt64 1 + 357. 356 UInt64 0 + 358. 357 UInt64 1 + 359. 358 UInt64 0 + 360. 359 UInt64 1 + 361. 360 UInt64 0 + 362. 361 UInt64 1 + 363. 362 UInt64 0 + 364. 363 UInt64 1 + 365. 364 UInt64 0 + 366. 365 UInt64 1 + 367. 366 UInt64 0 + 368. 367 UInt64 1 + 369. 368 UInt64 0 + 370. 369 UInt64 1 + 371. 370 UInt64 0 + 372. 371 UInt64 1 + 373. 372 UInt64 0 + 374. 373 UInt64 1 + 375. 374 UInt64 0 + 376. 375 UInt64 1 + 377. 376 UInt64 0 + 378. 377 UInt64 1 + 379. 378 UInt64 0 + 380. 379 UInt64 1 + 381. 380 UInt64 0 + 382. 381 UInt64 1 + 383. 382 UInt64 0 + 384. 383 UInt64 1 + 385. 384 UInt64 0 + 386. 385 UInt64 1 + 387. 386 UInt64 0 + 388. 387 UInt64 1 + 389. 388 UInt64 0 + 390. 389 UInt64 1 + 391. 390 UInt64 0 + 392. 391 UInt64 1 + 393. 392 UInt64 0 + 394. 393 UInt64 1 + 395. 394 UInt64 0 + 396. 395 UInt64 1 + 397. 396 UInt64 0 + 398. 397 UInt64 1 + 399. 398 UInt64 0 + 400. 399 UInt64 1 + 401. 400 UInt64 0 + 402. 401 UInt64 1 + 403. 402 UInt64 0 + 404. 403 UInt64 1 + 405. 404 UInt64 0 + 406. 405 UInt64 1 + 407. 406 UInt64 0 + 408. 407 UInt64 1 + 409. 408 UInt64 0 + 410. 409 UInt64 1 + 411. 410 UInt64 0 + 412. 411 UInt64 1 + 413. 412 UInt64 0 + 414. 413 UInt64 1 + 415. 414 UInt64 0 + 416. 415 UInt64 1 + 417. 416 UInt64 0 + 418. 417 UInt64 1 + 419. 418 UInt64 0 + 420. 419 UInt64 1 + 421. 420 UInt64 0 + 422. 421 UInt64 1 + 423. 422 UInt64 0 + 424. 423 UInt64 1 + 425. 424 UInt64 0 + 426. 425 UInt64 1 + 427. 426 UInt64 0 + 428. 427 UInt64 1 + 429. 428 UInt64 0 + 430. 429 UInt64 1 + 431. 430 UInt64 0 + 432. 431 UInt64 1 + 433. 432 UInt64 0 + 434. 433 UInt64 1 + 435. 434 UInt64 0 + 436. 435 UInt64 1 + 437. 436 UInt64 0 + 438. 437 UInt64 1 + 439. 438 UInt64 0 + 440. 439 UInt64 1 + 441. 440 UInt64 0 + 442. 441 UInt64 1 + 443. 442 UInt64 0 + 444. 443 UInt64 1 + 445. 444 UInt64 0 + 446. 445 UInt64 1 + 447. 446 UInt64 0 + 448. 447 UInt64 1 + 449. 448 UInt64 0 + 450. 449 UInt64 1 + 451. 450 UInt64 0 + 452. 451 UInt64 1 + 453. 452 UInt64 0 + 454. 453 UInt64 1 + 455. 454 UInt64 0 + 456. 455 UInt64 1 + 457. 456 UInt64 0 + 458. 457 UInt64 1 + 459. 458 UInt64 0 + 460. 459 UInt64 1 + 461. 460 UInt64 0 + 462. 461 UInt64 1 + 463. 462 UInt64 0 + 464. 463 UInt64 1 + 465. 464 UInt64 0 + 466. 465 UInt64 1 + 467. 466 UInt64 0 + 468. 467 UInt64 1 + 469. 468 UInt64 0 + 470. 469 UInt64 1 + 471. 470 UInt64 0 + 472. 471 UInt64 1 + 473. 472 UInt64 0 + 474. 473 UInt64 1 + 475. 474 UInt64 0 + 476. 475 UInt64 1 + 477. 476 UInt64 0 + 478. 477 UInt64 1 + 479. 478 UInt64 0 + 480. 479 UInt64 1 + 481. 480 UInt64 0 + 482. 481 UInt64 1 + 483. 482 UInt64 0 + 484. 483 UInt64 1 + 485. 484 UInt64 0 + 486. 485 UInt64 1 + 487. 486 UInt64 0 + 488. 487 UInt64 1 + 489. 488 UInt64 0 + 490. 489 UInt64 1 + 491. 490 UInt64 0 + 492. 491 UInt64 1 + 493. 492 UInt64 0 + 494. 493 UInt64 1 + 495. 494 UInt64 0 + 496. 495 UInt64 1 + 497. 496 UInt64 0 + 498. 497 UInt64 1 + 499. 498 UInt64 0 + 500. 499 UInt64 1 + 501. 500 UInt64 0 + 502. 501 UInt64 1 + 503. 502 UInt64 0 + 504. 503 UInt64 1 + 505. 504 UInt64 0 + 506. 505 UInt64 1 + 507. 506 UInt64 0 + 508. 507 UInt64 1 + 509. 508 UInt64 0 + 510. 509 UInt64 1 + 511. 510 UInt64 0 + 512. 511 UInt64 1 + 513. 512 UInt64 0 + 514. 513 UInt64 1 + 515. 514 UInt64 0 + 516. 515 UInt64 1 + 517. 516 UInt64 0 + 518. 517 UInt64 1 + 519. 518 UInt64 0 + 520. 519 UInt64 1 + 521. 520 UInt64 0 + 522. 521 UInt64 1 + 523. 522 UInt64 0 + 524. 523 UInt64 1 + 525. 524 UInt64 0 + 526. 525 UInt64 1 + 527. 526 UInt64 0 + 528. 527 UInt64 1 + 529. 528 UInt64 0 + 530. 529 UInt64 1 + 531. 530 UInt64 0 + 532. 531 UInt64 1 + 533. 532 UInt64 0 + 534. 533 UInt64 1 + 535. 534 UInt64 0 + 536. 535 UInt64 1 + 537. 536 UInt64 0 + 538. 537 UInt64 1 + 539. 538 UInt64 0 + 540. 539 UInt64 1 + 541. 540 UInt64 0 + 542. 541 UInt64 1 + 543. 542 UInt64 0 + 544. 543 UInt64 1 + 545. 544 UInt64 0 + 546. 545 UInt64 1 + 547. 546 UInt64 0 + 548. 547 UInt64 1 + 549. 548 UInt64 0 + 550. 549 UInt64 1 + 551. 550 UInt64 0 + 552. 551 UInt64 1 + 553. 552 UInt64 0 + 554. 553 UInt64 1 + 555. 554 UInt64 0 + 556. 555 UInt64 1 + 557. 556 UInt64 0 + 558. 557 UInt64 1 + 559. 558 UInt64 0 + 560. 559 UInt64 1 + 561. 560 UInt64 0 + 562. 561 UInt64 1 + 563. 562 UInt64 0 + 564. 563 UInt64 1 + 565. 564 UInt64 0 + 566. 565 UInt64 1 + 567. 566 UInt64 0 + 568. 567 UInt64 1 + 569. 568 UInt64 0 + 570. 569 UInt64 1 + 571. 570 UInt64 0 + 572. 571 UInt64 1 + 573. 572 UInt64 0 + 574. 573 UInt64 1 + 575. 574 UInt64 0 + 576. 575 UInt64 1 + 577. 576 UInt64 0 + 578. 577 UInt64 1 + 579. 578 UInt64 0 + 580. 579 UInt64 1 + 581. 580 UInt64 0 + 582. 581 UInt64 1 + 583. 582 UInt64 0 + 584. 583 UInt64 1 + 585. 584 UInt64 0 + 586. 585 UInt64 1 + 587. 586 UInt64 0 + 588. 587 UInt64 1 + 589. 588 UInt64 0 + 590. 589 UInt64 1 + 591. 590 UInt64 0 + 592. 591 UInt64 1 + 593. 592 UInt64 0 + 594. 593 UInt64 1 + 595. 594 UInt64 0 + 596. 595 UInt64 1 + 597. 596 UInt64 0 + 598. 597 UInt64 1 + 599. 598 UInt64 0 + 600. 599 UInt64 1 + 601. 600 UInt64 0 + 602. 601 UInt64 1 + 603. 602 UInt64 0 + 604. 603 UInt64 1 + 605. 604 UInt64 0 + 606. 605 UInt64 1 + 607. 606 UInt64 0 + 608. 607 UInt64 1 + 609. 608 UInt64 0 + 610. 609 UInt64 1 + 611. 610 UInt64 0 + 612. 611 UInt64 1 + 613. 612 UInt64 0 + 614. 613 UInt64 1 + 615. 614 UInt64 0 + 616. 615 UInt64 1 + 617. 616 UInt64 0 + 618. 617 UInt64 1 + 619. 618 UInt64 0 + 620. 619 UInt64 1 + 621. 620 UInt64 0 + 622. 621 UInt64 1 + 623. 622 UInt64 0 + 624. 623 UInt64 1 + 625. 624 UInt64 0 + 626. 625 UInt64 1 + 627. 626 UInt64 0 + 628. 627 UInt64 1 + 629. 628 UInt64 0 + 630. 629 UInt64 1 + 631. 630 UInt64 0 + 632. 631 UInt64 1 + 633. 632 UInt64 0 + 634. 633 UInt64 1 + 635. 634 UInt64 0 + 636. 635 UInt64 1 + 637. 636 UInt64 0 + 638. 637 UInt64 1 + 639. 638 UInt64 0 + 640. 639 UInt64 1 + 641. 640 UInt64 0 + 642. 641 UInt64 1 + 643. 642 UInt64 0 + 644. 643 UInt64 1 + 645. 644 UInt64 0 + 646. 645 UInt64 1 + 647. 646 UInt64 0 + 648. 647 UInt64 1 + 649. 648 UInt64 0 + 650. 649 UInt64 1 + 651. 650 UInt64 0 + 652. 651 UInt64 1 + 653. 652 UInt64 0 + 654. 653 UInt64 1 + 655. 654 UInt64 0 + 656. 655 UInt64 1 + 657. 656 UInt64 0 + 658. 657 UInt64 1 + 659. 658 UInt64 0 + 660. 659 UInt64 1 + 661. 660 UInt64 0 + 662. 661 UInt64 1 + 663. 662 UInt64 0 + 664. 663 UInt64 1 + 665. 664 UInt64 0 + 666. 665 UInt64 1 + 667. 666 UInt64 0 + 668. 667 UInt64 1 + 669. 668 UInt64 0 + 670. 669 UInt64 1 + 671. 670 UInt64 0 + 672. 671 UInt64 1 + 673. 672 UInt64 0 + 674. 673 UInt64 1 + 675. 674 UInt64 0 + 676. 675 UInt64 1 + 677. 676 UInt64 0 + 678. 677 UInt64 1 + 679. 678 UInt64 0 + 680. 679 UInt64 1 + 681. 680 UInt64 0 + 682. 681 UInt64 1 + 683. 682 UInt64 0 + 684. 683 UInt64 1 + 685. 684 UInt64 0 + 686. 685 UInt64 1 + 687. 686 UInt64 0 + 688. 687 UInt64 1 + 689. 688 UInt64 0 + 690. 689 UInt64 1 + 691. 690 UInt64 0 + 692. 691 UInt64 1 + 693. 692 UInt64 0 + 694. 693 UInt64 1 + 695. 694 UInt64 0 + 696. 695 UInt64 1 + 697. 696 UInt64 0 + 698. 697 UInt64 1 + 699. 698 UInt64 0 + 700. 699 UInt64 1 + 701. 700 UInt64 0 + 702. 701 UInt64 1 + 703. 702 UInt64 0 + 704. 703 UInt64 1 + 705. 704 UInt64 0 + 706. 705 UInt64 1 + 707. 706 UInt64 0 + 708. 707 UInt64 1 + 709. 708 UInt64 0 + 710. 709 UInt64 1 + 711. 710 UInt64 0 + 712. 711 UInt64 1 + 713. 712 UInt64 0 + 714. 713 UInt64 1 + 715. 714 UInt64 0 + 716. 715 UInt64 1 + 717. 716 UInt64 0 + 718. 717 UInt64 1 + 719. 718 UInt64 0 + 720. 719 UInt64 1 + 721. 720 UInt64 0 + 722. 721 UInt64 1 + 723. 722 UInt64 0 + 724. 723 UInt64 1 + 725. 724 UInt64 0 + 726. 725 UInt64 1 + 727. 726 UInt64 0 + 728. 727 UInt64 1 + 729. 728 UInt64 0 + 730. 729 UInt64 1 + 731. 730 UInt64 0 + 732. 731 UInt64 1 + 733. 732 UInt64 0 + 734. 733 UInt64 1 + 735. 734 UInt64 0 + 736. 735 UInt64 1 + 737. 736 UInt64 0 + 738. 737 UInt64 1 + 739. 738 UInt64 0 + 740. 739 UInt64 1 + 741. 740 UInt64 0 + 742. 741 UInt64 1 + 743. 742 UInt64 0 + 744. 743 UInt64 1 + 745. 744 UInt64 0 + 746. 745 UInt64 1 + 747. 746 UInt64 0 + 748. 747 UInt64 1 + 749. 748 UInt64 0 + 750. 749 UInt64 1 + 751. 750 UInt64 0 + 752. 751 UInt64 1 + 753. 752 UInt64 0 + 754. 753 UInt64 1 + 755. 754 UInt64 0 + 756. 755 UInt64 1 + 757. 756 UInt64 0 + 758. 757 UInt64 1 + 759. 758 UInt64 0 + 760. 759 UInt64 1 + 761. 760 UInt64 0 + 762. 761 UInt64 1 + 763. 762 UInt64 0 + 764. 763 UInt64 1 + 765. 764 UInt64 0 + 766. 765 UInt64 1 + 767. 766 UInt64 0 + 768. 767 UInt64 1 + 769. 768 UInt64 0 + 770. 769 UInt64 1 + 771. 770 UInt64 0 + 772. 771 UInt64 1 + 773. 772 UInt64 0 + 774. 773 UInt64 1 + 775. 774 UInt64 0 + 776. 775 UInt64 1 + 777. 776 UInt64 0 + 778. 777 UInt64 1 + 779. 778 UInt64 0 + 780. 779 UInt64 1 + 781. 780 UInt64 0 + 782. 781 UInt64 1 + 783. 782 UInt64 0 + 784. 783 UInt64 1 + 785. 784 UInt64 0 + 786. 785 UInt64 1 + 787. 786 UInt64 0 + 788. 787 UInt64 1 + 789. 788 UInt64 0 + 790. 789 UInt64 1 + 791. 790 UInt64 0 + 792. 791 UInt64 1 + 793. 792 UInt64 0 + 794. 793 UInt64 1 + 795. 794 UInt64 0 + 796. 795 UInt64 1 + 797. 796 UInt64 0 + 798. 797 UInt64 1 + 799. 798 UInt64 0 + 800. 799 UInt64 1 + 801. 800 UInt64 0 + 802. 801 UInt64 1 + 803. 802 UInt64 0 + 804. 803 UInt64 1 + 805. 804 UInt64 0 + 806. 805 UInt64 1 + 807. 806 UInt64 0 + 808. 807 UInt64 1 + 809. 808 UInt64 0 + 810. 809 UInt64 1 + 811. 810 UInt64 0 + 812. 811 UInt64 1 + 813. 812 UInt64 0 + 814. 813 UInt64 1 + 815. 814 UInt64 0 + 816. 815 UInt64 1 + 817. 816 UInt64 0 + 818. 817 UInt64 1 + 819. 818 UInt64 0 + 820. 819 UInt64 1 + 821. 820 UInt64 0 + 822. 821 UInt64 1 + 823. 822 UInt64 0 + 824. 823 UInt64 1 + 825. 824 UInt64 0 + 826. 825 UInt64 1 + 827. 826 UInt64 0 + 828. 827 UInt64 1 + 829. 828 UInt64 0 + 830. 829 UInt64 1 + 831. 830 UInt64 0 + 832. 831 UInt64 1 + 833. 832 UInt64 0 + 834. 833 UInt64 1 + 835. 834 UInt64 0 + 836. 835 UInt64 1 + 837. 836 UInt64 0 + 838. 837 UInt64 1 + 839. 838 UInt64 0 + 840. 839 UInt64 1 + 841. 840 UInt64 0 + 842. 841 UInt64 1 + 843. 842 UInt64 0 + 844. 843 UInt64 1 + 845. 844 UInt64 0 + 846. 845 UInt64 1 + 847. 846 UInt64 0 + 848. 847 UInt64 1 + 849. 848 UInt64 0 + 850. 849 UInt64 1 + 851. 850 UInt64 0 + 852. 851 UInt64 1 + 853. 852 UInt64 0 + 854. 853 UInt64 1 + 855. 854 UInt64 0 + 856. 855 UInt64 1 + 857. 856 UInt64 0 + 858. 857 UInt64 1 + 859. 858 UInt64 0 + 860. 859 UInt64 1 + 861. 860 UInt64 0 + 862. 861 UInt64 1 + 863. 862 UInt64 0 + 864. 863 UInt64 1 + 865. 864 UInt64 0 + 866. 865 UInt64 1 + 867. 866 UInt64 0 + 868. 867 UInt64 1 + 869. 868 UInt64 0 + 870. 869 UInt64 1 + 871. 870 UInt64 0 + 872. 871 UInt64 1 + 873. 872 UInt64 0 + 874. 873 UInt64 1 + 875. 874 UInt64 0 + 876. 875 UInt64 1 + 877. 876 UInt64 0 + 878. 877 UInt64 1 + 879. 878 UInt64 0 + 880. 879 UInt64 1 + 881. 880 UInt64 0 + 882. 881 UInt64 1 + 883. 882 UInt64 0 + 884. 883 UInt64 1 + 885. 884 UInt64 0 + 886. 885 UInt64 1 + 887. 886 UInt64 0 + 888. 887 UInt64 1 + 889. 888 UInt64 0 + 890. 889 UInt64 1 + 891. 890 UInt64 0 + 892. 891 UInt64 1 + 893. 892 UInt64 0 + 894. 893 UInt64 1 + 895. 894 UInt64 0 + 896. 895 UInt64 1 + 897. 896 UInt64 0 + 898. 897 UInt64 1 + 899. 898 UInt64 0 + 900. 899 UInt64 1 + 901. 900 UInt64 0 + 902. 901 UInt64 1 + 903. 902 UInt64 0 + 904. 903 UInt64 1 + 905. 904 UInt64 0 + 906. 905 UInt64 1 + 907. 906 UInt64 0 + 908. 907 UInt64 1 + 909. 908 UInt64 0 + 910. 909 UInt64 1 + 911. 910 UInt64 0 + 912. 911 UInt64 1 + 913. 912 UInt64 0 + 914. 913 UInt64 1 + 915. 914 UInt64 0 + 916. 915 UInt64 1 + 917. 916 UInt64 0 + 918. 917 UInt64 1 + 919. 918 UInt64 0 + 920. 919 UInt64 1 + 921. 920 UInt64 0 + 922. 921 UInt64 1 + 923. 922 UInt64 0 + 924. 923 UInt64 1 + 925. 924 UInt64 0 + 926. 925 UInt64 1 + 927. 926 UInt64 0 + 928. 927 UInt64 1 + 929. 928 UInt64 0 + 930. 929 UInt64 1 + 931. 930 UInt64 0 + 932. 931 UInt64 1 + 933. 932 UInt64 0 + 934. 933 UInt64 1 + 935. 934 UInt64 0 + 936. 935 UInt64 1 + 937. 936 UInt64 0 + 938. 937 UInt64 1 + 939. 938 UInt64 0 + 940. 939 UInt64 1 + 941. 940 UInt64 0 + 942. 941 UInt64 1 + 943. 942 UInt64 0 + 944. 943 UInt64 1 + 945. 944 UInt64 0 + 946. 945 UInt64 1 + 947. 946 UInt64 0 + 948. 947 UInt64 1 + 949. 948 UInt64 0 + 950. 949 UInt64 1 + 951. 950 UInt64 0 + 952. 951 UInt64 1 + 953. 952 UInt64 0 + 954. 953 UInt64 1 + 955. 954 UInt64 0 + 956. 955 UInt64 1 + 957. 956 UInt64 0 + 958. 957 UInt64 1 + 959. 958 UInt64 0 + 960. 959 UInt64 1 + 961. 960 UInt64 0 + 962. 961 UInt64 1 + 963. 962 UInt64 0 + 964. 963 UInt64 1 + 965. 964 UInt64 0 + 966. 965 UInt64 1 + 967. 966 UInt64 0 + 968. 967 UInt64 1 + 969. 968 UInt64 0 + 970. 969 UInt64 1 + 971. 970 UInt64 0 + 972. 971 UInt64 1 + 973. 972 UInt64 0 + 974. 973 UInt64 1 + 975. 974 UInt64 0 + 976. 975 UInt64 1 + 977. 976 UInt64 0 + 978. 977 UInt64 1 + 979. 978 UInt64 0 + 980. 979 UInt64 1 + 981. 980 UInt64 0 + 982. 981 UInt64 1 + 983. 982 UInt64 0 + 984. 983 UInt64 1 + 985. 984 UInt64 0 + 986. 985 UInt64 1 + 987. 986 UInt64 0 + 988. 987 UInt64 1 + 989. 988 UInt64 0 + 990. 989 UInt64 1 + 991. 990 UInt64 0 + 992. 991 UInt64 1 + 993. 992 UInt64 0 + 994. 993 UInt64 1 + 995. 994 UInt64 0 + 996. 995 UInt64 1 + 997. 996 UInt64 0 + 998. 997 UInt64 1 + 999. 998 UInt64 0 +1000. 999 UInt64 1 + + number toTypeName(number) mod(number, 2) + + number toTypeName(number) mod(number, 2) + + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 + 100. 99 UInt64 1 + 101. 100 UInt64 0 + 102. 101 UInt64 1 + 103. 102 UInt64 0 + 104. 103 UInt64 1 + 105. 104 UInt64 0 + 106. 105 UInt64 1 + 107. 106 UInt64 0 + 108. 107 UInt64 1 + 109. 108 UInt64 0 + 110. 109 UInt64 1 + 111. 110 UInt64 0 + 112. 111 UInt64 1 + 113. 112 UInt64 0 + 114. 113 UInt64 1 + 115. 114 UInt64 0 + 116. 115 UInt64 1 + 117. 116 UInt64 0 + 118. 117 UInt64 1 + 119. 118 UInt64 0 + 120. 119 UInt64 1 + 121. 120 UInt64 0 + 122. 121 UInt64 1 + 123. 122 UInt64 0 + 124. 123 UInt64 1 + 125. 124 UInt64 0 + 126. 125 UInt64 1 + 127. 126 UInt64 0 + 128. 127 UInt64 1 + 129. 128 UInt64 0 + 130. 129 UInt64 1 + 131. 130 UInt64 0 + 132. 131 UInt64 1 + 133. 132 UInt64 0 + 134. 133 UInt64 1 + 135. 134 UInt64 0 + 136. 135 UInt64 1 + 137. 136 UInt64 0 + 138. 137 UInt64 1 + 139. 138 UInt64 0 + 140. 139 UInt64 1 + 141. 140 UInt64 0 + 142. 141 UInt64 1 + 143. 142 UInt64 0 + 144. 143 UInt64 1 + 145. 144 UInt64 0 + 146. 145 UInt64 1 + 147. 146 UInt64 0 + 148. 147 UInt64 1 + 149. 148 UInt64 0 + 150. 149 UInt64 1 + 151. 150 UInt64 0 + 152. 151 UInt64 1 + 153. 152 UInt64 0 + 154. 153 UInt64 1 + 155. 154 UInt64 0 + 156. 155 UInt64 1 + 157. 156 UInt64 0 + 158. 157 UInt64 1 + 159. 158 UInt64 0 + 160. 159 UInt64 1 + 161. 160 UInt64 0 + 162. 161 UInt64 1 + 163. 162 UInt64 0 + 164. 163 UInt64 1 + 165. 164 UInt64 0 + 166. 165 UInt64 1 + 167. 166 UInt64 0 + 168. 167 UInt64 1 + 169. 168 UInt64 0 + 170. 169 UInt64 1 + 171. 170 UInt64 0 + 172. 171 UInt64 1 + 173. 172 UInt64 0 + 174. 173 UInt64 1 + 175. 174 UInt64 0 + 176. 175 UInt64 1 + 177. 176 UInt64 0 + 178. 177 UInt64 1 + 179. 178 UInt64 0 + 180. 179 UInt64 1 + 181. 180 UInt64 0 + 182. 181 UInt64 1 + 183. 182 UInt64 0 + 184. 183 UInt64 1 + 185. 184 UInt64 0 + 186. 185 UInt64 1 + 187. 186 UInt64 0 + 188. 187 UInt64 1 + 189. 188 UInt64 0 + 190. 189 UInt64 1 + 191. 190 UInt64 0 + 192. 191 UInt64 1 + 193. 192 UInt64 0 + 194. 193 UInt64 1 + 195. 194 UInt64 0 + 196. 195 UInt64 1 + 197. 196 UInt64 0 + 198. 197 UInt64 1 + 199. 198 UInt64 0 + 200. 199 UInt64 1 + 201. 200 UInt64 0 + 202. 201 UInt64 1 + 203. 202 UInt64 0 + 204. 203 UInt64 1 + 205. 204 UInt64 0 + 206. 205 UInt64 1 + 207. 206 UInt64 0 + 208. 207 UInt64 1 + 209. 208 UInt64 0 + 210. 209 UInt64 1 + 211. 210 UInt64 0 + 212. 211 UInt64 1 + 213. 212 UInt64 0 + 214. 213 UInt64 1 + 215. 214 UInt64 0 + 216. 215 UInt64 1 + 217. 216 UInt64 0 + 218. 217 UInt64 1 + 219. 218 UInt64 0 + 220. 219 UInt64 1 + 221. 220 UInt64 0 + 222. 221 UInt64 1 + 223. 222 UInt64 0 + 224. 223 UInt64 1 + 225. 224 UInt64 0 + 226. 225 UInt64 1 + 227. 226 UInt64 0 + 228. 227 UInt64 1 + 229. 228 UInt64 0 + 230. 229 UInt64 1 + 231. 230 UInt64 0 + 232. 231 UInt64 1 + 233. 232 UInt64 0 + 234. 233 UInt64 1 + 235. 234 UInt64 0 + 236. 235 UInt64 1 + 237. 236 UInt64 0 + 238. 237 UInt64 1 + 239. 238 UInt64 0 + 240. 239 UInt64 1 + 241. 240 UInt64 0 + 242. 241 UInt64 1 + 243. 242 UInt64 0 + 244. 243 UInt64 1 + 245. 244 UInt64 0 + 246. 245 UInt64 1 + 247. 246 UInt64 0 + 248. 247 UInt64 1 + 249. 248 UInt64 0 + 250. 249 UInt64 1 + 251. 250 UInt64 0 + 252. 251 UInt64 1 + 253. 252 UInt64 0 + 254. 253 UInt64 1 + 255. 254 UInt64 0 + 256. 255 UInt64 1 + 257. 256 UInt64 0 + 258. 257 UInt64 1 + 259. 258 UInt64 0 + 260. 259 UInt64 1 + 261. 260 UInt64 0 + 262. 261 UInt64 1 + 263. 262 UInt64 0 + 264. 263 UInt64 1 + 265. 264 UInt64 0 + 266. 265 UInt64 1 + 267. 266 UInt64 0 + 268. 267 UInt64 1 + 269. 268 UInt64 0 + 270. 269 UInt64 1 + 271. 270 UInt64 0 + 272. 271 UInt64 1 + 273. 272 UInt64 0 + 274. 273 UInt64 1 + 275. 274 UInt64 0 + 276. 275 UInt64 1 + 277. 276 UInt64 0 + 278. 277 UInt64 1 + 279. 278 UInt64 0 + 280. 279 UInt64 1 + 281. 280 UInt64 0 + 282. 281 UInt64 1 + 283. 282 UInt64 0 + 284. 283 UInt64 1 + 285. 284 UInt64 0 + 286. 285 UInt64 1 + 287. 286 UInt64 0 + 288. 287 UInt64 1 + 289. 288 UInt64 0 + 290. 289 UInt64 1 + 291. 290 UInt64 0 + 292. 291 UInt64 1 + 293. 292 UInt64 0 + 294. 293 UInt64 1 + 295. 294 UInt64 0 + 296. 295 UInt64 1 + 297. 296 UInt64 0 + 298. 297 UInt64 1 + 299. 298 UInt64 0 + 300. 299 UInt64 1 + 301. 300 UInt64 0 + 302. 301 UInt64 1 + 303. 302 UInt64 0 + 304. 303 UInt64 1 + 305. 304 UInt64 0 + 306. 305 UInt64 1 + 307. 306 UInt64 0 + 308. 307 UInt64 1 + 309. 308 UInt64 0 + 310. 309 UInt64 1 + 311. 310 UInt64 0 + 312. 311 UInt64 1 + 313. 312 UInt64 0 + 314. 313 UInt64 1 + 315. 314 UInt64 0 + 316. 315 UInt64 1 + 317. 316 UInt64 0 + 318. 317 UInt64 1 + 319. 318 UInt64 0 + 320. 319 UInt64 1 + 321. 320 UInt64 0 + 322. 321 UInt64 1 + 323. 322 UInt64 0 + 324. 323 UInt64 1 + 325. 324 UInt64 0 + 326. 325 UInt64 1 + 327. 326 UInt64 0 + 328. 327 UInt64 1 + 329. 328 UInt64 0 + 330. 329 UInt64 1 + 331. 330 UInt64 0 + 332. 331 UInt64 1 + 333. 332 UInt64 0 + 334. 333 UInt64 1 + 335. 334 UInt64 0 + 336. 335 UInt64 1 + 337. 336 UInt64 0 + 338. 337 UInt64 1 + 339. 338 UInt64 0 + 340. 339 UInt64 1 + 341. 340 UInt64 0 + 342. 341 UInt64 1 + 343. 342 UInt64 0 + 344. 343 UInt64 1 + 345. 344 UInt64 0 + 346. 345 UInt64 1 + 347. 346 UInt64 0 + 348. 347 UInt64 1 + 349. 348 UInt64 0 + 350. 349 UInt64 1 + 351. 350 UInt64 0 + 352. 351 UInt64 1 + 353. 352 UInt64 0 + 354. 353 UInt64 1 + 355. 354 UInt64 0 + 356. 355 UInt64 1 + 357. 356 UInt64 0 + 358. 357 UInt64 1 + 359. 358 UInt64 0 + 360. 359 UInt64 1 + 361. 360 UInt64 0 + 362. 361 UInt64 1 + 363. 362 UInt64 0 + 364. 363 UInt64 1 + 365. 364 UInt64 0 + 366. 365 UInt64 1 + 367. 366 UInt64 0 + 368. 367 UInt64 1 + 369. 368 UInt64 0 + 370. 369 UInt64 1 + 371. 370 UInt64 0 + 372. 371 UInt64 1 + 373. 372 UInt64 0 + 374. 373 UInt64 1 + 375. 374 UInt64 0 + 376. 375 UInt64 1 + 377. 376 UInt64 0 + 378. 377 UInt64 1 + 379. 378 UInt64 0 + 380. 379 UInt64 1 + 381. 380 UInt64 0 + 382. 381 UInt64 1 + 383. 382 UInt64 0 + 384. 383 UInt64 1 + 385. 384 UInt64 0 + 386. 385 UInt64 1 + 387. 386 UInt64 0 + 388. 387 UInt64 1 + 389. 388 UInt64 0 + 390. 389 UInt64 1 + 391. 390 UInt64 0 + 392. 391 UInt64 1 + 393. 392 UInt64 0 + 394. 393 UInt64 1 + 395. 394 UInt64 0 + 396. 395 UInt64 1 + 397. 396 UInt64 0 + 398. 397 UInt64 1 + 399. 398 UInt64 0 + 400. 399 UInt64 1 + 401. 400 UInt64 0 + 402. 401 UInt64 1 + 403. 402 UInt64 0 + 404. 403 UInt64 1 + 405. 404 UInt64 0 + 406. 405 UInt64 1 + 407. 406 UInt64 0 + 408. 407 UInt64 1 + 409. 408 UInt64 0 + 410. 409 UInt64 1 + 411. 410 UInt64 0 + 412. 411 UInt64 1 + 413. 412 UInt64 0 + 414. 413 UInt64 1 + 415. 414 UInt64 0 + 416. 415 UInt64 1 + 417. 416 UInt64 0 + 418. 417 UInt64 1 + 419. 418 UInt64 0 + 420. 419 UInt64 1 + 421. 420 UInt64 0 + 422. 421 UInt64 1 + 423. 422 UInt64 0 + 424. 423 UInt64 1 + 425. 424 UInt64 0 + 426. 425 UInt64 1 + 427. 426 UInt64 0 + 428. 427 UInt64 1 + 429. 428 UInt64 0 + 430. 429 UInt64 1 + 431. 430 UInt64 0 + 432. 431 UInt64 1 + 433. 432 UInt64 0 + 434. 433 UInt64 1 + 435. 434 UInt64 0 + 436. 435 UInt64 1 + 437. 436 UInt64 0 + 438. 437 UInt64 1 + 439. 438 UInt64 0 + 440. 439 UInt64 1 + 441. 440 UInt64 0 + 442. 441 UInt64 1 + 443. 442 UInt64 0 + 444. 443 UInt64 1 + 445. 444 UInt64 0 + 446. 445 UInt64 1 + 447. 446 UInt64 0 + 448. 447 UInt64 1 + 449. 448 UInt64 0 + 450. 449 UInt64 1 + 451. 450 UInt64 0 + 452. 451 UInt64 1 + 453. 452 UInt64 0 + 454. 453 UInt64 1 + 455. 454 UInt64 0 + 456. 455 UInt64 1 + 457. 456 UInt64 0 + 458. 457 UInt64 1 + 459. 458 UInt64 0 + 460. 459 UInt64 1 + 461. 460 UInt64 0 + 462. 461 UInt64 1 + 463. 462 UInt64 0 + 464. 463 UInt64 1 + 465. 464 UInt64 0 + 466. 465 UInt64 1 + 467. 466 UInt64 0 + 468. 467 UInt64 1 + 469. 468 UInt64 0 + 470. 469 UInt64 1 + 471. 470 UInt64 0 + 472. 471 UInt64 1 + 473. 472 UInt64 0 + 474. 473 UInt64 1 + 475. 474 UInt64 0 + 476. 475 UInt64 1 + 477. 476 UInt64 0 + 478. 477 UInt64 1 + 479. 478 UInt64 0 + 480. 479 UInt64 1 + 481. 480 UInt64 0 + 482. 481 UInt64 1 + 483. 482 UInt64 0 + 484. 483 UInt64 1 + 485. 484 UInt64 0 + 486. 485 UInt64 1 + 487. 486 UInt64 0 + 488. 487 UInt64 1 + 489. 488 UInt64 0 + 490. 489 UInt64 1 + 491. 490 UInt64 0 + 492. 491 UInt64 1 + 493. 492 UInt64 0 + 494. 493 UInt64 1 + 495. 494 UInt64 0 + 496. 495 UInt64 1 + 497. 496 UInt64 0 + 498. 497 UInt64 1 + 499. 498 UInt64 0 + 500. 499 UInt64 1 + 501. 500 UInt64 0 + 502. 501 UInt64 1 + 503. 502 UInt64 0 + 504. 503 UInt64 1 + 505. 504 UInt64 0 + 506. 505 UInt64 1 + 507. 506 UInt64 0 + 508. 507 UInt64 1 + 509. 508 UInt64 0 + 510. 509 UInt64 1 + 511. 510 UInt64 0 + 512. 511 UInt64 1 + 513. 512 UInt64 0 + 514. 513 UInt64 1 + 515. 514 UInt64 0 + 516. 515 UInt64 1 + 517. 516 UInt64 0 + 518. 517 UInt64 1 + 519. 518 UInt64 0 + 520. 519 UInt64 1 + 521. 520 UInt64 0 + 522. 521 UInt64 1 + 523. 522 UInt64 0 + 524. 523 UInt64 1 + 525. 524 UInt64 0 + 526. 525 UInt64 1 + 527. 526 UInt64 0 + 528. 527 UInt64 1 + 529. 528 UInt64 0 + 530. 529 UInt64 1 + 531. 530 UInt64 0 + 532. 531 UInt64 1 + 533. 532 UInt64 0 + 534. 533 UInt64 1 + 535. 534 UInt64 0 + 536. 535 UInt64 1 + 537. 536 UInt64 0 + 538. 537 UInt64 1 + 539. 538 UInt64 0 + 540. 539 UInt64 1 + 541. 540 UInt64 0 + 542. 541 UInt64 1 + 543. 542 UInt64 0 + 544. 543 UInt64 1 + 545. 544 UInt64 0 + 546. 545 UInt64 1 + 547. 546 UInt64 0 + 548. 547 UInt64 1 + 549. 548 UInt64 0 + 550. 549 UInt64 1 + 551. 550 UInt64 0 + 552. 551 UInt64 1 + 553. 552 UInt64 0 + 554. 553 UInt64 1 + 555. 554 UInt64 0 + 556. 555 UInt64 1 + 557. 556 UInt64 0 + 558. 557 UInt64 1 + 559. 558 UInt64 0 + 560. 559 UInt64 1 + 561. 560 UInt64 0 + 562. 561 UInt64 1 + 563. 562 UInt64 0 + 564. 563 UInt64 1 + 565. 564 UInt64 0 + 566. 565 UInt64 1 + 567. 566 UInt64 0 + 568. 567 UInt64 1 + 569. 568 UInt64 0 + 570. 569 UInt64 1 + 571. 570 UInt64 0 + 572. 571 UInt64 1 + 573. 572 UInt64 0 + 574. 573 UInt64 1 + 575. 574 UInt64 0 + 576. 575 UInt64 1 + 577. 576 UInt64 0 + 578. 577 UInt64 1 + 579. 578 UInt64 0 + 580. 579 UInt64 1 + 581. 580 UInt64 0 + 582. 581 UInt64 1 + 583. 582 UInt64 0 + 584. 583 UInt64 1 + 585. 584 UInt64 0 + 586. 585 UInt64 1 + 587. 586 UInt64 0 + 588. 587 UInt64 1 + 589. 588 UInt64 0 + 590. 589 UInt64 1 + 591. 590 UInt64 0 + 592. 591 UInt64 1 + 593. 592 UInt64 0 + 594. 593 UInt64 1 + 595. 594 UInt64 0 + 596. 595 UInt64 1 + 597. 596 UInt64 0 + 598. 597 UInt64 1 + 599. 598 UInt64 0 + 600. 599 UInt64 1 + 601. 600 UInt64 0 + 602. 601 UInt64 1 + 603. 602 UInt64 0 + 604. 603 UInt64 1 + 605. 604 UInt64 0 + 606. 605 UInt64 1 + 607. 606 UInt64 0 + 608. 607 UInt64 1 + 609. 608 UInt64 0 + 610. 609 UInt64 1 + 611. 610 UInt64 0 + 612. 611 UInt64 1 + 613. 612 UInt64 0 + 614. 613 UInt64 1 + 615. 614 UInt64 0 + 616. 615 UInt64 1 + 617. 616 UInt64 0 + 618. 617 UInt64 1 + 619. 618 UInt64 0 + 620. 619 UInt64 1 + 621. 620 UInt64 0 + 622. 621 UInt64 1 + 623. 622 UInt64 0 + 624. 623 UInt64 1 + 625. 624 UInt64 0 + 626. 625 UInt64 1 + 627. 626 UInt64 0 + 628. 627 UInt64 1 + 629. 628 UInt64 0 + 630. 629 UInt64 1 + 631. 630 UInt64 0 + 632. 631 UInt64 1 + 633. 632 UInt64 0 + 634. 633 UInt64 1 + 635. 634 UInt64 0 + 636. 635 UInt64 1 + 637. 636 UInt64 0 + 638. 637 UInt64 1 + 639. 638 UInt64 0 + 640. 639 UInt64 1 + 641. 640 UInt64 0 + 642. 641 UInt64 1 + 643. 642 UInt64 0 + 644. 643 UInt64 1 + 645. 644 UInt64 0 + 646. 645 UInt64 1 + 647. 646 UInt64 0 + 648. 647 UInt64 1 + 649. 648 UInt64 0 + 650. 649 UInt64 1 + 651. 650 UInt64 0 + 652. 651 UInt64 1 + 653. 652 UInt64 0 + 654. 653 UInt64 1 + 655. 654 UInt64 0 + 656. 655 UInt64 1 + 657. 656 UInt64 0 + 658. 657 UInt64 1 + 659. 658 UInt64 0 + 660. 659 UInt64 1 + 661. 660 UInt64 0 + 662. 661 UInt64 1 + 663. 662 UInt64 0 + 664. 663 UInt64 1 + 665. 664 UInt64 0 + 666. 665 UInt64 1 + 667. 666 UInt64 0 + 668. 667 UInt64 1 + 669. 668 UInt64 0 + 670. 669 UInt64 1 + 671. 670 UInt64 0 + 672. 671 UInt64 1 + 673. 672 UInt64 0 + 674. 673 UInt64 1 + 675. 674 UInt64 0 + 676. 675 UInt64 1 + 677. 676 UInt64 0 + 678. 677 UInt64 1 + 679. 678 UInt64 0 + 680. 679 UInt64 1 + 681. 680 UInt64 0 + 682. 681 UInt64 1 + 683. 682 UInt64 0 + 684. 683 UInt64 1 + 685. 684 UInt64 0 + 686. 685 UInt64 1 + 687. 686 UInt64 0 + 688. 687 UInt64 1 + 689. 688 UInt64 0 + 690. 689 UInt64 1 + 691. 690 UInt64 0 + 692. 691 UInt64 1 + 693. 692 UInt64 0 + 694. 693 UInt64 1 + 695. 694 UInt64 0 + 696. 695 UInt64 1 + 697. 696 UInt64 0 + 698. 697 UInt64 1 + 699. 698 UInt64 0 + 700. 699 UInt64 1 + 701. 700 UInt64 0 + 702. 701 UInt64 1 + 703. 702 UInt64 0 + 704. 703 UInt64 1 + 705. 704 UInt64 0 + 706. 705 UInt64 1 + 707. 706 UInt64 0 + 708. 707 UInt64 1 + 709. 708 UInt64 0 + 710. 709 UInt64 1 + 711. 710 UInt64 0 + 712. 711 UInt64 1 + 713. 712 UInt64 0 + 714. 713 UInt64 1 + 715. 714 UInt64 0 + 716. 715 UInt64 1 + 717. 716 UInt64 0 + 718. 717 UInt64 1 + 719. 718 UInt64 0 + 720. 719 UInt64 1 + 721. 720 UInt64 0 + 722. 721 UInt64 1 + 723. 722 UInt64 0 + 724. 723 UInt64 1 + 725. 724 UInt64 0 + 726. 725 UInt64 1 + 727. 726 UInt64 0 + 728. 727 UInt64 1 + 729. 728 UInt64 0 + 730. 729 UInt64 1 + 731. 730 UInt64 0 + 732. 731 UInt64 1 + 733. 732 UInt64 0 + 734. 733 UInt64 1 + 735. 734 UInt64 0 + 736. 735 UInt64 1 + 737. 736 UInt64 0 + 738. 737 UInt64 1 + 739. 738 UInt64 0 + 740. 739 UInt64 1 + 741. 740 UInt64 0 + 742. 741 UInt64 1 + 743. 742 UInt64 0 + 744. 743 UInt64 1 + 745. 744 UInt64 0 + 746. 745 UInt64 1 + 747. 746 UInt64 0 + 748. 747 UInt64 1 + 749. 748 UInt64 0 + 750. 749 UInt64 1 + 751. 750 UInt64 0 + 752. 751 UInt64 1 + 753. 752 UInt64 0 + 754. 753 UInt64 1 + 755. 754 UInt64 0 + 756. 755 UInt64 1 + 757. 756 UInt64 0 + 758. 757 UInt64 1 + 759. 758 UInt64 0 + 760. 759 UInt64 1 + 761. 760 UInt64 0 + 762. 761 UInt64 1 + 763. 762 UInt64 0 + 764. 763 UInt64 1 + 765. 764 UInt64 0 + 766. 765 UInt64 1 + 767. 766 UInt64 0 + 768. 767 UInt64 1 + 769. 768 UInt64 0 + 770. 769 UInt64 1 + 771. 770 UInt64 0 + 772. 771 UInt64 1 + 773. 772 UInt64 0 + 774. 773 UInt64 1 + 775. 774 UInt64 0 + 776. 775 UInt64 1 + 777. 776 UInt64 0 + 778. 777 UInt64 1 + 779. 778 UInt64 0 + 780. 779 UInt64 1 + 781. 780 UInt64 0 + 782. 781 UInt64 1 + 783. 782 UInt64 0 + 784. 783 UInt64 1 + 785. 784 UInt64 0 + 786. 785 UInt64 1 + 787. 786 UInt64 0 + 788. 787 UInt64 1 + 789. 788 UInt64 0 + 790. 789 UInt64 1 + 791. 790 UInt64 0 + 792. 791 UInt64 1 + 793. 792 UInt64 0 + 794. 793 UInt64 1 + 795. 794 UInt64 0 + 796. 795 UInt64 1 + 797. 796 UInt64 0 + 798. 797 UInt64 1 + 799. 798 UInt64 0 + 800. 799 UInt64 1 + 801. 800 UInt64 0 + 802. 801 UInt64 1 + 803. 802 UInt64 0 + 804. 803 UInt64 1 + 805. 804 UInt64 0 + 806. 805 UInt64 1 + 807. 806 UInt64 0 + 808. 807 UInt64 1 + 809. 808 UInt64 0 + 810. 809 UInt64 1 + 811. 810 UInt64 0 + 812. 811 UInt64 1 + 813. 812 UInt64 0 + 814. 813 UInt64 1 + 815. 814 UInt64 0 + 816. 815 UInt64 1 + 817. 816 UInt64 0 + 818. 817 UInt64 1 + 819. 818 UInt64 0 + 820. 819 UInt64 1 + 821. 820 UInt64 0 + 822. 821 UInt64 1 + 823. 822 UInt64 0 + 824. 823 UInt64 1 + 825. 824 UInt64 0 + 826. 825 UInt64 1 + 827. 826 UInt64 0 + 828. 827 UInt64 1 + 829. 828 UInt64 0 + 830. 829 UInt64 1 + 831. 830 UInt64 0 + 832. 831 UInt64 1 + 833. 832 UInt64 0 + 834. 833 UInt64 1 + 835. 834 UInt64 0 + 836. 835 UInt64 1 + 837. 836 UInt64 0 + 838. 837 UInt64 1 + 839. 838 UInt64 0 + 840. 839 UInt64 1 + 841. 840 UInt64 0 + 842. 841 UInt64 1 + 843. 842 UInt64 0 + 844. 843 UInt64 1 + 845. 844 UInt64 0 + 846. 845 UInt64 1 + 847. 846 UInt64 0 + 848. 847 UInt64 1 + 849. 848 UInt64 0 + 850. 849 UInt64 1 + 851. 850 UInt64 0 + 852. 851 UInt64 1 + 853. 852 UInt64 0 + 854. 853 UInt64 1 + 855. 854 UInt64 0 + 856. 855 UInt64 1 + 857. 856 UInt64 0 + 858. 857 UInt64 1 + 859. 858 UInt64 0 + 860. 859 UInt64 1 + 861. 860 UInt64 0 + 862. 861 UInt64 1 + 863. 862 UInt64 0 + 864. 863 UInt64 1 + 865. 864 UInt64 0 + 866. 865 UInt64 1 + 867. 866 UInt64 0 + 868. 867 UInt64 1 + 869. 868 UInt64 0 + 870. 869 UInt64 1 + 871. 870 UInt64 0 + 872. 871 UInt64 1 + 873. 872 UInt64 0 + 874. 873 UInt64 1 + 875. 874 UInt64 0 + 876. 875 UInt64 1 + 877. 876 UInt64 0 + 878. 877 UInt64 1 + 879. 878 UInt64 0 + 880. 879 UInt64 1 + 881. 880 UInt64 0 + 882. 881 UInt64 1 + 883. 882 UInt64 0 + 884. 883 UInt64 1 + 885. 884 UInt64 0 + 886. 885 UInt64 1 + 887. 886 UInt64 0 + 888. 887 UInt64 1 + 889. 888 UInt64 0 + 890. 889 UInt64 1 + 891. 890 UInt64 0 + 892. 891 UInt64 1 + 893. 892 UInt64 0 + 894. 893 UInt64 1 + 895. 894 UInt64 0 + 896. 895 UInt64 1 + 897. 896 UInt64 0 + 898. 897 UInt64 1 + 899. 898 UInt64 0 + 900. 899 UInt64 1 + 901. 900 UInt64 0 + 902. 901 UInt64 1 + 903. 902 UInt64 0 + 904. 903 UInt64 1 + 905. 904 UInt64 0 + 906. 905 UInt64 1 + 907. 906 UInt64 0 + 908. 907 UInt64 1 + 909. 908 UInt64 0 + 910. 909 UInt64 1 + 911. 910 UInt64 0 + 912. 911 UInt64 1 + 913. 912 UInt64 0 + 914. 913 UInt64 1 + 915. 914 UInt64 0 + 916. 915 UInt64 1 + 917. 916 UInt64 0 + 918. 917 UInt64 1 + 919. 918 UInt64 0 + 920. 919 UInt64 1 + 921. 920 UInt64 0 + 922. 921 UInt64 1 + 923. 922 UInt64 0 + 924. 923 UInt64 1 + 925. 924 UInt64 0 + 926. 925 UInt64 1 + 927. 926 UInt64 0 + 928. 927 UInt64 1 + 929. 928 UInt64 0 + 930. 929 UInt64 1 + 931. 930 UInt64 0 + 932. 931 UInt64 1 + 933. 932 UInt64 0 + 934. 933 UInt64 1 + 935. 934 UInt64 0 + 936. 935 UInt64 1 + 937. 936 UInt64 0 + 938. 937 UInt64 1 + 939. 938 UInt64 0 + 940. 939 UInt64 1 + 941. 940 UInt64 0 + 942. 941 UInt64 1 + 943. 942 UInt64 0 + 944. 943 UInt64 1 + 945. 944 UInt64 0 + 946. 945 UInt64 1 + 947. 946 UInt64 0 + 948. 947 UInt64 1 + 949. 948 UInt64 0 + 950. 949 UInt64 1 + 951. 950 UInt64 0 + 952. 951 UInt64 1 + 953. 952 UInt64 0 + 954. 953 UInt64 1 + 955. 954 UInt64 0 + 956. 955 UInt64 1 + 957. 956 UInt64 0 + 958. 957 UInt64 1 + 959. 958 UInt64 0 + 960. 959 UInt64 1 + 961. 960 UInt64 0 + 962. 961 UInt64 1 + 963. 962 UInt64 0 + 964. 963 UInt64 1 + 965. 964 UInt64 0 + 966. 965 UInt64 1 + 967. 966 UInt64 0 + 968. 967 UInt64 1 + 969. 968 UInt64 0 + 970. 969 UInt64 1 + 971. 970 UInt64 0 + 972. 971 UInt64 1 + 973. 972 UInt64 0 + 974. 973 UInt64 1 + 975. 974 UInt64 0 + 976. 975 UInt64 1 + 977. 976 UInt64 0 + 978. 977 UInt64 1 + 979. 978 UInt64 0 + 980. 979 UInt64 1 + 981. 980 UInt64 0 + 982. 981 UInt64 1 + 983. 982 UInt64 0 + 984. 983 UInt64 1 + 985. 984 UInt64 0 + 986. 985 UInt64 1 + 987. 986 UInt64 0 + 988. 987 UInt64 1 + 989. 988 UInt64 0 + 990. 989 UInt64 1 + 991. 990 UInt64 0 + 992. 991 UInt64 1 + 993. 992 UInt64 0 + 994. 993 UInt64 1 + 995. 994 UInt64 0 + 996. 995 UInt64 1 + 997. 996 UInt64 0 + 998. 997 UInt64 1 + 999. 998 UInt64 0 +1000. 999 UInt64 1 + + number toTypeName(number) mod(number, 2) + diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.sql b/tests/queries/0_stateless/03169_display_column_names_in_footer.sql index fa7f50d52cc..8d2e36da74a 100644 --- a/tests/queries/0_stateless/03169_display_column_names_in_footer.sql +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.sql @@ -1,2 +1,14 @@ -- https://github.com/ClickHouse/ClickHouse/issues/65035 -SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 1000); \ No newline at end of file +SET output_format_pretty_display_footer_column_names=1; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT Pretty; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyNoEscapes; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyNoEscapesMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyCompact; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyCompactNoEscapes; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyCompactMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyCompactNoEscapesMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettySpace; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettySpaceNoEscapes; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettySpaceMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettySpaceNoEscapesMonoBlock; From 17760a93dfc29c78861799b1d5632db49b2eb5ac Mon Sep 17 00:00:00 2001 From: 0x01f Date: Tue, 28 May 2024 13:07:23 +0800 Subject: [PATCH 561/856] 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 09e59765c52705a1f68f0224b047990d5540ef75 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 12 Jun 2024 09:45:36 +0200 Subject: [PATCH 562/856] Add setting output_format_pretty_display_footer_column_names_min_rows --- docs/en/interfaces/formats.md | 3 +- .../operations/settings/settings-formats.md | 9 +- src/Core/Settings.h | 3 +- src/Core/SettingsChangesHistory.h | 3 +- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 3 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 2 +- .../Impl/PrettyCompactBlockOutputFormat.cpp | 2 +- .../Impl/PrettySpaceBlockOutputFormat.cpp | 2 +- ...9_display_column_names_in_footer.reference | 18410 ++-------------- .../03169_display_column_names_in_footer.sql | 31 +- 11 files changed, 2411 insertions(+), 16058 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index be3fc078070..228f6c0a159 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1490,7 +1490,8 @@ Differs from [PrettySpaceNoEscapes](#prettyspacenoescapes) in that up to 10,000 - [output_format_pretty_color](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_color) - use ANSI escape sequences to paint colors in Pretty formats. Default value - `true`. - [output_format_pretty_grid_charset](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_grid_charset) - Charset for printing grid borders. Available charsets: ASCII, UTF-8. Default value - `UTF-8`. - [output_format_pretty_row_numbers](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_row_numbers) - Add row numbers before each row for pretty output format. Default value - `true`. -- [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) - Display column names in the footer if there are more than 1000 rows. Default value - `false`. +- [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) - Display column names in the footer if table contains many rows. Default value - `true`. +- [output_format_pretty_display_footer_column_names_min_rows](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names_min_rows) - Sets the minimum number of rows for which a footer will be displayed if [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) is enabled. Default value - 50. ## RowBinary {#rowbinary} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 87ab0d17675..0a6f26e8e87 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1708,12 +1708,14 @@ Result: ## output_format_pretty_display_footer_column_names -Display column names in the footer if there are 999 or more rows. +Display column names in the footer if there are many table rows. The minimum number of rows for which Possible values: - 0 — No column names are displayed in the footer. -- 1 — Column names are displayed in the footer if row count is greater than or equal to 999. +- 1 — Column names are displayed in the footer if row count is greater than or equal to the threshold value set by [output_format_pretty_display_footer_column_names_min_rows](#output_format_pretty_display_footer_column_names_min_rows). + +Default value: `1`. **Example** @@ -1735,8 +1737,11 @@ Result: 1000. │ 999 │ UInt64 │ └─number─┴─toTypeName(number)─┘ ``` +## output_format_pretty_display_footer_column_names_min_rows +Sets the minimum number of rows for which a footer with column names will be displayed if setting [output_format_pretty_display_footer_column_names](#output_format_pretty_display_footer_column_names) is enabled. +Default value: `50`. ## Template format settings {#template-format-settings} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 606441352e9..b39a1fad505 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1140,7 +1140,8 @@ class IColumn; M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \ M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \ M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ - M(UInt64, output_format_pretty_display_footer_column_names, false, "Display column names in the footer if there are 999 or more rows.", 0) \ + M(UInt64, output_format_pretty_display_footer_column_names, true, "Display column names in the footer if there are 999 or more rows.", 0) \ + M(UInt64, output_format_pretty_display_footer_column_names_min_rows, 50, "Sets the minimum threshold value of rows for which to enable displaying column names in the footer. 50 (default)", 0) \ M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \ M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \ M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 6a6c42c67f5..670ffa76d62 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -105,7 +105,8 @@ static const std::map= 999) && format_settings.pretty.output_format_pretty_display_footer_column_names) + if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names) { writeString(footer_top_separator_s, out); diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 86fbae3a671..57ec23e7e3b 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -210,7 +210,7 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i) writeRow(i, header, chunk, widths, max_widths); - if ((num_rows >= 999) && format_settings.pretty.output_format_pretty_display_footer_column_names) + if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names) { writeHeader(header, max_widths, name_widths, true); } diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 8f13563421d..40df517fec8 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -110,7 +110,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port if (format_settings.pretty.output_format_pretty_row_numbers && format_settings.pretty.output_format_pretty_display_footer_column_names) writeString(String(row_number_width, ' '), out); /// Write footer - if ((num_rows >= 999) && format_settings.pretty.output_format_pretty_display_footer_column_names) + if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names) write_names(); total_rows += num_rows; } diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference index a8818991e1b..426b7798090 100644 --- a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference @@ -1,16048 +1,2386 @@ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 100. │ 99 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 101. │ 100 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 102. │ 101 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 103. │ 102 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 104. │ 103 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 105. │ 104 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 106. │ 105 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 107. │ 106 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 108. │ 107 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 109. │ 108 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 110. │ 109 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 111. │ 110 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 112. │ 111 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 113. │ 112 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 114. │ 113 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 115. │ 114 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 116. │ 115 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 117. │ 116 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 118. │ 117 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 119. │ 118 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 120. │ 119 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 121. │ 120 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 122. │ 121 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 123. │ 122 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 124. │ 123 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 125. │ 124 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 126. │ 125 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 127. │ 126 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 128. │ 127 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 129. │ 128 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 130. │ 129 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 131. │ 130 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 132. │ 131 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 133. │ 132 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 134. │ 133 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 135. │ 134 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 136. │ 135 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 137. │ 136 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 138. │ 137 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 139. │ 138 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 140. │ 139 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 141. │ 140 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 142. │ 141 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 143. │ 142 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 144. │ 143 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 145. │ 144 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 146. │ 145 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 147. │ 146 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 148. │ 147 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 149. │ 148 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 150. │ 149 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 151. │ 150 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 152. │ 151 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 153. │ 152 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 154. │ 153 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 155. │ 154 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 156. │ 155 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 157. │ 156 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 158. │ 157 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 159. │ 158 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 160. │ 159 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 161. │ 160 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 162. │ 161 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 163. │ 162 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 164. │ 163 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 165. │ 164 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 166. │ 165 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 167. │ 166 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 168. │ 167 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 169. │ 168 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 170. │ 169 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 171. │ 170 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 172. │ 171 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 173. │ 172 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 174. │ 173 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 175. │ 174 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 176. │ 175 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 177. │ 176 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 178. │ 177 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 179. │ 178 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 180. │ 179 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 181. │ 180 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 182. │ 181 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 183. │ 182 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 184. │ 183 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 185. │ 184 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 186. │ 185 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 187. │ 186 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 188. │ 187 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 189. │ 188 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 190. │ 189 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 191. │ 190 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 192. │ 191 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 193. │ 192 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 194. │ 193 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 195. │ 194 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 196. │ 195 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 197. │ 196 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 198. │ 197 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 199. │ 198 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 200. │ 199 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 201. │ 200 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 202. │ 201 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 203. │ 202 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 204. │ 203 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 205. │ 204 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 206. │ 205 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 207. │ 206 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 208. │ 207 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 209. │ 208 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 210. │ 209 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 211. │ 210 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 212. │ 211 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 213. │ 212 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 214. │ 213 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 215. │ 214 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 216. │ 215 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 217. │ 216 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 218. │ 217 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 219. │ 218 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 220. │ 219 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 221. │ 220 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 222. │ 221 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 223. │ 222 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 224. │ 223 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 225. │ 224 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 226. │ 225 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 227. │ 226 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 228. │ 227 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 229. │ 228 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 230. │ 229 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 231. │ 230 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 232. │ 231 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 233. │ 232 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 234. │ 233 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 235. │ 234 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 236. │ 235 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 237. │ 236 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 238. │ 237 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 239. │ 238 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 240. │ 239 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 241. │ 240 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 242. │ 241 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 243. │ 242 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 244. │ 243 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 245. │ 244 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 246. │ 245 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 247. │ 246 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 248. │ 247 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 249. │ 248 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 250. │ 249 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 251. │ 250 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 252. │ 251 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 253. │ 252 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 254. │ 253 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 255. │ 254 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 256. │ 255 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 257. │ 256 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 258. │ 257 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 259. │ 258 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 260. │ 259 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 261. │ 260 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 262. │ 261 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 263. │ 262 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 264. │ 263 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 265. │ 264 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 266. │ 265 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 267. │ 266 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 268. │ 267 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 269. │ 268 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 270. │ 269 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 271. │ 270 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 272. │ 271 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 273. │ 272 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 274. │ 273 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 275. │ 274 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 276. │ 275 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 277. │ 276 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 278. │ 277 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 279. │ 278 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 280. │ 279 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 281. │ 280 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 282. │ 281 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 283. │ 282 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 284. │ 283 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 285. │ 284 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 286. │ 285 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 287. │ 286 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 288. │ 287 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 289. │ 288 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 290. │ 289 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 291. │ 290 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 292. │ 291 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 293. │ 292 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 294. │ 293 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 295. │ 294 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 296. │ 295 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 297. │ 296 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 298. │ 297 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 299. │ 298 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 300. │ 299 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 301. │ 300 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 302. │ 301 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 303. │ 302 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 304. │ 303 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 305. │ 304 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 306. │ 305 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 307. │ 306 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 308. │ 307 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 309. │ 308 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 310. │ 309 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 311. │ 310 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 312. │ 311 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 313. │ 312 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 314. │ 313 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 315. │ 314 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 316. │ 315 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 317. │ 316 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 318. │ 317 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 319. │ 318 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 320. │ 319 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 321. │ 320 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 322. │ 321 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 323. │ 322 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 324. │ 323 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 325. │ 324 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 326. │ 325 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 327. │ 326 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 328. │ 327 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 329. │ 328 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 330. │ 329 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 331. │ 330 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 332. │ 331 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 333. │ 332 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 334. │ 333 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 335. │ 334 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 336. │ 335 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 337. │ 336 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 338. │ 337 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 339. │ 338 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 340. │ 339 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 341. │ 340 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 342. │ 341 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 343. │ 342 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 344. │ 343 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 345. │ 344 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 346. │ 345 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 347. │ 346 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 348. │ 347 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 349. │ 348 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 350. │ 349 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 351. │ 350 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 352. │ 351 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 353. │ 352 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 354. │ 353 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 355. │ 354 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 356. │ 355 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 357. │ 356 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 358. │ 357 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 359. │ 358 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 360. │ 359 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 361. │ 360 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 362. │ 361 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 363. │ 362 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 364. │ 363 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 365. │ 364 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 366. │ 365 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 367. │ 366 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 368. │ 367 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 369. │ 368 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 370. │ 369 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 371. │ 370 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 372. │ 371 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 373. │ 372 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 374. │ 373 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 375. │ 374 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 376. │ 375 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 377. │ 376 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 378. │ 377 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 379. │ 378 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 380. │ 379 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 381. │ 380 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 382. │ 381 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 383. │ 382 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 384. │ 383 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 385. │ 384 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 386. │ 385 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 387. │ 386 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 388. │ 387 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 389. │ 388 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 390. │ 389 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 391. │ 390 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 392. │ 391 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 393. │ 392 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 394. │ 393 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 395. │ 394 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 396. │ 395 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 397. │ 396 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 398. │ 397 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 399. │ 398 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 400. │ 399 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 401. │ 400 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 402. │ 401 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 403. │ 402 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 404. │ 403 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 405. │ 404 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 406. │ 405 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 407. │ 406 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 408. │ 407 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 409. │ 408 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 410. │ 409 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 411. │ 410 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 412. │ 411 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 413. │ 412 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 414. │ 413 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 415. │ 414 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 416. │ 415 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 417. │ 416 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 418. │ 417 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 419. │ 418 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 420. │ 419 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 421. │ 420 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 422. │ 421 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 423. │ 422 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 424. │ 423 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 425. │ 424 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 426. │ 425 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 427. │ 426 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 428. │ 427 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 429. │ 428 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 430. │ 429 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 431. │ 430 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 432. │ 431 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 433. │ 432 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 434. │ 433 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 435. │ 434 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 436. │ 435 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 437. │ 436 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 438. │ 437 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 439. │ 438 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 440. │ 439 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 441. │ 440 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 442. │ 441 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 443. │ 442 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 444. │ 443 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 445. │ 444 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 446. │ 445 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 447. │ 446 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 448. │ 447 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 449. │ 448 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 450. │ 449 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 451. │ 450 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 452. │ 451 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 453. │ 452 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 454. │ 453 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 455. │ 454 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 456. │ 455 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 457. │ 456 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 458. │ 457 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 459. │ 458 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 460. │ 459 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 461. │ 460 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 462. │ 461 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 463. │ 462 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 464. │ 463 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 465. │ 464 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 466. │ 465 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 467. │ 466 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 468. │ 467 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 469. │ 468 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 470. │ 469 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 471. │ 470 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 472. │ 471 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 473. │ 472 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 474. │ 473 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 475. │ 474 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 476. │ 475 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 477. │ 476 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 478. │ 477 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 479. │ 478 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 480. │ 479 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 481. │ 480 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 482. │ 481 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 483. │ 482 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 484. │ 483 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 485. │ 484 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 486. │ 485 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 487. │ 486 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 488. │ 487 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 489. │ 488 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 490. │ 489 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 491. │ 490 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 492. │ 491 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 493. │ 492 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 494. │ 493 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 495. │ 494 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 496. │ 495 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 497. │ 496 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 498. │ 497 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 499. │ 498 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 500. │ 499 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 501. │ 500 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 502. │ 501 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 503. │ 502 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 504. │ 503 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 505. │ 504 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 506. │ 505 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 507. │ 506 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 508. │ 507 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 509. │ 508 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 510. │ 509 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 511. │ 510 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 512. │ 511 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 513. │ 512 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 514. │ 513 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 515. │ 514 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 516. │ 515 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 517. │ 516 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 518. │ 517 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 519. │ 518 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 520. │ 519 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 521. │ 520 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 522. │ 521 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 523. │ 522 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 524. │ 523 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 525. │ 524 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 526. │ 525 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 527. │ 526 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 528. │ 527 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 529. │ 528 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 530. │ 529 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 531. │ 530 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 532. │ 531 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 533. │ 532 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 534. │ 533 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 535. │ 534 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 536. │ 535 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 537. │ 536 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 538. │ 537 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 539. │ 538 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 540. │ 539 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 541. │ 540 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 542. │ 541 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 543. │ 542 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 544. │ 543 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 545. │ 544 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 546. │ 545 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 547. │ 546 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 548. │ 547 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 549. │ 548 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 550. │ 549 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 551. │ 550 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 552. │ 551 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 553. │ 552 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 554. │ 553 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 555. │ 554 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 556. │ 555 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 557. │ 556 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 558. │ 557 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 559. │ 558 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 560. │ 559 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 561. │ 560 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 562. │ 561 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 563. │ 562 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 564. │ 563 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 565. │ 564 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 566. │ 565 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 567. │ 566 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 568. │ 567 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 569. │ 568 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 570. │ 569 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 571. │ 570 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 572. │ 571 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 573. │ 572 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 574. │ 573 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 575. │ 574 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 576. │ 575 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 577. │ 576 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 578. │ 577 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 579. │ 578 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 580. │ 579 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 581. │ 580 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 582. │ 581 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 583. │ 582 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 584. │ 583 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 585. │ 584 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 586. │ 585 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 587. │ 586 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 588. │ 587 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 589. │ 588 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 590. │ 589 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 591. │ 590 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 592. │ 591 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 593. │ 592 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 594. │ 593 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 595. │ 594 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 596. │ 595 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 597. │ 596 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 598. │ 597 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 599. │ 598 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 600. │ 599 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 601. │ 600 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 602. │ 601 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 603. │ 602 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 604. │ 603 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 605. │ 604 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 606. │ 605 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 607. │ 606 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 608. │ 607 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 609. │ 608 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 610. │ 609 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 611. │ 610 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 612. │ 611 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 613. │ 612 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 614. │ 613 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 615. │ 614 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 616. │ 615 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 617. │ 616 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 618. │ 617 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 619. │ 618 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 620. │ 619 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 621. │ 620 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 622. │ 621 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 623. │ 622 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 624. │ 623 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 625. │ 624 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 626. │ 625 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 627. │ 626 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 628. │ 627 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 629. │ 628 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 630. │ 629 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 631. │ 630 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 632. │ 631 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 633. │ 632 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 634. │ 633 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 635. │ 634 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 636. │ 635 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 637. │ 636 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 638. │ 637 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 639. │ 638 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 640. │ 639 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 641. │ 640 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 642. │ 641 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 643. │ 642 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 644. │ 643 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 645. │ 644 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 646. │ 645 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 647. │ 646 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 648. │ 647 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 649. │ 648 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 650. │ 649 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 651. │ 650 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 652. │ 651 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 653. │ 652 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 654. │ 653 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 655. │ 654 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 656. │ 655 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 657. │ 656 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 658. │ 657 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 659. │ 658 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 660. │ 659 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 661. │ 660 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 662. │ 661 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 663. │ 662 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 664. │ 663 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 665. │ 664 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 666. │ 665 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 667. │ 666 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 668. │ 667 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 669. │ 668 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 670. │ 669 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 671. │ 670 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 672. │ 671 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 673. │ 672 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 674. │ 673 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 675. │ 674 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 676. │ 675 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 677. │ 676 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 678. │ 677 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 679. │ 678 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 680. │ 679 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 681. │ 680 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 682. │ 681 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 683. │ 682 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 684. │ 683 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 685. │ 684 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 686. │ 685 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 687. │ 686 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 688. │ 687 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 689. │ 688 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 690. │ 689 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 691. │ 690 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 692. │ 691 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 693. │ 692 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 694. │ 693 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 695. │ 694 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 696. │ 695 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 697. │ 696 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 698. │ 697 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 699. │ 698 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 700. │ 699 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 701. │ 700 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 702. │ 701 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 703. │ 702 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 704. │ 703 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 705. │ 704 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 706. │ 705 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 707. │ 706 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 708. │ 707 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 709. │ 708 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 710. │ 709 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 711. │ 710 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 712. │ 711 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 713. │ 712 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 714. │ 713 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 715. │ 714 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 716. │ 715 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 717. │ 716 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 718. │ 717 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 719. │ 718 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 720. │ 719 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 721. │ 720 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 722. │ 721 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 723. │ 722 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 724. │ 723 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 725. │ 724 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 726. │ 725 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 727. │ 726 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 728. │ 727 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 729. │ 728 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 730. │ 729 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 731. │ 730 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 732. │ 731 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 733. │ 732 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 734. │ 733 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 735. │ 734 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 736. │ 735 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 737. │ 736 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 738. │ 737 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 739. │ 738 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 740. │ 739 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 741. │ 740 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 742. │ 741 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 743. │ 742 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 744. │ 743 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 745. │ 744 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 746. │ 745 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 747. │ 746 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 748. │ 747 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 749. │ 748 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 750. │ 749 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 751. │ 750 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 752. │ 751 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 753. │ 752 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 754. │ 753 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 755. │ 754 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 756. │ 755 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 757. │ 756 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 758. │ 757 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 759. │ 758 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 760. │ 759 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 761. │ 760 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 762. │ 761 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 763. │ 762 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 764. │ 763 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 765. │ 764 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 766. │ 765 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 767. │ 766 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 768. │ 767 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 769. │ 768 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 770. │ 769 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 771. │ 770 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 772. │ 771 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 773. │ 772 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 774. │ 773 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 775. │ 774 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 776. │ 775 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 777. │ 776 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 778. │ 777 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 779. │ 778 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 780. │ 779 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 781. │ 780 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 782. │ 781 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 783. │ 782 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 784. │ 783 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 785. │ 784 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 786. │ 785 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 787. │ 786 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 788. │ 787 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 789. │ 788 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 790. │ 789 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 791. │ 790 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 792. │ 791 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 793. │ 792 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 794. │ 793 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 795. │ 794 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 796. │ 795 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 797. │ 796 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 798. │ 797 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 799. │ 798 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 800. │ 799 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 801. │ 800 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 802. │ 801 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 803. │ 802 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 804. │ 803 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 805. │ 804 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 806. │ 805 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 807. │ 806 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 808. │ 807 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 809. │ 808 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 810. │ 809 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 811. │ 810 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 812. │ 811 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 813. │ 812 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 814. │ 813 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 815. │ 814 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 816. │ 815 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 817. │ 816 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 818. │ 817 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 819. │ 818 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 820. │ 819 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 821. │ 820 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 822. │ 821 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 823. │ 822 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 824. │ 823 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 825. │ 824 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 826. │ 825 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 827. │ 826 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 828. │ 827 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 829. │ 828 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 830. │ 829 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 831. │ 830 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 832. │ 831 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 833. │ 832 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 834. │ 833 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 835. │ 834 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 836. │ 835 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 837. │ 836 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 838. │ 837 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 839. │ 838 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 840. │ 839 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 841. │ 840 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 842. │ 841 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 843. │ 842 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 844. │ 843 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 845. │ 844 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 846. │ 845 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 847. │ 846 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 848. │ 847 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 849. │ 848 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 850. │ 849 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 851. │ 850 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 852. │ 851 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 853. │ 852 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 854. │ 853 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 855. │ 854 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 856. │ 855 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 857. │ 856 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 858. │ 857 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 859. │ 858 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 860. │ 859 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 861. │ 860 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 862. │ 861 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 863. │ 862 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 864. │ 863 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 865. │ 864 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 866. │ 865 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 867. │ 866 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 868. │ 867 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 869. │ 868 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 870. │ 869 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 871. │ 870 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 872. │ 871 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 873. │ 872 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 874. │ 873 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 875. │ 874 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 876. │ 875 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 877. │ 876 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 878. │ 877 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 879. │ 878 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 880. │ 879 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 881. │ 880 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 882. │ 881 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 883. │ 882 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 884. │ 883 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 885. │ 884 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 886. │ 885 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 887. │ 886 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 888. │ 887 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 889. │ 888 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 890. │ 889 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 891. │ 890 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 892. │ 891 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 893. │ 892 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 894. │ 893 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 895. │ 894 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 896. │ 895 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 897. │ 896 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 898. │ 897 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 899. │ 898 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 900. │ 899 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 901. │ 900 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 902. │ 901 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 903. │ 902 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 904. │ 903 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 905. │ 904 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 906. │ 905 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 907. │ 906 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 908. │ 907 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 909. │ 908 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 910. │ 909 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 911. │ 910 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 912. │ 911 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 913. │ 912 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 914. │ 913 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 915. │ 914 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 916. │ 915 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 917. │ 916 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 918. │ 917 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 919. │ 918 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 920. │ 919 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 921. │ 920 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 922. │ 921 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 923. │ 922 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 924. │ 923 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 925. │ 924 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 926. │ 925 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 927. │ 926 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 928. │ 927 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 929. │ 928 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 930. │ 929 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 931. │ 930 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 932. │ 931 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 933. │ 932 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 934. │ 933 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 935. │ 934 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 936. │ 935 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 937. │ 936 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 938. │ 937 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 939. │ 938 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 940. │ 939 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 941. │ 940 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 942. │ 941 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 943. │ 942 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 944. │ 943 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 945. │ 944 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 946. │ 945 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 947. │ 946 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 948. │ 947 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 949. │ 948 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 950. │ 949 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 951. │ 950 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 952. │ 951 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 953. │ 952 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 954. │ 953 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 955. │ 954 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 956. │ 955 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 957. │ 956 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 958. │ 957 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 959. │ 958 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 960. │ 959 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 961. │ 960 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 962. │ 961 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 963. │ 962 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 964. │ 963 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 965. │ 964 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 966. │ 965 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 967. │ 966 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 968. │ 967 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 969. │ 968 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 970. │ 969 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 971. │ 970 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 972. │ 971 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 973. │ 972 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 974. │ 973 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 975. │ 974 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 976. │ 975 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 977. │ 976 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 978. │ 977 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 979. │ 978 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 980. │ 979 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 981. │ 980 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 982. │ 981 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 983. │ 982 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 984. │ 983 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 985. │ 984 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 986. │ 985 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 987. │ 986 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 988. │ 987 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 989. │ 988 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 990. │ 989 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 991. │ 990 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 992. │ 991 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 993. │ 992 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 994. │ 993 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 995. │ 994 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 996. │ 995 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 997. │ 996 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 998. │ 997 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 999. │ 998 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -1000. │ 999 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 100. │ 99 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 101. │ 100 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 102. │ 101 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 103. │ 102 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 104. │ 103 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 105. │ 104 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 106. │ 105 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 107. │ 106 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 108. │ 107 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 109. │ 108 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 110. │ 109 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 111. │ 110 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 112. │ 111 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 113. │ 112 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 114. │ 113 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 115. │ 114 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 116. │ 115 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 117. │ 116 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 118. │ 117 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 119. │ 118 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 120. │ 119 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 121. │ 120 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 122. │ 121 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 123. │ 122 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 124. │ 123 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 125. │ 124 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 126. │ 125 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 127. │ 126 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 128. │ 127 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 129. │ 128 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 130. │ 129 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 131. │ 130 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 132. │ 131 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 133. │ 132 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 134. │ 133 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 135. │ 134 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 136. │ 135 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 137. │ 136 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 138. │ 137 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 139. │ 138 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 140. │ 139 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 141. │ 140 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 142. │ 141 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 143. │ 142 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 144. │ 143 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 145. │ 144 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 146. │ 145 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 147. │ 146 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 148. │ 147 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 149. │ 148 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 150. │ 149 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 151. │ 150 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 152. │ 151 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 153. │ 152 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 154. │ 153 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 155. │ 154 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 156. │ 155 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 157. │ 156 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 158. │ 157 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 159. │ 158 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 160. │ 159 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 161. │ 160 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 162. │ 161 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 163. │ 162 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 164. │ 163 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 165. │ 164 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 166. │ 165 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 167. │ 166 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 168. │ 167 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 169. │ 168 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 170. │ 169 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 171. │ 170 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 172. │ 171 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 173. │ 172 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 174. │ 173 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 175. │ 174 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 176. │ 175 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 177. │ 176 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 178. │ 177 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 179. │ 178 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 180. │ 179 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 181. │ 180 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 182. │ 181 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 183. │ 182 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 184. │ 183 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 185. │ 184 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 186. │ 185 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 187. │ 186 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 188. │ 187 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 189. │ 188 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 190. │ 189 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 191. │ 190 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 192. │ 191 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 193. │ 192 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 194. │ 193 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 195. │ 194 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 196. │ 195 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 197. │ 196 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 198. │ 197 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 199. │ 198 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 200. │ 199 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 201. │ 200 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 202. │ 201 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 203. │ 202 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 204. │ 203 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 205. │ 204 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 206. │ 205 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 207. │ 206 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 208. │ 207 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 209. │ 208 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 210. │ 209 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 211. │ 210 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 212. │ 211 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 213. │ 212 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 214. │ 213 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 215. │ 214 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 216. │ 215 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 217. │ 216 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 218. │ 217 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 219. │ 218 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 220. │ 219 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 221. │ 220 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 222. │ 221 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 223. │ 222 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 224. │ 223 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 225. │ 224 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 226. │ 225 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 227. │ 226 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 228. │ 227 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 229. │ 228 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 230. │ 229 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 231. │ 230 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 232. │ 231 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 233. │ 232 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 234. │ 233 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 235. │ 234 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 236. │ 235 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 237. │ 236 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 238. │ 237 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 239. │ 238 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 240. │ 239 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 241. │ 240 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 242. │ 241 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 243. │ 242 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 244. │ 243 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 245. │ 244 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 246. │ 245 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 247. │ 246 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 248. │ 247 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 249. │ 248 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 250. │ 249 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 251. │ 250 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 252. │ 251 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 253. │ 252 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 254. │ 253 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 255. │ 254 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 256. │ 255 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 257. │ 256 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 258. │ 257 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 259. │ 258 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 260. │ 259 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 261. │ 260 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 262. │ 261 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 263. │ 262 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 264. │ 263 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 265. │ 264 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 266. │ 265 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 267. │ 266 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 268. │ 267 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 269. │ 268 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 270. │ 269 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 271. │ 270 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 272. │ 271 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 273. │ 272 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 274. │ 273 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 275. │ 274 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 276. │ 275 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 277. │ 276 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 278. │ 277 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 279. │ 278 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 280. │ 279 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 281. │ 280 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 282. │ 281 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 283. │ 282 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 284. │ 283 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 285. │ 284 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 286. │ 285 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 287. │ 286 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 288. │ 287 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 289. │ 288 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 290. │ 289 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 291. │ 290 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 292. │ 291 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 293. │ 292 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 294. │ 293 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 295. │ 294 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 296. │ 295 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 297. │ 296 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 298. │ 297 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 299. │ 298 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 300. │ 299 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 301. │ 300 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 302. │ 301 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 303. │ 302 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 304. │ 303 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 305. │ 304 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 306. │ 305 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 307. │ 306 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 308. │ 307 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 309. │ 308 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 310. │ 309 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 311. │ 310 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 312. │ 311 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 313. │ 312 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 314. │ 313 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 315. │ 314 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 316. │ 315 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 317. │ 316 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 318. │ 317 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 319. │ 318 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 320. │ 319 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 321. │ 320 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 322. │ 321 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 323. │ 322 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 324. │ 323 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 325. │ 324 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 326. │ 325 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 327. │ 326 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 328. │ 327 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 329. │ 328 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 330. │ 329 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 331. │ 330 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 332. │ 331 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 333. │ 332 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 334. │ 333 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 335. │ 334 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 336. │ 335 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 337. │ 336 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 338. │ 337 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 339. │ 338 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 340. │ 339 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 341. │ 340 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 342. │ 341 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 343. │ 342 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 344. │ 343 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 345. │ 344 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 346. │ 345 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 347. │ 346 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 348. │ 347 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 349. │ 348 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 350. │ 349 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 351. │ 350 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 352. │ 351 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 353. │ 352 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 354. │ 353 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 355. │ 354 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 356. │ 355 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 357. │ 356 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 358. │ 357 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 359. │ 358 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 360. │ 359 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 361. │ 360 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 362. │ 361 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 363. │ 362 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 364. │ 363 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 365. │ 364 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 366. │ 365 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 367. │ 366 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 368. │ 367 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 369. │ 368 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 370. │ 369 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 371. │ 370 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 372. │ 371 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 373. │ 372 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 374. │ 373 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 375. │ 374 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 376. │ 375 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 377. │ 376 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 378. │ 377 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 379. │ 378 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 380. │ 379 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 381. │ 380 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 382. │ 381 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 383. │ 382 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 384. │ 383 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 385. │ 384 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 386. │ 385 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 387. │ 386 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 388. │ 387 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 389. │ 388 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 390. │ 389 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 391. │ 390 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 392. │ 391 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 393. │ 392 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 394. │ 393 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 395. │ 394 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 396. │ 395 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 397. │ 396 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 398. │ 397 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 399. │ 398 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 400. │ 399 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 401. │ 400 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 402. │ 401 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 403. │ 402 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 404. │ 403 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 405. │ 404 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 406. │ 405 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 407. │ 406 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 408. │ 407 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 409. │ 408 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 410. │ 409 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 411. │ 410 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 412. │ 411 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 413. │ 412 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 414. │ 413 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 415. │ 414 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 416. │ 415 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 417. │ 416 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 418. │ 417 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 419. │ 418 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 420. │ 419 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 421. │ 420 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 422. │ 421 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 423. │ 422 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 424. │ 423 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 425. │ 424 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 426. │ 425 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 427. │ 426 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 428. │ 427 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 429. │ 428 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 430. │ 429 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 431. │ 430 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 432. │ 431 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 433. │ 432 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 434. │ 433 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 435. │ 434 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 436. │ 435 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 437. │ 436 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 438. │ 437 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 439. │ 438 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 440. │ 439 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 441. │ 440 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 442. │ 441 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 443. │ 442 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 444. │ 443 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 445. │ 444 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 446. │ 445 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 447. │ 446 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 448. │ 447 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 449. │ 448 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 450. │ 449 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 451. │ 450 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 452. │ 451 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 453. │ 452 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 454. │ 453 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 455. │ 454 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 456. │ 455 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 457. │ 456 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 458. │ 457 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 459. │ 458 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 460. │ 459 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 461. │ 460 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 462. │ 461 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 463. │ 462 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 464. │ 463 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 465. │ 464 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 466. │ 465 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 467. │ 466 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 468. │ 467 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 469. │ 468 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 470. │ 469 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 471. │ 470 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 472. │ 471 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 473. │ 472 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 474. │ 473 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 475. │ 474 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 476. │ 475 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 477. │ 476 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 478. │ 477 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 479. │ 478 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 480. │ 479 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 481. │ 480 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 482. │ 481 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 483. │ 482 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 484. │ 483 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 485. │ 484 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 486. │ 485 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 487. │ 486 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 488. │ 487 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 489. │ 488 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 490. │ 489 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 491. │ 490 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 492. │ 491 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 493. │ 492 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 494. │ 493 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 495. │ 494 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 496. │ 495 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 497. │ 496 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 498. │ 497 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 499. │ 498 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 500. │ 499 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 501. │ 500 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 502. │ 501 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 503. │ 502 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 504. │ 503 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 505. │ 504 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 506. │ 505 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 507. │ 506 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 508. │ 507 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 509. │ 508 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 510. │ 509 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 511. │ 510 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 512. │ 511 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 513. │ 512 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 514. │ 513 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 515. │ 514 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 516. │ 515 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 517. │ 516 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 518. │ 517 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 519. │ 518 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 520. │ 519 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 521. │ 520 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 522. │ 521 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 523. │ 522 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 524. │ 523 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 525. │ 524 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 526. │ 525 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 527. │ 526 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 528. │ 527 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 529. │ 528 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 530. │ 529 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 531. │ 530 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 532. │ 531 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 533. │ 532 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 534. │ 533 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 535. │ 534 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 536. │ 535 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 537. │ 536 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 538. │ 537 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 539. │ 538 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 540. │ 539 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 541. │ 540 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 542. │ 541 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 543. │ 542 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 544. │ 543 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 545. │ 544 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 546. │ 545 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 547. │ 546 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 548. │ 547 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 549. │ 548 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 550. │ 549 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 551. │ 550 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 552. │ 551 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 553. │ 552 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 554. │ 553 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 555. │ 554 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 556. │ 555 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 557. │ 556 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 558. │ 557 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 559. │ 558 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 560. │ 559 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 561. │ 560 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 562. │ 561 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 563. │ 562 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 564. │ 563 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 565. │ 564 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 566. │ 565 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 567. │ 566 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 568. │ 567 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 569. │ 568 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 570. │ 569 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 571. │ 570 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 572. │ 571 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 573. │ 572 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 574. │ 573 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 575. │ 574 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 576. │ 575 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 577. │ 576 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 578. │ 577 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 579. │ 578 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 580. │ 579 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 581. │ 580 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 582. │ 581 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 583. │ 582 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 584. │ 583 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 585. │ 584 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 586. │ 585 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 587. │ 586 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 588. │ 587 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 589. │ 588 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 590. │ 589 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 591. │ 590 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 592. │ 591 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 593. │ 592 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 594. │ 593 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 595. │ 594 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 596. │ 595 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 597. │ 596 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 598. │ 597 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 599. │ 598 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 600. │ 599 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 601. │ 600 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 602. │ 601 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 603. │ 602 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 604. │ 603 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 605. │ 604 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 606. │ 605 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 607. │ 606 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 608. │ 607 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 609. │ 608 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 610. │ 609 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 611. │ 610 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 612. │ 611 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 613. │ 612 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 614. │ 613 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 615. │ 614 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 616. │ 615 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 617. │ 616 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 618. │ 617 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 619. │ 618 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 620. │ 619 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 621. │ 620 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 622. │ 621 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 623. │ 622 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 624. │ 623 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 625. │ 624 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 626. │ 625 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 627. │ 626 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 628. │ 627 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 629. │ 628 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 630. │ 629 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 631. │ 630 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 632. │ 631 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 633. │ 632 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 634. │ 633 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 635. │ 634 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 636. │ 635 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 637. │ 636 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 638. │ 637 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 639. │ 638 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 640. │ 639 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 641. │ 640 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 642. │ 641 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 643. │ 642 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 644. │ 643 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 645. │ 644 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 646. │ 645 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 647. │ 646 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 648. │ 647 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 649. │ 648 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 650. │ 649 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 651. │ 650 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 652. │ 651 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 653. │ 652 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 654. │ 653 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 655. │ 654 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 656. │ 655 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 657. │ 656 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 658. │ 657 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 659. │ 658 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 660. │ 659 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 661. │ 660 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 662. │ 661 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 663. │ 662 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 664. │ 663 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 665. │ 664 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 666. │ 665 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 667. │ 666 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 668. │ 667 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 669. │ 668 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 670. │ 669 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 671. │ 670 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 672. │ 671 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 673. │ 672 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 674. │ 673 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 675. │ 674 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 676. │ 675 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 677. │ 676 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 678. │ 677 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 679. │ 678 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 680. │ 679 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 681. │ 680 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 682. │ 681 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 683. │ 682 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 684. │ 683 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 685. │ 684 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 686. │ 685 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 687. │ 686 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 688. │ 687 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 689. │ 688 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 690. │ 689 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 691. │ 690 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 692. │ 691 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 693. │ 692 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 694. │ 693 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 695. │ 694 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 696. │ 695 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 697. │ 696 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 698. │ 697 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 699. │ 698 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 700. │ 699 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 701. │ 700 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 702. │ 701 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 703. │ 702 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 704. │ 703 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 705. │ 704 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 706. │ 705 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 707. │ 706 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 708. │ 707 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 709. │ 708 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 710. │ 709 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 711. │ 710 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 712. │ 711 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 713. │ 712 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 714. │ 713 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 715. │ 714 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 716. │ 715 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 717. │ 716 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 718. │ 717 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 719. │ 718 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 720. │ 719 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 721. │ 720 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 722. │ 721 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 723. │ 722 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 724. │ 723 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 725. │ 724 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 726. │ 725 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 727. │ 726 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 728. │ 727 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 729. │ 728 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 730. │ 729 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 731. │ 730 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 732. │ 731 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 733. │ 732 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 734. │ 733 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 735. │ 734 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 736. │ 735 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 737. │ 736 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 738. │ 737 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 739. │ 738 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 740. │ 739 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 741. │ 740 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 742. │ 741 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 743. │ 742 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 744. │ 743 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 745. │ 744 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 746. │ 745 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 747. │ 746 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 748. │ 747 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 749. │ 748 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 750. │ 749 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 751. │ 750 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 752. │ 751 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 753. │ 752 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 754. │ 753 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 755. │ 754 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 756. │ 755 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 757. │ 756 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 758. │ 757 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 759. │ 758 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 760. │ 759 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 761. │ 760 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 762. │ 761 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 763. │ 762 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 764. │ 763 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 765. │ 764 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 766. │ 765 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 767. │ 766 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 768. │ 767 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 769. │ 768 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 770. │ 769 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 771. │ 770 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 772. │ 771 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 773. │ 772 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 774. │ 773 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 775. │ 774 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 776. │ 775 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 777. │ 776 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 778. │ 777 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 779. │ 778 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 780. │ 779 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 781. │ 780 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 782. │ 781 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 783. │ 782 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 784. │ 783 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 785. │ 784 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 786. │ 785 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 787. │ 786 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 788. │ 787 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 789. │ 788 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 790. │ 789 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 791. │ 790 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 792. │ 791 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 793. │ 792 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 794. │ 793 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 795. │ 794 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 796. │ 795 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 797. │ 796 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 798. │ 797 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 799. │ 798 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 800. │ 799 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 801. │ 800 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 802. │ 801 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 803. │ 802 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 804. │ 803 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 805. │ 804 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 806. │ 805 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 807. │ 806 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 808. │ 807 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 809. │ 808 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 810. │ 809 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 811. │ 810 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 812. │ 811 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 813. │ 812 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 814. │ 813 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 815. │ 814 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 816. │ 815 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 817. │ 816 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 818. │ 817 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 819. │ 818 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 820. │ 819 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 821. │ 820 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 822. │ 821 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 823. │ 822 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 824. │ 823 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 825. │ 824 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 826. │ 825 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 827. │ 826 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 828. │ 827 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 829. │ 828 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 830. │ 829 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 831. │ 830 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 832. │ 831 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 833. │ 832 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 834. │ 833 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 835. │ 834 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 836. │ 835 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 837. │ 836 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 838. │ 837 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 839. │ 838 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 840. │ 839 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 841. │ 840 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 842. │ 841 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 843. │ 842 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 844. │ 843 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 845. │ 844 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 846. │ 845 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 847. │ 846 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 848. │ 847 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 849. │ 848 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 850. │ 849 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 851. │ 850 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 852. │ 851 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 853. │ 852 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 854. │ 853 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 855. │ 854 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 856. │ 855 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 857. │ 856 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 858. │ 857 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 859. │ 858 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 860. │ 859 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 861. │ 860 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 862. │ 861 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 863. │ 862 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 864. │ 863 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 865. │ 864 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 866. │ 865 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 867. │ 866 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 868. │ 867 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 869. │ 868 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 870. │ 869 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 871. │ 870 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 872. │ 871 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 873. │ 872 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 874. │ 873 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 875. │ 874 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 876. │ 875 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 877. │ 876 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 878. │ 877 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 879. │ 878 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 880. │ 879 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 881. │ 880 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 882. │ 881 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 883. │ 882 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 884. │ 883 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 885. │ 884 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 886. │ 885 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 887. │ 886 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 888. │ 887 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 889. │ 888 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 890. │ 889 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 891. │ 890 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 892. │ 891 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 893. │ 892 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 894. │ 893 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 895. │ 894 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 896. │ 895 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 897. │ 896 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 898. │ 897 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 899. │ 898 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 900. │ 899 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 901. │ 900 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 902. │ 901 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 903. │ 902 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 904. │ 903 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 905. │ 904 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 906. │ 905 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 907. │ 906 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 908. │ 907 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 909. │ 908 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 910. │ 909 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 911. │ 910 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 912. │ 911 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 913. │ 912 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 914. │ 913 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 915. │ 914 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 916. │ 915 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 917. │ 916 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 918. │ 917 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 919. │ 918 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 920. │ 919 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 921. │ 920 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 922. │ 921 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 923. │ 922 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 924. │ 923 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 925. │ 924 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 926. │ 925 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 927. │ 926 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 928. │ 927 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 929. │ 928 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 930. │ 929 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 931. │ 930 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 932. │ 931 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 933. │ 932 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 934. │ 933 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 935. │ 934 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 936. │ 935 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 937. │ 936 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 938. │ 937 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 939. │ 938 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 940. │ 939 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 941. │ 940 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 942. │ 941 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 943. │ 942 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 944. │ 943 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 945. │ 944 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 946. │ 945 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 947. │ 946 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 948. │ 947 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 949. │ 948 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 950. │ 949 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 951. │ 950 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 952. │ 951 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 953. │ 952 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 954. │ 953 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 955. │ 954 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 956. │ 955 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 957. │ 956 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 958. │ 957 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 959. │ 958 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 960. │ 959 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 961. │ 960 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 962. │ 961 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 963. │ 962 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 964. │ 963 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 965. │ 964 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 966. │ 965 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 967. │ 966 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 968. │ 967 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 969. │ 968 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 970. │ 969 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 971. │ 970 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 972. │ 971 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 973. │ 972 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 974. │ 973 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 975. │ 974 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 976. │ 975 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 977. │ 976 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 978. │ 977 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 979. │ 978 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 980. │ 979 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 981. │ 980 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 982. │ 981 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 983. │ 982 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 984. │ 983 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 985. │ 984 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 986. │ 985 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 987. │ 986 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 988. │ 987 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 989. │ 988 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 990. │ 989 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 991. │ 990 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 992. │ 991 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 993. │ 992 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 994. │ 993 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 995. │ 994 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 996. │ 995 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 997. │ 996 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 998. │ 997 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 999. │ 998 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -1000. │ 999 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 100. │ 99 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 101. │ 100 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 102. │ 101 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 103. │ 102 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 104. │ 103 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 105. │ 104 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 106. │ 105 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 107. │ 106 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 108. │ 107 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 109. │ 108 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 110. │ 109 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 111. │ 110 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 112. │ 111 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 113. │ 112 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 114. │ 113 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 115. │ 114 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 116. │ 115 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 117. │ 116 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 118. │ 117 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 119. │ 118 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 120. │ 119 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 121. │ 120 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 122. │ 121 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 123. │ 122 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 124. │ 123 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 125. │ 124 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 126. │ 125 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 127. │ 126 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 128. │ 127 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 129. │ 128 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 130. │ 129 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 131. │ 130 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 132. │ 131 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 133. │ 132 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 134. │ 133 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 135. │ 134 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 136. │ 135 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 137. │ 136 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 138. │ 137 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 139. │ 138 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 140. │ 139 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 141. │ 140 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 142. │ 141 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 143. │ 142 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 144. │ 143 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 145. │ 144 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 146. │ 145 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 147. │ 146 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 148. │ 147 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 149. │ 148 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 150. │ 149 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 151. │ 150 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 152. │ 151 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 153. │ 152 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 154. │ 153 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 155. │ 154 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 156. │ 155 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 157. │ 156 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 158. │ 157 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 159. │ 158 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 160. │ 159 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 161. │ 160 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 162. │ 161 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 163. │ 162 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 164. │ 163 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 165. │ 164 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 166. │ 165 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 167. │ 166 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 168. │ 167 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 169. │ 168 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 170. │ 169 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 171. │ 170 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 172. │ 171 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 173. │ 172 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 174. │ 173 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 175. │ 174 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 176. │ 175 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 177. │ 176 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 178. │ 177 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 179. │ 178 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 180. │ 179 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 181. │ 180 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 182. │ 181 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 183. │ 182 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 184. │ 183 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 185. │ 184 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 186. │ 185 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 187. │ 186 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 188. │ 187 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 189. │ 188 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 190. │ 189 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 191. │ 190 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 192. │ 191 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 193. │ 192 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 194. │ 193 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 195. │ 194 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 196. │ 195 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 197. │ 196 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 198. │ 197 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 199. │ 198 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 200. │ 199 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 201. │ 200 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 202. │ 201 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 203. │ 202 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 204. │ 203 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 205. │ 204 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 206. │ 205 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 207. │ 206 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 208. │ 207 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 209. │ 208 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 210. │ 209 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 211. │ 210 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 212. │ 211 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 213. │ 212 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 214. │ 213 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 215. │ 214 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 216. │ 215 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 217. │ 216 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 218. │ 217 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 219. │ 218 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 220. │ 219 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 221. │ 220 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 222. │ 221 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 223. │ 222 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 224. │ 223 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 225. │ 224 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 226. │ 225 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 227. │ 226 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 228. │ 227 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 229. │ 228 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 230. │ 229 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 231. │ 230 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 232. │ 231 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 233. │ 232 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 234. │ 233 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 235. │ 234 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 236. │ 235 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 237. │ 236 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 238. │ 237 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 239. │ 238 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 240. │ 239 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 241. │ 240 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 242. │ 241 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 243. │ 242 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 244. │ 243 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 245. │ 244 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 246. │ 245 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 247. │ 246 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 248. │ 247 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 249. │ 248 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 250. │ 249 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 251. │ 250 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 252. │ 251 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 253. │ 252 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 254. │ 253 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 255. │ 254 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 256. │ 255 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 257. │ 256 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 258. │ 257 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 259. │ 258 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 260. │ 259 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 261. │ 260 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 262. │ 261 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 263. │ 262 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 264. │ 263 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 265. │ 264 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 266. │ 265 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 267. │ 266 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 268. │ 267 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 269. │ 268 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 270. │ 269 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 271. │ 270 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 272. │ 271 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 273. │ 272 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 274. │ 273 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 275. │ 274 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 276. │ 275 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 277. │ 276 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 278. │ 277 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 279. │ 278 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 280. │ 279 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 281. │ 280 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 282. │ 281 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 283. │ 282 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 284. │ 283 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 285. │ 284 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 286. │ 285 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 287. │ 286 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 288. │ 287 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 289. │ 288 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 290. │ 289 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 291. │ 290 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 292. │ 291 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 293. │ 292 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 294. │ 293 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 295. │ 294 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 296. │ 295 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 297. │ 296 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 298. │ 297 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 299. │ 298 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 300. │ 299 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 301. │ 300 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 302. │ 301 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 303. │ 302 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 304. │ 303 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 305. │ 304 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 306. │ 305 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 307. │ 306 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 308. │ 307 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 309. │ 308 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 310. │ 309 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 311. │ 310 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 312. │ 311 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 313. │ 312 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 314. │ 313 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 315. │ 314 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 316. │ 315 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 317. │ 316 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 318. │ 317 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 319. │ 318 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 320. │ 319 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 321. │ 320 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 322. │ 321 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 323. │ 322 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 324. │ 323 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 325. │ 324 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 326. │ 325 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 327. │ 326 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 328. │ 327 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 329. │ 328 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 330. │ 329 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 331. │ 330 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 332. │ 331 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 333. │ 332 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 334. │ 333 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 335. │ 334 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 336. │ 335 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 337. │ 336 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 338. │ 337 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 339. │ 338 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 340. │ 339 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 341. │ 340 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 342. │ 341 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 343. │ 342 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 344. │ 343 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 345. │ 344 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 346. │ 345 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 347. │ 346 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 348. │ 347 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 349. │ 348 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 350. │ 349 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 351. │ 350 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 352. │ 351 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 353. │ 352 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 354. │ 353 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 355. │ 354 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 356. │ 355 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 357. │ 356 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 358. │ 357 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 359. │ 358 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 360. │ 359 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 361. │ 360 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 362. │ 361 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 363. │ 362 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 364. │ 363 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 365. │ 364 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 366. │ 365 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 367. │ 366 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 368. │ 367 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 369. │ 368 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 370. │ 369 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 371. │ 370 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 372. │ 371 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 373. │ 372 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 374. │ 373 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 375. │ 374 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 376. │ 375 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 377. │ 376 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 378. │ 377 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 379. │ 378 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 380. │ 379 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 381. │ 380 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 382. │ 381 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 383. │ 382 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 384. │ 383 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 385. │ 384 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 386. │ 385 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 387. │ 386 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 388. │ 387 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 389. │ 388 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 390. │ 389 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 391. │ 390 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 392. │ 391 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 393. │ 392 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 394. │ 393 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 395. │ 394 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 396. │ 395 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 397. │ 396 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 398. │ 397 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 399. │ 398 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 400. │ 399 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 401. │ 400 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 402. │ 401 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 403. │ 402 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 404. │ 403 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 405. │ 404 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 406. │ 405 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 407. │ 406 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 408. │ 407 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 409. │ 408 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 410. │ 409 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 411. │ 410 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 412. │ 411 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 413. │ 412 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 414. │ 413 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 415. │ 414 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 416. │ 415 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 417. │ 416 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 418. │ 417 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 419. │ 418 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 420. │ 419 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 421. │ 420 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 422. │ 421 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 423. │ 422 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 424. │ 423 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 425. │ 424 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 426. │ 425 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 427. │ 426 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 428. │ 427 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 429. │ 428 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 430. │ 429 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 431. │ 430 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 432. │ 431 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 433. │ 432 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 434. │ 433 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 435. │ 434 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 436. │ 435 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 437. │ 436 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 438. │ 437 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 439. │ 438 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 440. │ 439 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 441. │ 440 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 442. │ 441 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 443. │ 442 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 444. │ 443 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 445. │ 444 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 446. │ 445 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 447. │ 446 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 448. │ 447 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 449. │ 448 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 450. │ 449 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 451. │ 450 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 452. │ 451 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 453. │ 452 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 454. │ 453 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 455. │ 454 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 456. │ 455 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 457. │ 456 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 458. │ 457 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 459. │ 458 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 460. │ 459 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 461. │ 460 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 462. │ 461 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 463. │ 462 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 464. │ 463 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 465. │ 464 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 466. │ 465 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 467. │ 466 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 468. │ 467 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 469. │ 468 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 470. │ 469 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 471. │ 470 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 472. │ 471 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 473. │ 472 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 474. │ 473 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 475. │ 474 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 476. │ 475 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 477. │ 476 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 478. │ 477 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 479. │ 478 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 480. │ 479 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 481. │ 480 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 482. │ 481 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 483. │ 482 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 484. │ 483 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 485. │ 484 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 486. │ 485 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 487. │ 486 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 488. │ 487 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 489. │ 488 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 490. │ 489 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 491. │ 490 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 492. │ 491 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 493. │ 492 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 494. │ 493 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 495. │ 494 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 496. │ 495 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 497. │ 496 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 498. │ 497 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 499. │ 498 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 500. │ 499 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 501. │ 500 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 502. │ 501 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 503. │ 502 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 504. │ 503 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 505. │ 504 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 506. │ 505 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 507. │ 506 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 508. │ 507 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 509. │ 508 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 510. │ 509 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 511. │ 510 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 512. │ 511 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 513. │ 512 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 514. │ 513 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 515. │ 514 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 516. │ 515 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 517. │ 516 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 518. │ 517 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 519. │ 518 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 520. │ 519 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 521. │ 520 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 522. │ 521 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 523. │ 522 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 524. │ 523 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 525. │ 524 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 526. │ 525 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 527. │ 526 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 528. │ 527 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 529. │ 528 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 530. │ 529 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 531. │ 530 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 532. │ 531 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 533. │ 532 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 534. │ 533 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 535. │ 534 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 536. │ 535 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 537. │ 536 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 538. │ 537 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 539. │ 538 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 540. │ 539 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 541. │ 540 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 542. │ 541 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 543. │ 542 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 544. │ 543 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 545. │ 544 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 546. │ 545 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 547. │ 546 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 548. │ 547 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 549. │ 548 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 550. │ 549 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 551. │ 550 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 552. │ 551 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 553. │ 552 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 554. │ 553 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 555. │ 554 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 556. │ 555 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 557. │ 556 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 558. │ 557 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 559. │ 558 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 560. │ 559 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 561. │ 560 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 562. │ 561 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 563. │ 562 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 564. │ 563 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 565. │ 564 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 566. │ 565 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 567. │ 566 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 568. │ 567 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 569. │ 568 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 570. │ 569 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 571. │ 570 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 572. │ 571 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 573. │ 572 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 574. │ 573 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 575. │ 574 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 576. │ 575 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 577. │ 576 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 578. │ 577 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 579. │ 578 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 580. │ 579 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 581. │ 580 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 582. │ 581 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 583. │ 582 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 584. │ 583 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 585. │ 584 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 586. │ 585 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 587. │ 586 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 588. │ 587 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 589. │ 588 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 590. │ 589 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 591. │ 590 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 592. │ 591 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 593. │ 592 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 594. │ 593 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 595. │ 594 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 596. │ 595 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 597. │ 596 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 598. │ 597 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 599. │ 598 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 600. │ 599 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 601. │ 600 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 602. │ 601 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 603. │ 602 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 604. │ 603 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 605. │ 604 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 606. │ 605 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 607. │ 606 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 608. │ 607 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 609. │ 608 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 610. │ 609 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 611. │ 610 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 612. │ 611 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 613. │ 612 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 614. │ 613 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 615. │ 614 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 616. │ 615 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 617. │ 616 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 618. │ 617 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 619. │ 618 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 620. │ 619 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 621. │ 620 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 622. │ 621 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 623. │ 622 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 624. │ 623 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 625. │ 624 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 626. │ 625 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 627. │ 626 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 628. │ 627 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 629. │ 628 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 630. │ 629 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 631. │ 630 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 632. │ 631 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 633. │ 632 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 634. │ 633 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 635. │ 634 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 636. │ 635 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 637. │ 636 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 638. │ 637 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 639. │ 638 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 640. │ 639 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 641. │ 640 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 642. │ 641 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 643. │ 642 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 644. │ 643 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 645. │ 644 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 646. │ 645 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 647. │ 646 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 648. │ 647 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 649. │ 648 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 650. │ 649 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 651. │ 650 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 652. │ 651 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 653. │ 652 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 654. │ 653 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 655. │ 654 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 656. │ 655 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 657. │ 656 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 658. │ 657 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 659. │ 658 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 660. │ 659 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 661. │ 660 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 662. │ 661 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 663. │ 662 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 664. │ 663 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 665. │ 664 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 666. │ 665 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 667. │ 666 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 668. │ 667 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 669. │ 668 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 670. │ 669 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 671. │ 670 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 672. │ 671 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 673. │ 672 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 674. │ 673 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 675. │ 674 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 676. │ 675 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 677. │ 676 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 678. │ 677 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 679. │ 678 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 680. │ 679 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 681. │ 680 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 682. │ 681 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 683. │ 682 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 684. │ 683 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 685. │ 684 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 686. │ 685 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 687. │ 686 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 688. │ 687 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 689. │ 688 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 690. │ 689 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 691. │ 690 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 692. │ 691 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 693. │ 692 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 694. │ 693 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 695. │ 694 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 696. │ 695 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 697. │ 696 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 698. │ 697 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 699. │ 698 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 700. │ 699 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 701. │ 700 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 702. │ 701 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 703. │ 702 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 704. │ 703 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 705. │ 704 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 706. │ 705 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 707. │ 706 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 708. │ 707 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 709. │ 708 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 710. │ 709 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 711. │ 710 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 712. │ 711 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 713. │ 712 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 714. │ 713 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 715. │ 714 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 716. │ 715 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 717. │ 716 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 718. │ 717 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 719. │ 718 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 720. │ 719 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 721. │ 720 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 722. │ 721 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 723. │ 722 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 724. │ 723 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 725. │ 724 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 726. │ 725 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 727. │ 726 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 728. │ 727 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 729. │ 728 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 730. │ 729 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 731. │ 730 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 732. │ 731 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 733. │ 732 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 734. │ 733 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 735. │ 734 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 736. │ 735 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 737. │ 736 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 738. │ 737 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 739. │ 738 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 740. │ 739 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 741. │ 740 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 742. │ 741 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 743. │ 742 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 744. │ 743 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 745. │ 744 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 746. │ 745 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 747. │ 746 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 748. │ 747 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 749. │ 748 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 750. │ 749 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 751. │ 750 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 752. │ 751 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 753. │ 752 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 754. │ 753 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 755. │ 754 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 756. │ 755 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 757. │ 756 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 758. │ 757 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 759. │ 758 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 760. │ 759 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 761. │ 760 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 762. │ 761 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 763. │ 762 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 764. │ 763 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 765. │ 764 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 766. │ 765 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 767. │ 766 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 768. │ 767 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 769. │ 768 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 770. │ 769 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 771. │ 770 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 772. │ 771 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 773. │ 772 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 774. │ 773 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 775. │ 774 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 776. │ 775 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 777. │ 776 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 778. │ 777 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 779. │ 778 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 780. │ 779 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 781. │ 780 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 782. │ 781 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 783. │ 782 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 784. │ 783 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 785. │ 784 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 786. │ 785 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 787. │ 786 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 788. │ 787 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 789. │ 788 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 790. │ 789 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 791. │ 790 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 792. │ 791 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 793. │ 792 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 794. │ 793 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 795. │ 794 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 796. │ 795 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 797. │ 796 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 798. │ 797 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 799. │ 798 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 800. │ 799 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 801. │ 800 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 802. │ 801 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 803. │ 802 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 804. │ 803 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 805. │ 804 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 806. │ 805 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 807. │ 806 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 808. │ 807 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 809. │ 808 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 810. │ 809 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 811. │ 810 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 812. │ 811 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 813. │ 812 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 814. │ 813 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 815. │ 814 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 816. │ 815 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 817. │ 816 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 818. │ 817 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 819. │ 818 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 820. │ 819 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 821. │ 820 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 822. │ 821 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 823. │ 822 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 824. │ 823 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 825. │ 824 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 826. │ 825 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 827. │ 826 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 828. │ 827 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 829. │ 828 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 830. │ 829 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 831. │ 830 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 832. │ 831 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 833. │ 832 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 834. │ 833 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 835. │ 834 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 836. │ 835 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 837. │ 836 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 838. │ 837 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 839. │ 838 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 840. │ 839 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 841. │ 840 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 842. │ 841 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 843. │ 842 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 844. │ 843 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 845. │ 844 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 846. │ 845 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 847. │ 846 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 848. │ 847 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 849. │ 848 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 850. │ 849 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 851. │ 850 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 852. │ 851 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 853. │ 852 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 854. │ 853 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 855. │ 854 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 856. │ 855 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 857. │ 856 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 858. │ 857 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 859. │ 858 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 860. │ 859 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 861. │ 860 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 862. │ 861 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 863. │ 862 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 864. │ 863 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 865. │ 864 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 866. │ 865 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 867. │ 866 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 868. │ 867 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 869. │ 868 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 870. │ 869 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 871. │ 870 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 872. │ 871 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 873. │ 872 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 874. │ 873 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 875. │ 874 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 876. │ 875 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 877. │ 876 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 878. │ 877 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 879. │ 878 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 880. │ 879 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 881. │ 880 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 882. │ 881 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 883. │ 882 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 884. │ 883 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 885. │ 884 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 886. │ 885 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 887. │ 886 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 888. │ 887 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 889. │ 888 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 890. │ 889 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 891. │ 890 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 892. │ 891 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 893. │ 892 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 894. │ 893 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 895. │ 894 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 896. │ 895 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 897. │ 896 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 898. │ 897 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 899. │ 898 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 900. │ 899 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 901. │ 900 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 902. │ 901 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 903. │ 902 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 904. │ 903 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 905. │ 904 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 906. │ 905 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 907. │ 906 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 908. │ 907 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 909. │ 908 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 910. │ 909 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 911. │ 910 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 912. │ 911 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 913. │ 912 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 914. │ 913 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 915. │ 914 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 916. │ 915 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 917. │ 916 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 918. │ 917 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 919. │ 918 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 920. │ 919 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 921. │ 920 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 922. │ 921 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 923. │ 922 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 924. │ 923 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 925. │ 924 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 926. │ 925 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 927. │ 926 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 928. │ 927 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 929. │ 928 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 930. │ 929 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 931. │ 930 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 932. │ 931 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 933. │ 932 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 934. │ 933 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 935. │ 934 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 936. │ 935 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 937. │ 936 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 938. │ 937 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 939. │ 938 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 940. │ 939 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 941. │ 940 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 942. │ 941 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 943. │ 942 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 944. │ 943 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 945. │ 944 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 946. │ 945 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 947. │ 946 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 948. │ 947 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 949. │ 948 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 950. │ 949 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 951. │ 950 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 952. │ 951 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 953. │ 952 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 954. │ 953 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 955. │ 954 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 956. │ 955 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 957. │ 956 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 958. │ 957 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 959. │ 958 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 960. │ 959 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 961. │ 960 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 962. │ 961 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 963. │ 962 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 964. │ 963 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 965. │ 964 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 966. │ 965 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 967. │ 966 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 968. │ 967 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 969. │ 968 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 970. │ 969 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 971. │ 970 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 972. │ 971 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 973. │ 972 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 974. │ 973 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 975. │ 974 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 976. │ 975 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 977. │ 976 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 978. │ 977 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 979. │ 978 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 980. │ 979 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 981. │ 980 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 982. │ 981 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 983. │ 982 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 984. │ 983 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 985. │ 984 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 986. │ 985 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 987. │ 986 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 988. │ 987 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 989. │ 988 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 990. │ 989 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 991. │ 990 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 992. │ 991 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 993. │ 992 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 994. │ 993 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 995. │ 994 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 996. │ 995 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 997. │ 996 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 998. │ 997 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 999. │ 998 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -1000. │ 999 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 100. │ 99 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 101. │ 100 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 102. │ 101 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 103. │ 102 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 104. │ 103 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 105. │ 104 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 106. │ 105 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 107. │ 106 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 108. │ 107 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 109. │ 108 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 110. │ 109 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 111. │ 110 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 112. │ 111 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 113. │ 112 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 114. │ 113 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 115. │ 114 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 116. │ 115 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 117. │ 116 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 118. │ 117 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 119. │ 118 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 120. │ 119 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 121. │ 120 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 122. │ 121 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 123. │ 122 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 124. │ 123 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 125. │ 124 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 126. │ 125 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 127. │ 126 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 128. │ 127 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 129. │ 128 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 130. │ 129 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 131. │ 130 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 132. │ 131 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 133. │ 132 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 134. │ 133 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 135. │ 134 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 136. │ 135 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 137. │ 136 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 138. │ 137 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 139. │ 138 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 140. │ 139 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 141. │ 140 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 142. │ 141 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 143. │ 142 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 144. │ 143 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 145. │ 144 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 146. │ 145 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 147. │ 146 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 148. │ 147 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 149. │ 148 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 150. │ 149 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 151. │ 150 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 152. │ 151 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 153. │ 152 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 154. │ 153 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 155. │ 154 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 156. │ 155 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 157. │ 156 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 158. │ 157 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 159. │ 158 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 160. │ 159 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 161. │ 160 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 162. │ 161 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 163. │ 162 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 164. │ 163 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 165. │ 164 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 166. │ 165 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 167. │ 166 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 168. │ 167 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 169. │ 168 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 170. │ 169 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 171. │ 170 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 172. │ 171 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 173. │ 172 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 174. │ 173 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 175. │ 174 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 176. │ 175 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 177. │ 176 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 178. │ 177 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 179. │ 178 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 180. │ 179 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 181. │ 180 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 182. │ 181 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 183. │ 182 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 184. │ 183 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 185. │ 184 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 186. │ 185 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 187. │ 186 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 188. │ 187 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 189. │ 188 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 190. │ 189 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 191. │ 190 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 192. │ 191 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 193. │ 192 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 194. │ 193 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 195. │ 194 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 196. │ 195 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 197. │ 196 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 198. │ 197 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 199. │ 198 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 200. │ 199 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 201. │ 200 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 202. │ 201 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 203. │ 202 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 204. │ 203 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 205. │ 204 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 206. │ 205 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 207. │ 206 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 208. │ 207 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 209. │ 208 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 210. │ 209 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 211. │ 210 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 212. │ 211 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 213. │ 212 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 214. │ 213 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 215. │ 214 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 216. │ 215 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 217. │ 216 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 218. │ 217 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 219. │ 218 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 220. │ 219 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 221. │ 220 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 222. │ 221 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 223. │ 222 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 224. │ 223 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 225. │ 224 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 226. │ 225 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 227. │ 226 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 228. │ 227 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 229. │ 228 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 230. │ 229 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 231. │ 230 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 232. │ 231 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 233. │ 232 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 234. │ 233 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 235. │ 234 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 236. │ 235 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 237. │ 236 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 238. │ 237 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 239. │ 238 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 240. │ 239 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 241. │ 240 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 242. │ 241 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 243. │ 242 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 244. │ 243 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 245. │ 244 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 246. │ 245 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 247. │ 246 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 248. │ 247 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 249. │ 248 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 250. │ 249 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 251. │ 250 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 252. │ 251 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 253. │ 252 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 254. │ 253 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 255. │ 254 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 256. │ 255 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 257. │ 256 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 258. │ 257 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 259. │ 258 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 260. │ 259 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 261. │ 260 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 262. │ 261 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 263. │ 262 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 264. │ 263 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 265. │ 264 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 266. │ 265 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 267. │ 266 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 268. │ 267 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 269. │ 268 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 270. │ 269 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 271. │ 270 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 272. │ 271 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 273. │ 272 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 274. │ 273 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 275. │ 274 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 276. │ 275 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 277. │ 276 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 278. │ 277 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 279. │ 278 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 280. │ 279 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 281. │ 280 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 282. │ 281 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 283. │ 282 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 284. │ 283 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 285. │ 284 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 286. │ 285 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 287. │ 286 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 288. │ 287 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 289. │ 288 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 290. │ 289 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 291. │ 290 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 292. │ 291 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 293. │ 292 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 294. │ 293 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 295. │ 294 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 296. │ 295 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 297. │ 296 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 298. │ 297 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 299. │ 298 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 300. │ 299 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 301. │ 300 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 302. │ 301 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 303. │ 302 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 304. │ 303 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 305. │ 304 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 306. │ 305 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 307. │ 306 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 308. │ 307 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 309. │ 308 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 310. │ 309 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 311. │ 310 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 312. │ 311 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 313. │ 312 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 314. │ 313 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 315. │ 314 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 316. │ 315 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 317. │ 316 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 318. │ 317 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 319. │ 318 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 320. │ 319 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 321. │ 320 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 322. │ 321 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 323. │ 322 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 324. │ 323 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 325. │ 324 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 326. │ 325 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 327. │ 326 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 328. │ 327 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 329. │ 328 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 330. │ 329 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 331. │ 330 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 332. │ 331 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 333. │ 332 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 334. │ 333 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 335. │ 334 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 336. │ 335 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 337. │ 336 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 338. │ 337 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 339. │ 338 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 340. │ 339 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 341. │ 340 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 342. │ 341 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 343. │ 342 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 344. │ 343 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 345. │ 344 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 346. │ 345 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 347. │ 346 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 348. │ 347 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 349. │ 348 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 350. │ 349 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 351. │ 350 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 352. │ 351 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 353. │ 352 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 354. │ 353 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 355. │ 354 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 356. │ 355 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 357. │ 356 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 358. │ 357 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 359. │ 358 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 360. │ 359 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 361. │ 360 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 362. │ 361 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 363. │ 362 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 364. │ 363 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 365. │ 364 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 366. │ 365 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 367. │ 366 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 368. │ 367 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 369. │ 368 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 370. │ 369 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 371. │ 370 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 372. │ 371 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 373. │ 372 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 374. │ 373 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 375. │ 374 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 376. │ 375 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 377. │ 376 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 378. │ 377 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 379. │ 378 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 380. │ 379 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 381. │ 380 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 382. │ 381 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 383. │ 382 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 384. │ 383 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 385. │ 384 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 386. │ 385 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 387. │ 386 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 388. │ 387 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 389. │ 388 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 390. │ 389 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 391. │ 390 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 392. │ 391 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 393. │ 392 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 394. │ 393 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 395. │ 394 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 396. │ 395 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 397. │ 396 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 398. │ 397 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 399. │ 398 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 400. │ 399 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 401. │ 400 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 402. │ 401 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 403. │ 402 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 404. │ 403 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 405. │ 404 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 406. │ 405 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 407. │ 406 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 408. │ 407 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 409. │ 408 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 410. │ 409 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 411. │ 410 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 412. │ 411 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 413. │ 412 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 414. │ 413 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 415. │ 414 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 416. │ 415 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 417. │ 416 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 418. │ 417 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 419. │ 418 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 420. │ 419 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 421. │ 420 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 422. │ 421 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 423. │ 422 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 424. │ 423 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 425. │ 424 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 426. │ 425 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 427. │ 426 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 428. │ 427 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 429. │ 428 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 430. │ 429 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 431. │ 430 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 432. │ 431 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 433. │ 432 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 434. │ 433 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 435. │ 434 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 436. │ 435 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 437. │ 436 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 438. │ 437 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 439. │ 438 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 440. │ 439 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 441. │ 440 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 442. │ 441 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 443. │ 442 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 444. │ 443 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 445. │ 444 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 446. │ 445 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 447. │ 446 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 448. │ 447 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 449. │ 448 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 450. │ 449 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 451. │ 450 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 452. │ 451 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 453. │ 452 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 454. │ 453 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 455. │ 454 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 456. │ 455 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 457. │ 456 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 458. │ 457 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 459. │ 458 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 460. │ 459 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 461. │ 460 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 462. │ 461 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 463. │ 462 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 464. │ 463 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 465. │ 464 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 466. │ 465 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 467. │ 466 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 468. │ 467 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 469. │ 468 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 470. │ 469 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 471. │ 470 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 472. │ 471 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 473. │ 472 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 474. │ 473 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 475. │ 474 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 476. │ 475 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 477. │ 476 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 478. │ 477 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 479. │ 478 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 480. │ 479 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 481. │ 480 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 482. │ 481 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 483. │ 482 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 484. │ 483 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 485. │ 484 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 486. │ 485 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 487. │ 486 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 488. │ 487 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 489. │ 488 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 490. │ 489 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 491. │ 490 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 492. │ 491 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 493. │ 492 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 494. │ 493 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 495. │ 494 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 496. │ 495 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 497. │ 496 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 498. │ 497 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 499. │ 498 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 500. │ 499 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 501. │ 500 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 502. │ 501 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 503. │ 502 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 504. │ 503 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 505. │ 504 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 506. │ 505 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 507. │ 506 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 508. │ 507 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 509. │ 508 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 510. │ 509 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 511. │ 510 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 512. │ 511 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 513. │ 512 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 514. │ 513 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 515. │ 514 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 516. │ 515 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 517. │ 516 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 518. │ 517 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 519. │ 518 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 520. │ 519 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 521. │ 520 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 522. │ 521 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 523. │ 522 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 524. │ 523 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 525. │ 524 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 526. │ 525 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 527. │ 526 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 528. │ 527 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 529. │ 528 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 530. │ 529 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 531. │ 530 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 532. │ 531 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 533. │ 532 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 534. │ 533 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 535. │ 534 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 536. │ 535 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 537. │ 536 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 538. │ 537 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 539. │ 538 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 540. │ 539 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 541. │ 540 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 542. │ 541 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 543. │ 542 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 544. │ 543 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 545. │ 544 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 546. │ 545 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 547. │ 546 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 548. │ 547 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 549. │ 548 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 550. │ 549 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 551. │ 550 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 552. │ 551 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 553. │ 552 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 554. │ 553 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 555. │ 554 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 556. │ 555 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 557. │ 556 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 558. │ 557 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 559. │ 558 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 560. │ 559 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 561. │ 560 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 562. │ 561 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 563. │ 562 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 564. │ 563 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 565. │ 564 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 566. │ 565 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 567. │ 566 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 568. │ 567 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 569. │ 568 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 570. │ 569 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 571. │ 570 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 572. │ 571 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 573. │ 572 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 574. │ 573 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 575. │ 574 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 576. │ 575 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 577. │ 576 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 578. │ 577 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 579. │ 578 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 580. │ 579 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 581. │ 580 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 582. │ 581 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 583. │ 582 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 584. │ 583 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 585. │ 584 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 586. │ 585 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 587. │ 586 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 588. │ 587 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 589. │ 588 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 590. │ 589 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 591. │ 590 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 592. │ 591 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 593. │ 592 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 594. │ 593 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 595. │ 594 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 596. │ 595 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 597. │ 596 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 598. │ 597 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 599. │ 598 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 600. │ 599 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 601. │ 600 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 602. │ 601 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 603. │ 602 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 604. │ 603 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 605. │ 604 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 606. │ 605 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 607. │ 606 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 608. │ 607 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 609. │ 608 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 610. │ 609 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 611. │ 610 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 612. │ 611 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 613. │ 612 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 614. │ 613 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 615. │ 614 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 616. │ 615 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 617. │ 616 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 618. │ 617 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 619. │ 618 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 620. │ 619 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 621. │ 620 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 622. │ 621 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 623. │ 622 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 624. │ 623 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 625. │ 624 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 626. │ 625 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 627. │ 626 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 628. │ 627 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 629. │ 628 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 630. │ 629 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 631. │ 630 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 632. │ 631 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 633. │ 632 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 634. │ 633 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 635. │ 634 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 636. │ 635 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 637. │ 636 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 638. │ 637 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 639. │ 638 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 640. │ 639 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 641. │ 640 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 642. │ 641 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 643. │ 642 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 644. │ 643 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 645. │ 644 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 646. │ 645 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 647. │ 646 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 648. │ 647 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 649. │ 648 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 650. │ 649 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 651. │ 650 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 652. │ 651 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 653. │ 652 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 654. │ 653 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 655. │ 654 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 656. │ 655 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 657. │ 656 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 658. │ 657 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 659. │ 658 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 660. │ 659 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 661. │ 660 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 662. │ 661 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 663. │ 662 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 664. │ 663 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 665. │ 664 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 666. │ 665 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 667. │ 666 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 668. │ 667 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 669. │ 668 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 670. │ 669 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 671. │ 670 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 672. │ 671 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 673. │ 672 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 674. │ 673 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 675. │ 674 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 676. │ 675 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 677. │ 676 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 678. │ 677 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 679. │ 678 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 680. │ 679 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 681. │ 680 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 682. │ 681 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 683. │ 682 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 684. │ 683 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 685. │ 684 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 686. │ 685 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 687. │ 686 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 688. │ 687 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 689. │ 688 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 690. │ 689 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 691. │ 690 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 692. │ 691 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 693. │ 692 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 694. │ 693 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 695. │ 694 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 696. │ 695 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 697. │ 696 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 698. │ 697 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 699. │ 698 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 700. │ 699 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 701. │ 700 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 702. │ 701 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 703. │ 702 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 704. │ 703 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 705. │ 704 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 706. │ 705 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 707. │ 706 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 708. │ 707 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 709. │ 708 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 710. │ 709 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 711. │ 710 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 712. │ 711 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 713. │ 712 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 714. │ 713 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 715. │ 714 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 716. │ 715 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 717. │ 716 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 718. │ 717 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 719. │ 718 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 720. │ 719 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 721. │ 720 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 722. │ 721 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 723. │ 722 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 724. │ 723 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 725. │ 724 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 726. │ 725 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 727. │ 726 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 728. │ 727 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 729. │ 728 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 730. │ 729 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 731. │ 730 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 732. │ 731 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 733. │ 732 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 734. │ 733 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 735. │ 734 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 736. │ 735 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 737. │ 736 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 738. │ 737 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 739. │ 738 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 740. │ 739 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 741. │ 740 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 742. │ 741 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 743. │ 742 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 744. │ 743 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 745. │ 744 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 746. │ 745 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 747. │ 746 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 748. │ 747 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 749. │ 748 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 750. │ 749 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 751. │ 750 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 752. │ 751 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 753. │ 752 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 754. │ 753 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 755. │ 754 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 756. │ 755 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 757. │ 756 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 758. │ 757 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 759. │ 758 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 760. │ 759 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 761. │ 760 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 762. │ 761 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 763. │ 762 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 764. │ 763 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 765. │ 764 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 766. │ 765 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 767. │ 766 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 768. │ 767 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 769. │ 768 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 770. │ 769 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 771. │ 770 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 772. │ 771 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 773. │ 772 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 774. │ 773 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 775. │ 774 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 776. │ 775 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 777. │ 776 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 778. │ 777 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 779. │ 778 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 780. │ 779 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 781. │ 780 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 782. │ 781 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 783. │ 782 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 784. │ 783 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 785. │ 784 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 786. │ 785 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 787. │ 786 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 788. │ 787 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 789. │ 788 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 790. │ 789 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 791. │ 790 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 792. │ 791 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 793. │ 792 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 794. │ 793 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 795. │ 794 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 796. │ 795 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 797. │ 796 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 798. │ 797 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 799. │ 798 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 800. │ 799 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 801. │ 800 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 802. │ 801 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 803. │ 802 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 804. │ 803 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 805. │ 804 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 806. │ 805 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 807. │ 806 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 808. │ 807 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 809. │ 808 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 810. │ 809 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 811. │ 810 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 812. │ 811 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 813. │ 812 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 814. │ 813 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 815. │ 814 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 816. │ 815 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 817. │ 816 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 818. │ 817 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 819. │ 818 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 820. │ 819 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 821. │ 820 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 822. │ 821 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 823. │ 822 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 824. │ 823 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 825. │ 824 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 826. │ 825 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 827. │ 826 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 828. │ 827 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 829. │ 828 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 830. │ 829 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 831. │ 830 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 832. │ 831 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 833. │ 832 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 834. │ 833 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 835. │ 834 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 836. │ 835 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 837. │ 836 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 838. │ 837 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 839. │ 838 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 840. │ 839 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 841. │ 840 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 842. │ 841 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 843. │ 842 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 844. │ 843 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 845. │ 844 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 846. │ 845 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 847. │ 846 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 848. │ 847 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 849. │ 848 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 850. │ 849 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 851. │ 850 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 852. │ 851 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 853. │ 852 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 854. │ 853 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 855. │ 854 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 856. │ 855 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 857. │ 856 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 858. │ 857 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 859. │ 858 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 860. │ 859 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 861. │ 860 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 862. │ 861 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 863. │ 862 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 864. │ 863 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 865. │ 864 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 866. │ 865 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 867. │ 866 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 868. │ 867 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 869. │ 868 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 870. │ 869 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 871. │ 870 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 872. │ 871 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 873. │ 872 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 874. │ 873 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 875. │ 874 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 876. │ 875 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 877. │ 876 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 878. │ 877 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 879. │ 878 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 880. │ 879 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 881. │ 880 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 882. │ 881 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 883. │ 882 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 884. │ 883 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 885. │ 884 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 886. │ 885 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 887. │ 886 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 888. │ 887 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 889. │ 888 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 890. │ 889 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 891. │ 890 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 892. │ 891 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 893. │ 892 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 894. │ 893 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 895. │ 894 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 896. │ 895 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 897. │ 896 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 898. │ 897 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 899. │ 898 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 900. │ 899 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 901. │ 900 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 902. │ 901 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 903. │ 902 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 904. │ 903 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 905. │ 904 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 906. │ 905 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 907. │ 906 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 908. │ 907 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 909. │ 908 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 910. │ 909 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 911. │ 910 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 912. │ 911 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 913. │ 912 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 914. │ 913 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 915. │ 914 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 916. │ 915 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 917. │ 916 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 918. │ 917 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 919. │ 918 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 920. │ 919 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 921. │ 920 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 922. │ 921 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 923. │ 922 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 924. │ 923 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 925. │ 924 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 926. │ 925 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 927. │ 926 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 928. │ 927 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 929. │ 928 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 930. │ 929 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 931. │ 930 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 932. │ 931 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 933. │ 932 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 934. │ 933 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 935. │ 934 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 936. │ 935 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 937. │ 936 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 938. │ 937 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 939. │ 938 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 940. │ 939 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 941. │ 940 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 942. │ 941 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 943. │ 942 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 944. │ 943 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 945. │ 944 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 946. │ 945 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 947. │ 946 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 948. │ 947 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 949. │ 948 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 950. │ 949 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 951. │ 950 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 952. │ 951 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 953. │ 952 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 954. │ 953 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 955. │ 954 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 956. │ 955 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 957. │ 956 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 958. │ 957 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 959. │ 958 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 960. │ 959 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 961. │ 960 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 962. │ 961 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 963. │ 962 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 964. │ 963 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 965. │ 964 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 966. │ 965 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 967. │ 966 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 968. │ 967 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 969. │ 968 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 970. │ 969 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 971. │ 970 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 972. │ 971 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 973. │ 972 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 974. │ 973 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 975. │ 974 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 976. │ 975 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 977. │ 976 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 978. │ 977 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 979. │ 978 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 980. │ 979 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 981. │ 980 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 982. │ 981 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 983. │ 982 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 984. │ 983 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 985. │ 984 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 986. │ 985 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 987. │ 986 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 988. │ 987 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 989. │ 988 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 990. │ 989 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 991. │ 990 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 992. │ 991 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 993. │ 992 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 994. │ 993 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 995. │ 994 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 996. │ 995 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 997. │ 996 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 998. │ 997 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 999. │ 998 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -1000. │ 999 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ - 100. │ 99 │ UInt64 │ 1 │ - 101. │ 100 │ UInt64 │ 0 │ - 102. │ 101 │ UInt64 │ 1 │ - 103. │ 102 │ UInt64 │ 0 │ - 104. │ 103 │ UInt64 │ 1 │ - 105. │ 104 │ UInt64 │ 0 │ - 106. │ 105 │ UInt64 │ 1 │ - 107. │ 106 │ UInt64 │ 0 │ - 108. │ 107 │ UInt64 │ 1 │ - 109. │ 108 │ UInt64 │ 0 │ - 110. │ 109 │ UInt64 │ 1 │ - 111. │ 110 │ UInt64 │ 0 │ - 112. │ 111 │ UInt64 │ 1 │ - 113. │ 112 │ UInt64 │ 0 │ - 114. │ 113 │ UInt64 │ 1 │ - 115. │ 114 │ UInt64 │ 0 │ - 116. │ 115 │ UInt64 │ 1 │ - 117. │ 116 │ UInt64 │ 0 │ - 118. │ 117 │ UInt64 │ 1 │ - 119. │ 118 │ UInt64 │ 0 │ - 120. │ 119 │ UInt64 │ 1 │ - 121. │ 120 │ UInt64 │ 0 │ - 122. │ 121 │ UInt64 │ 1 │ - 123. │ 122 │ UInt64 │ 0 │ - 124. │ 123 │ UInt64 │ 1 │ - 125. │ 124 │ UInt64 │ 0 │ - 126. │ 125 │ UInt64 │ 1 │ - 127. │ 126 │ UInt64 │ 0 │ - 128. │ 127 │ UInt64 │ 1 │ - 129. │ 128 │ UInt64 │ 0 │ - 130. │ 129 │ UInt64 │ 1 │ - 131. │ 130 │ UInt64 │ 0 │ - 132. │ 131 │ UInt64 │ 1 │ - 133. │ 132 │ UInt64 │ 0 │ - 134. │ 133 │ UInt64 │ 1 │ - 135. │ 134 │ UInt64 │ 0 │ - 136. │ 135 │ UInt64 │ 1 │ - 137. │ 136 │ UInt64 │ 0 │ - 138. │ 137 │ UInt64 │ 1 │ - 139. │ 138 │ UInt64 │ 0 │ - 140. │ 139 │ UInt64 │ 1 │ - 141. │ 140 │ UInt64 │ 0 │ - 142. │ 141 │ UInt64 │ 1 │ - 143. │ 142 │ UInt64 │ 0 │ - 144. │ 143 │ UInt64 │ 1 │ - 145. │ 144 │ UInt64 │ 0 │ - 146. │ 145 │ UInt64 │ 1 │ - 147. │ 146 │ UInt64 │ 0 │ - 148. │ 147 │ UInt64 │ 1 │ - 149. │ 148 │ UInt64 │ 0 │ - 150. │ 149 │ UInt64 │ 1 │ - 151. │ 150 │ UInt64 │ 0 │ - 152. │ 151 │ UInt64 │ 1 │ - 153. │ 152 │ UInt64 │ 0 │ - 154. │ 153 │ UInt64 │ 1 │ - 155. │ 154 │ UInt64 │ 0 │ - 156. │ 155 │ UInt64 │ 1 │ - 157. │ 156 │ UInt64 │ 0 │ - 158. │ 157 │ UInt64 │ 1 │ - 159. │ 158 │ UInt64 │ 0 │ - 160. │ 159 │ UInt64 │ 1 │ - 161. │ 160 │ UInt64 │ 0 │ - 162. │ 161 │ UInt64 │ 1 │ - 163. │ 162 │ UInt64 │ 0 │ - 164. │ 163 │ UInt64 │ 1 │ - 165. │ 164 │ UInt64 │ 0 │ - 166. │ 165 │ UInt64 │ 1 │ - 167. │ 166 │ UInt64 │ 0 │ - 168. │ 167 │ UInt64 │ 1 │ - 169. │ 168 │ UInt64 │ 0 │ - 170. │ 169 │ UInt64 │ 1 │ - 171. │ 170 │ UInt64 │ 0 │ - 172. │ 171 │ UInt64 │ 1 │ - 173. │ 172 │ UInt64 │ 0 │ - 174. │ 173 │ UInt64 │ 1 │ - 175. │ 174 │ UInt64 │ 0 │ - 176. │ 175 │ UInt64 │ 1 │ - 177. │ 176 │ UInt64 │ 0 │ - 178. │ 177 │ UInt64 │ 1 │ - 179. │ 178 │ UInt64 │ 0 │ - 180. │ 179 │ UInt64 │ 1 │ - 181. │ 180 │ UInt64 │ 0 │ - 182. │ 181 │ UInt64 │ 1 │ - 183. │ 182 │ UInt64 │ 0 │ - 184. │ 183 │ UInt64 │ 1 │ - 185. │ 184 │ UInt64 │ 0 │ - 186. │ 185 │ UInt64 │ 1 │ - 187. │ 186 │ UInt64 │ 0 │ - 188. │ 187 │ UInt64 │ 1 │ - 189. │ 188 │ UInt64 │ 0 │ - 190. │ 189 │ UInt64 │ 1 │ - 191. │ 190 │ UInt64 │ 0 │ - 192. │ 191 │ UInt64 │ 1 │ - 193. │ 192 │ UInt64 │ 0 │ - 194. │ 193 │ UInt64 │ 1 │ - 195. │ 194 │ UInt64 │ 0 │ - 196. │ 195 │ UInt64 │ 1 │ - 197. │ 196 │ UInt64 │ 0 │ - 198. │ 197 │ UInt64 │ 1 │ - 199. │ 198 │ UInt64 │ 0 │ - 200. │ 199 │ UInt64 │ 1 │ - 201. │ 200 │ UInt64 │ 0 │ - 202. │ 201 │ UInt64 │ 1 │ - 203. │ 202 │ UInt64 │ 0 │ - 204. │ 203 │ UInt64 │ 1 │ - 205. │ 204 │ UInt64 │ 0 │ - 206. │ 205 │ UInt64 │ 1 │ - 207. │ 206 │ UInt64 │ 0 │ - 208. │ 207 │ UInt64 │ 1 │ - 209. │ 208 │ UInt64 │ 0 │ - 210. │ 209 │ UInt64 │ 1 │ - 211. │ 210 │ UInt64 │ 0 │ - 212. │ 211 │ UInt64 │ 1 │ - 213. │ 212 │ UInt64 │ 0 │ - 214. │ 213 │ UInt64 │ 1 │ - 215. │ 214 │ UInt64 │ 0 │ - 216. │ 215 │ UInt64 │ 1 │ - 217. │ 216 │ UInt64 │ 0 │ - 218. │ 217 │ UInt64 │ 1 │ - 219. │ 218 │ UInt64 │ 0 │ - 220. │ 219 │ UInt64 │ 1 │ - 221. │ 220 │ UInt64 │ 0 │ - 222. │ 221 │ UInt64 │ 1 │ - 223. │ 222 │ UInt64 │ 0 │ - 224. │ 223 │ UInt64 │ 1 │ - 225. │ 224 │ UInt64 │ 0 │ - 226. │ 225 │ UInt64 │ 1 │ - 227. │ 226 │ UInt64 │ 0 │ - 228. │ 227 │ UInt64 │ 1 │ - 229. │ 228 │ UInt64 │ 0 │ - 230. │ 229 │ UInt64 │ 1 │ - 231. │ 230 │ UInt64 │ 0 │ - 232. │ 231 │ UInt64 │ 1 │ - 233. │ 232 │ UInt64 │ 0 │ - 234. │ 233 │ UInt64 │ 1 │ - 235. │ 234 │ UInt64 │ 0 │ - 236. │ 235 │ UInt64 │ 1 │ - 237. │ 236 │ UInt64 │ 0 │ - 238. │ 237 │ UInt64 │ 1 │ - 239. │ 238 │ UInt64 │ 0 │ - 240. │ 239 │ UInt64 │ 1 │ - 241. │ 240 │ UInt64 │ 0 │ - 242. │ 241 │ UInt64 │ 1 │ - 243. │ 242 │ UInt64 │ 0 │ - 244. │ 243 │ UInt64 │ 1 │ - 245. │ 244 │ UInt64 │ 0 │ - 246. │ 245 │ UInt64 │ 1 │ - 247. │ 246 │ UInt64 │ 0 │ - 248. │ 247 │ UInt64 │ 1 │ - 249. │ 248 │ UInt64 │ 0 │ - 250. │ 249 │ UInt64 │ 1 │ - 251. │ 250 │ UInt64 │ 0 │ - 252. │ 251 │ UInt64 │ 1 │ - 253. │ 252 │ UInt64 │ 0 │ - 254. │ 253 │ UInt64 │ 1 │ - 255. │ 254 │ UInt64 │ 0 │ - 256. │ 255 │ UInt64 │ 1 │ - 257. │ 256 │ UInt64 │ 0 │ - 258. │ 257 │ UInt64 │ 1 │ - 259. │ 258 │ UInt64 │ 0 │ - 260. │ 259 │ UInt64 │ 1 │ - 261. │ 260 │ UInt64 │ 0 │ - 262. │ 261 │ UInt64 │ 1 │ - 263. │ 262 │ UInt64 │ 0 │ - 264. │ 263 │ UInt64 │ 1 │ - 265. │ 264 │ UInt64 │ 0 │ - 266. │ 265 │ UInt64 │ 1 │ - 267. │ 266 │ UInt64 │ 0 │ - 268. │ 267 │ UInt64 │ 1 │ - 269. │ 268 │ UInt64 │ 0 │ - 270. │ 269 │ UInt64 │ 1 │ - 271. │ 270 │ UInt64 │ 0 │ - 272. │ 271 │ UInt64 │ 1 │ - 273. │ 272 │ UInt64 │ 0 │ - 274. │ 273 │ UInt64 │ 1 │ - 275. │ 274 │ UInt64 │ 0 │ - 276. │ 275 │ UInt64 │ 1 │ - 277. │ 276 │ UInt64 │ 0 │ - 278. │ 277 │ UInt64 │ 1 │ - 279. │ 278 │ UInt64 │ 0 │ - 280. │ 279 │ UInt64 │ 1 │ - 281. │ 280 │ UInt64 │ 0 │ - 282. │ 281 │ UInt64 │ 1 │ - 283. │ 282 │ UInt64 │ 0 │ - 284. │ 283 │ UInt64 │ 1 │ - 285. │ 284 │ UInt64 │ 0 │ - 286. │ 285 │ UInt64 │ 1 │ - 287. │ 286 │ UInt64 │ 0 │ - 288. │ 287 │ UInt64 │ 1 │ - 289. │ 288 │ UInt64 │ 0 │ - 290. │ 289 │ UInt64 │ 1 │ - 291. │ 290 │ UInt64 │ 0 │ - 292. │ 291 │ UInt64 │ 1 │ - 293. │ 292 │ UInt64 │ 0 │ - 294. │ 293 │ UInt64 │ 1 │ - 295. │ 294 │ UInt64 │ 0 │ - 296. │ 295 │ UInt64 │ 1 │ - 297. │ 296 │ UInt64 │ 0 │ - 298. │ 297 │ UInt64 │ 1 │ - 299. │ 298 │ UInt64 │ 0 │ - 300. │ 299 │ UInt64 │ 1 │ - 301. │ 300 │ UInt64 │ 0 │ - 302. │ 301 │ UInt64 │ 1 │ - 303. │ 302 │ UInt64 │ 0 │ - 304. │ 303 │ UInt64 │ 1 │ - 305. │ 304 │ UInt64 │ 0 │ - 306. │ 305 │ UInt64 │ 1 │ - 307. │ 306 │ UInt64 │ 0 │ - 308. │ 307 │ UInt64 │ 1 │ - 309. │ 308 │ UInt64 │ 0 │ - 310. │ 309 │ UInt64 │ 1 │ - 311. │ 310 │ UInt64 │ 0 │ - 312. │ 311 │ UInt64 │ 1 │ - 313. │ 312 │ UInt64 │ 0 │ - 314. │ 313 │ UInt64 │ 1 │ - 315. │ 314 │ UInt64 │ 0 │ - 316. │ 315 │ UInt64 │ 1 │ - 317. │ 316 │ UInt64 │ 0 │ - 318. │ 317 │ UInt64 │ 1 │ - 319. │ 318 │ UInt64 │ 0 │ - 320. │ 319 │ UInt64 │ 1 │ - 321. │ 320 │ UInt64 │ 0 │ - 322. │ 321 │ UInt64 │ 1 │ - 323. │ 322 │ UInt64 │ 0 │ - 324. │ 323 │ UInt64 │ 1 │ - 325. │ 324 │ UInt64 │ 0 │ - 326. │ 325 │ UInt64 │ 1 │ - 327. │ 326 │ UInt64 │ 0 │ - 328. │ 327 │ UInt64 │ 1 │ - 329. │ 328 │ UInt64 │ 0 │ - 330. │ 329 │ UInt64 │ 1 │ - 331. │ 330 │ UInt64 │ 0 │ - 332. │ 331 │ UInt64 │ 1 │ - 333. │ 332 │ UInt64 │ 0 │ - 334. │ 333 │ UInt64 │ 1 │ - 335. │ 334 │ UInt64 │ 0 │ - 336. │ 335 │ UInt64 │ 1 │ - 337. │ 336 │ UInt64 │ 0 │ - 338. │ 337 │ UInt64 │ 1 │ - 339. │ 338 │ UInt64 │ 0 │ - 340. │ 339 │ UInt64 │ 1 │ - 341. │ 340 │ UInt64 │ 0 │ - 342. │ 341 │ UInt64 │ 1 │ - 343. │ 342 │ UInt64 │ 0 │ - 344. │ 343 │ UInt64 │ 1 │ - 345. │ 344 │ UInt64 │ 0 │ - 346. │ 345 │ UInt64 │ 1 │ - 347. │ 346 │ UInt64 │ 0 │ - 348. │ 347 │ UInt64 │ 1 │ - 349. │ 348 │ UInt64 │ 0 │ - 350. │ 349 │ UInt64 │ 1 │ - 351. │ 350 │ UInt64 │ 0 │ - 352. │ 351 │ UInt64 │ 1 │ - 353. │ 352 │ UInt64 │ 0 │ - 354. │ 353 │ UInt64 │ 1 │ - 355. │ 354 │ UInt64 │ 0 │ - 356. │ 355 │ UInt64 │ 1 │ - 357. │ 356 │ UInt64 │ 0 │ - 358. │ 357 │ UInt64 │ 1 │ - 359. │ 358 │ UInt64 │ 0 │ - 360. │ 359 │ UInt64 │ 1 │ - 361. │ 360 │ UInt64 │ 0 │ - 362. │ 361 │ UInt64 │ 1 │ - 363. │ 362 │ UInt64 │ 0 │ - 364. │ 363 │ UInt64 │ 1 │ - 365. │ 364 │ UInt64 │ 0 │ - 366. │ 365 │ UInt64 │ 1 │ - 367. │ 366 │ UInt64 │ 0 │ - 368. │ 367 │ UInt64 │ 1 │ - 369. │ 368 │ UInt64 │ 0 │ - 370. │ 369 │ UInt64 │ 1 │ - 371. │ 370 │ UInt64 │ 0 │ - 372. │ 371 │ UInt64 │ 1 │ - 373. │ 372 │ UInt64 │ 0 │ - 374. │ 373 │ UInt64 │ 1 │ - 375. │ 374 │ UInt64 │ 0 │ - 376. │ 375 │ UInt64 │ 1 │ - 377. │ 376 │ UInt64 │ 0 │ - 378. │ 377 │ UInt64 │ 1 │ - 379. │ 378 │ UInt64 │ 0 │ - 380. │ 379 │ UInt64 │ 1 │ - 381. │ 380 │ UInt64 │ 0 │ - 382. │ 381 │ UInt64 │ 1 │ - 383. │ 382 │ UInt64 │ 0 │ - 384. │ 383 │ UInt64 │ 1 │ - 385. │ 384 │ UInt64 │ 0 │ - 386. │ 385 │ UInt64 │ 1 │ - 387. │ 386 │ UInt64 │ 0 │ - 388. │ 387 │ UInt64 │ 1 │ - 389. │ 388 │ UInt64 │ 0 │ - 390. │ 389 │ UInt64 │ 1 │ - 391. │ 390 │ UInt64 │ 0 │ - 392. │ 391 │ UInt64 │ 1 │ - 393. │ 392 │ UInt64 │ 0 │ - 394. │ 393 │ UInt64 │ 1 │ - 395. │ 394 │ UInt64 │ 0 │ - 396. │ 395 │ UInt64 │ 1 │ - 397. │ 396 │ UInt64 │ 0 │ - 398. │ 397 │ UInt64 │ 1 │ - 399. │ 398 │ UInt64 │ 0 │ - 400. │ 399 │ UInt64 │ 1 │ - 401. │ 400 │ UInt64 │ 0 │ - 402. │ 401 │ UInt64 │ 1 │ - 403. │ 402 │ UInt64 │ 0 │ - 404. │ 403 │ UInt64 │ 1 │ - 405. │ 404 │ UInt64 │ 0 │ - 406. │ 405 │ UInt64 │ 1 │ - 407. │ 406 │ UInt64 │ 0 │ - 408. │ 407 │ UInt64 │ 1 │ - 409. │ 408 │ UInt64 │ 0 │ - 410. │ 409 │ UInt64 │ 1 │ - 411. │ 410 │ UInt64 │ 0 │ - 412. │ 411 │ UInt64 │ 1 │ - 413. │ 412 │ UInt64 │ 0 │ - 414. │ 413 │ UInt64 │ 1 │ - 415. │ 414 │ UInt64 │ 0 │ - 416. │ 415 │ UInt64 │ 1 │ - 417. │ 416 │ UInt64 │ 0 │ - 418. │ 417 │ UInt64 │ 1 │ - 419. │ 418 │ UInt64 │ 0 │ - 420. │ 419 │ UInt64 │ 1 │ - 421. │ 420 │ UInt64 │ 0 │ - 422. │ 421 │ UInt64 │ 1 │ - 423. │ 422 │ UInt64 │ 0 │ - 424. │ 423 │ UInt64 │ 1 │ - 425. │ 424 │ UInt64 │ 0 │ - 426. │ 425 │ UInt64 │ 1 │ - 427. │ 426 │ UInt64 │ 0 │ - 428. │ 427 │ UInt64 │ 1 │ - 429. │ 428 │ UInt64 │ 0 │ - 430. │ 429 │ UInt64 │ 1 │ - 431. │ 430 │ UInt64 │ 0 │ - 432. │ 431 │ UInt64 │ 1 │ - 433. │ 432 │ UInt64 │ 0 │ - 434. │ 433 │ UInt64 │ 1 │ - 435. │ 434 │ UInt64 │ 0 │ - 436. │ 435 │ UInt64 │ 1 │ - 437. │ 436 │ UInt64 │ 0 │ - 438. │ 437 │ UInt64 │ 1 │ - 439. │ 438 │ UInt64 │ 0 │ - 440. │ 439 │ UInt64 │ 1 │ - 441. │ 440 │ UInt64 │ 0 │ - 442. │ 441 │ UInt64 │ 1 │ - 443. │ 442 │ UInt64 │ 0 │ - 444. │ 443 │ UInt64 │ 1 │ - 445. │ 444 │ UInt64 │ 0 │ - 446. │ 445 │ UInt64 │ 1 │ - 447. │ 446 │ UInt64 │ 0 │ - 448. │ 447 │ UInt64 │ 1 │ - 449. │ 448 │ UInt64 │ 0 │ - 450. │ 449 │ UInt64 │ 1 │ - 451. │ 450 │ UInt64 │ 0 │ - 452. │ 451 │ UInt64 │ 1 │ - 453. │ 452 │ UInt64 │ 0 │ - 454. │ 453 │ UInt64 │ 1 │ - 455. │ 454 │ UInt64 │ 0 │ - 456. │ 455 │ UInt64 │ 1 │ - 457. │ 456 │ UInt64 │ 0 │ - 458. │ 457 │ UInt64 │ 1 │ - 459. │ 458 │ UInt64 │ 0 │ - 460. │ 459 │ UInt64 │ 1 │ - 461. │ 460 │ UInt64 │ 0 │ - 462. │ 461 │ UInt64 │ 1 │ - 463. │ 462 │ UInt64 │ 0 │ - 464. │ 463 │ UInt64 │ 1 │ - 465. │ 464 │ UInt64 │ 0 │ - 466. │ 465 │ UInt64 │ 1 │ - 467. │ 466 │ UInt64 │ 0 │ - 468. │ 467 │ UInt64 │ 1 │ - 469. │ 468 │ UInt64 │ 0 │ - 470. │ 469 │ UInt64 │ 1 │ - 471. │ 470 │ UInt64 │ 0 │ - 472. │ 471 │ UInt64 │ 1 │ - 473. │ 472 │ UInt64 │ 0 │ - 474. │ 473 │ UInt64 │ 1 │ - 475. │ 474 │ UInt64 │ 0 │ - 476. │ 475 │ UInt64 │ 1 │ - 477. │ 476 │ UInt64 │ 0 │ - 478. │ 477 │ UInt64 │ 1 │ - 479. │ 478 │ UInt64 │ 0 │ - 480. │ 479 │ UInt64 │ 1 │ - 481. │ 480 │ UInt64 │ 0 │ - 482. │ 481 │ UInt64 │ 1 │ - 483. │ 482 │ UInt64 │ 0 │ - 484. │ 483 │ UInt64 │ 1 │ - 485. │ 484 │ UInt64 │ 0 │ - 486. │ 485 │ UInt64 │ 1 │ - 487. │ 486 │ UInt64 │ 0 │ - 488. │ 487 │ UInt64 │ 1 │ - 489. │ 488 │ UInt64 │ 0 │ - 490. │ 489 │ UInt64 │ 1 │ - 491. │ 490 │ UInt64 │ 0 │ - 492. │ 491 │ UInt64 │ 1 │ - 493. │ 492 │ UInt64 │ 0 │ - 494. │ 493 │ UInt64 │ 1 │ - 495. │ 494 │ UInt64 │ 0 │ - 496. │ 495 │ UInt64 │ 1 │ - 497. │ 496 │ UInt64 │ 0 │ - 498. │ 497 │ UInt64 │ 1 │ - 499. │ 498 │ UInt64 │ 0 │ - 500. │ 499 │ UInt64 │ 1 │ - 501. │ 500 │ UInt64 │ 0 │ - 502. │ 501 │ UInt64 │ 1 │ - 503. │ 502 │ UInt64 │ 0 │ - 504. │ 503 │ UInt64 │ 1 │ - 505. │ 504 │ UInt64 │ 0 │ - 506. │ 505 │ UInt64 │ 1 │ - 507. │ 506 │ UInt64 │ 0 │ - 508. │ 507 │ UInt64 │ 1 │ - 509. │ 508 │ UInt64 │ 0 │ - 510. │ 509 │ UInt64 │ 1 │ - 511. │ 510 │ UInt64 │ 0 │ - 512. │ 511 │ UInt64 │ 1 │ - 513. │ 512 │ UInt64 │ 0 │ - 514. │ 513 │ UInt64 │ 1 │ - 515. │ 514 │ UInt64 │ 0 │ - 516. │ 515 │ UInt64 │ 1 │ - 517. │ 516 │ UInt64 │ 0 │ - 518. │ 517 │ UInt64 │ 1 │ - 519. │ 518 │ UInt64 │ 0 │ - 520. │ 519 │ UInt64 │ 1 │ - 521. │ 520 │ UInt64 │ 0 │ - 522. │ 521 │ UInt64 │ 1 │ - 523. │ 522 │ UInt64 │ 0 │ - 524. │ 523 │ UInt64 │ 1 │ - 525. │ 524 │ UInt64 │ 0 │ - 526. │ 525 │ UInt64 │ 1 │ - 527. │ 526 │ UInt64 │ 0 │ - 528. │ 527 │ UInt64 │ 1 │ - 529. │ 528 │ UInt64 │ 0 │ - 530. │ 529 │ UInt64 │ 1 │ - 531. │ 530 │ UInt64 │ 0 │ - 532. │ 531 │ UInt64 │ 1 │ - 533. │ 532 │ UInt64 │ 0 │ - 534. │ 533 │ UInt64 │ 1 │ - 535. │ 534 │ UInt64 │ 0 │ - 536. │ 535 │ UInt64 │ 1 │ - 537. │ 536 │ UInt64 │ 0 │ - 538. │ 537 │ UInt64 │ 1 │ - 539. │ 538 │ UInt64 │ 0 │ - 540. │ 539 │ UInt64 │ 1 │ - 541. │ 540 │ UInt64 │ 0 │ - 542. │ 541 │ UInt64 │ 1 │ - 543. │ 542 │ UInt64 │ 0 │ - 544. │ 543 │ UInt64 │ 1 │ - 545. │ 544 │ UInt64 │ 0 │ - 546. │ 545 │ UInt64 │ 1 │ - 547. │ 546 │ UInt64 │ 0 │ - 548. │ 547 │ UInt64 │ 1 │ - 549. │ 548 │ UInt64 │ 0 │ - 550. │ 549 │ UInt64 │ 1 │ - 551. │ 550 │ UInt64 │ 0 │ - 552. │ 551 │ UInt64 │ 1 │ - 553. │ 552 │ UInt64 │ 0 │ - 554. │ 553 │ UInt64 │ 1 │ - 555. │ 554 │ UInt64 │ 0 │ - 556. │ 555 │ UInt64 │ 1 │ - 557. │ 556 │ UInt64 │ 0 │ - 558. │ 557 │ UInt64 │ 1 │ - 559. │ 558 │ UInt64 │ 0 │ - 560. │ 559 │ UInt64 │ 1 │ - 561. │ 560 │ UInt64 │ 0 │ - 562. │ 561 │ UInt64 │ 1 │ - 563. │ 562 │ UInt64 │ 0 │ - 564. │ 563 │ UInt64 │ 1 │ - 565. │ 564 │ UInt64 │ 0 │ - 566. │ 565 │ UInt64 │ 1 │ - 567. │ 566 │ UInt64 │ 0 │ - 568. │ 567 │ UInt64 │ 1 │ - 569. │ 568 │ UInt64 │ 0 │ - 570. │ 569 │ UInt64 │ 1 │ - 571. │ 570 │ UInt64 │ 0 │ - 572. │ 571 │ UInt64 │ 1 │ - 573. │ 572 │ UInt64 │ 0 │ - 574. │ 573 │ UInt64 │ 1 │ - 575. │ 574 │ UInt64 │ 0 │ - 576. │ 575 │ UInt64 │ 1 │ - 577. │ 576 │ UInt64 │ 0 │ - 578. │ 577 │ UInt64 │ 1 │ - 579. │ 578 │ UInt64 │ 0 │ - 580. │ 579 │ UInt64 │ 1 │ - 581. │ 580 │ UInt64 │ 0 │ - 582. │ 581 │ UInt64 │ 1 │ - 583. │ 582 │ UInt64 │ 0 │ - 584. │ 583 │ UInt64 │ 1 │ - 585. │ 584 │ UInt64 │ 0 │ - 586. │ 585 │ UInt64 │ 1 │ - 587. │ 586 │ UInt64 │ 0 │ - 588. │ 587 │ UInt64 │ 1 │ - 589. │ 588 │ UInt64 │ 0 │ - 590. │ 589 │ UInt64 │ 1 │ - 591. │ 590 │ UInt64 │ 0 │ - 592. │ 591 │ UInt64 │ 1 │ - 593. │ 592 │ UInt64 │ 0 │ - 594. │ 593 │ UInt64 │ 1 │ - 595. │ 594 │ UInt64 │ 0 │ - 596. │ 595 │ UInt64 │ 1 │ - 597. │ 596 │ UInt64 │ 0 │ - 598. │ 597 │ UInt64 │ 1 │ - 599. │ 598 │ UInt64 │ 0 │ - 600. │ 599 │ UInt64 │ 1 │ - 601. │ 600 │ UInt64 │ 0 │ - 602. │ 601 │ UInt64 │ 1 │ - 603. │ 602 │ UInt64 │ 0 │ - 604. │ 603 │ UInt64 │ 1 │ - 605. │ 604 │ UInt64 │ 0 │ - 606. │ 605 │ UInt64 │ 1 │ - 607. │ 606 │ UInt64 │ 0 │ - 608. │ 607 │ UInt64 │ 1 │ - 609. │ 608 │ UInt64 │ 0 │ - 610. │ 609 │ UInt64 │ 1 │ - 611. │ 610 │ UInt64 │ 0 │ - 612. │ 611 │ UInt64 │ 1 │ - 613. │ 612 │ UInt64 │ 0 │ - 614. │ 613 │ UInt64 │ 1 │ - 615. │ 614 │ UInt64 │ 0 │ - 616. │ 615 │ UInt64 │ 1 │ - 617. │ 616 │ UInt64 │ 0 │ - 618. │ 617 │ UInt64 │ 1 │ - 619. │ 618 │ UInt64 │ 0 │ - 620. │ 619 │ UInt64 │ 1 │ - 621. │ 620 │ UInt64 │ 0 │ - 622. │ 621 │ UInt64 │ 1 │ - 623. │ 622 │ UInt64 │ 0 │ - 624. │ 623 │ UInt64 │ 1 │ - 625. │ 624 │ UInt64 │ 0 │ - 626. │ 625 │ UInt64 │ 1 │ - 627. │ 626 │ UInt64 │ 0 │ - 628. │ 627 │ UInt64 │ 1 │ - 629. │ 628 │ UInt64 │ 0 │ - 630. │ 629 │ UInt64 │ 1 │ - 631. │ 630 │ UInt64 │ 0 │ - 632. │ 631 │ UInt64 │ 1 │ - 633. │ 632 │ UInt64 │ 0 │ - 634. │ 633 │ UInt64 │ 1 │ - 635. │ 634 │ UInt64 │ 0 │ - 636. │ 635 │ UInt64 │ 1 │ - 637. │ 636 │ UInt64 │ 0 │ - 638. │ 637 │ UInt64 │ 1 │ - 639. │ 638 │ UInt64 │ 0 │ - 640. │ 639 │ UInt64 │ 1 │ - 641. │ 640 │ UInt64 │ 0 │ - 642. │ 641 │ UInt64 │ 1 │ - 643. │ 642 │ UInt64 │ 0 │ - 644. │ 643 │ UInt64 │ 1 │ - 645. │ 644 │ UInt64 │ 0 │ - 646. │ 645 │ UInt64 │ 1 │ - 647. │ 646 │ UInt64 │ 0 │ - 648. │ 647 │ UInt64 │ 1 │ - 649. │ 648 │ UInt64 │ 0 │ - 650. │ 649 │ UInt64 │ 1 │ - 651. │ 650 │ UInt64 │ 0 │ - 652. │ 651 │ UInt64 │ 1 │ - 653. │ 652 │ UInt64 │ 0 │ - 654. │ 653 │ UInt64 │ 1 │ - 655. │ 654 │ UInt64 │ 0 │ - 656. │ 655 │ UInt64 │ 1 │ - 657. │ 656 │ UInt64 │ 0 │ - 658. │ 657 │ UInt64 │ 1 │ - 659. │ 658 │ UInt64 │ 0 │ - 660. │ 659 │ UInt64 │ 1 │ - 661. │ 660 │ UInt64 │ 0 │ - 662. │ 661 │ UInt64 │ 1 │ - 663. │ 662 │ UInt64 │ 0 │ - 664. │ 663 │ UInt64 │ 1 │ - 665. │ 664 │ UInt64 │ 0 │ - 666. │ 665 │ UInt64 │ 1 │ - 667. │ 666 │ UInt64 │ 0 │ - 668. │ 667 │ UInt64 │ 1 │ - 669. │ 668 │ UInt64 │ 0 │ - 670. │ 669 │ UInt64 │ 1 │ - 671. │ 670 │ UInt64 │ 0 │ - 672. │ 671 │ UInt64 │ 1 │ - 673. │ 672 │ UInt64 │ 0 │ - 674. │ 673 │ UInt64 │ 1 │ - 675. │ 674 │ UInt64 │ 0 │ - 676. │ 675 │ UInt64 │ 1 │ - 677. │ 676 │ UInt64 │ 0 │ - 678. │ 677 │ UInt64 │ 1 │ - 679. │ 678 │ UInt64 │ 0 │ - 680. │ 679 │ UInt64 │ 1 │ - 681. │ 680 │ UInt64 │ 0 │ - 682. │ 681 │ UInt64 │ 1 │ - 683. │ 682 │ UInt64 │ 0 │ - 684. │ 683 │ UInt64 │ 1 │ - 685. │ 684 │ UInt64 │ 0 │ - 686. │ 685 │ UInt64 │ 1 │ - 687. │ 686 │ UInt64 │ 0 │ - 688. │ 687 │ UInt64 │ 1 │ - 689. │ 688 │ UInt64 │ 0 │ - 690. │ 689 │ UInt64 │ 1 │ - 691. │ 690 │ UInt64 │ 0 │ - 692. │ 691 │ UInt64 │ 1 │ - 693. │ 692 │ UInt64 │ 0 │ - 694. │ 693 │ UInt64 │ 1 │ - 695. │ 694 │ UInt64 │ 0 │ - 696. │ 695 │ UInt64 │ 1 │ - 697. │ 696 │ UInt64 │ 0 │ - 698. │ 697 │ UInt64 │ 1 │ - 699. │ 698 │ UInt64 │ 0 │ - 700. │ 699 │ UInt64 │ 1 │ - 701. │ 700 │ UInt64 │ 0 │ - 702. │ 701 │ UInt64 │ 1 │ - 703. │ 702 │ UInt64 │ 0 │ - 704. │ 703 │ UInt64 │ 1 │ - 705. │ 704 │ UInt64 │ 0 │ - 706. │ 705 │ UInt64 │ 1 │ - 707. │ 706 │ UInt64 │ 0 │ - 708. │ 707 │ UInt64 │ 1 │ - 709. │ 708 │ UInt64 │ 0 │ - 710. │ 709 │ UInt64 │ 1 │ - 711. │ 710 │ UInt64 │ 0 │ - 712. │ 711 │ UInt64 │ 1 │ - 713. │ 712 │ UInt64 │ 0 │ - 714. │ 713 │ UInt64 │ 1 │ - 715. │ 714 │ UInt64 │ 0 │ - 716. │ 715 │ UInt64 │ 1 │ - 717. │ 716 │ UInt64 │ 0 │ - 718. │ 717 │ UInt64 │ 1 │ - 719. │ 718 │ UInt64 │ 0 │ - 720. │ 719 │ UInt64 │ 1 │ - 721. │ 720 │ UInt64 │ 0 │ - 722. │ 721 │ UInt64 │ 1 │ - 723. │ 722 │ UInt64 │ 0 │ - 724. │ 723 │ UInt64 │ 1 │ - 725. │ 724 │ UInt64 │ 0 │ - 726. │ 725 │ UInt64 │ 1 │ - 727. │ 726 │ UInt64 │ 0 │ - 728. │ 727 │ UInt64 │ 1 │ - 729. │ 728 │ UInt64 │ 0 │ - 730. │ 729 │ UInt64 │ 1 │ - 731. │ 730 │ UInt64 │ 0 │ - 732. │ 731 │ UInt64 │ 1 │ - 733. │ 732 │ UInt64 │ 0 │ - 734. │ 733 │ UInt64 │ 1 │ - 735. │ 734 │ UInt64 │ 0 │ - 736. │ 735 │ UInt64 │ 1 │ - 737. │ 736 │ UInt64 │ 0 │ - 738. │ 737 │ UInt64 │ 1 │ - 739. │ 738 │ UInt64 │ 0 │ - 740. │ 739 │ UInt64 │ 1 │ - 741. │ 740 │ UInt64 │ 0 │ - 742. │ 741 │ UInt64 │ 1 │ - 743. │ 742 │ UInt64 │ 0 │ - 744. │ 743 │ UInt64 │ 1 │ - 745. │ 744 │ UInt64 │ 0 │ - 746. │ 745 │ UInt64 │ 1 │ - 747. │ 746 │ UInt64 │ 0 │ - 748. │ 747 │ UInt64 │ 1 │ - 749. │ 748 │ UInt64 │ 0 │ - 750. │ 749 │ UInt64 │ 1 │ - 751. │ 750 │ UInt64 │ 0 │ - 752. │ 751 │ UInt64 │ 1 │ - 753. │ 752 │ UInt64 │ 0 │ - 754. │ 753 │ UInt64 │ 1 │ - 755. │ 754 │ UInt64 │ 0 │ - 756. │ 755 │ UInt64 │ 1 │ - 757. │ 756 │ UInt64 │ 0 │ - 758. │ 757 │ UInt64 │ 1 │ - 759. │ 758 │ UInt64 │ 0 │ - 760. │ 759 │ UInt64 │ 1 │ - 761. │ 760 │ UInt64 │ 0 │ - 762. │ 761 │ UInt64 │ 1 │ - 763. │ 762 │ UInt64 │ 0 │ - 764. │ 763 │ UInt64 │ 1 │ - 765. │ 764 │ UInt64 │ 0 │ - 766. │ 765 │ UInt64 │ 1 │ - 767. │ 766 │ UInt64 │ 0 │ - 768. │ 767 │ UInt64 │ 1 │ - 769. │ 768 │ UInt64 │ 0 │ - 770. │ 769 │ UInt64 │ 1 │ - 771. │ 770 │ UInt64 │ 0 │ - 772. │ 771 │ UInt64 │ 1 │ - 773. │ 772 │ UInt64 │ 0 │ - 774. │ 773 │ UInt64 │ 1 │ - 775. │ 774 │ UInt64 │ 0 │ - 776. │ 775 │ UInt64 │ 1 │ - 777. │ 776 │ UInt64 │ 0 │ - 778. │ 777 │ UInt64 │ 1 │ - 779. │ 778 │ UInt64 │ 0 │ - 780. │ 779 │ UInt64 │ 1 │ - 781. │ 780 │ UInt64 │ 0 │ - 782. │ 781 │ UInt64 │ 1 │ - 783. │ 782 │ UInt64 │ 0 │ - 784. │ 783 │ UInt64 │ 1 │ - 785. │ 784 │ UInt64 │ 0 │ - 786. │ 785 │ UInt64 │ 1 │ - 787. │ 786 │ UInt64 │ 0 │ - 788. │ 787 │ UInt64 │ 1 │ - 789. │ 788 │ UInt64 │ 0 │ - 790. │ 789 │ UInt64 │ 1 │ - 791. │ 790 │ UInt64 │ 0 │ - 792. │ 791 │ UInt64 │ 1 │ - 793. │ 792 │ UInt64 │ 0 │ - 794. │ 793 │ UInt64 │ 1 │ - 795. │ 794 │ UInt64 │ 0 │ - 796. │ 795 │ UInt64 │ 1 │ - 797. │ 796 │ UInt64 │ 0 │ - 798. │ 797 │ UInt64 │ 1 │ - 799. │ 798 │ UInt64 │ 0 │ - 800. │ 799 │ UInt64 │ 1 │ - 801. │ 800 │ UInt64 │ 0 │ - 802. │ 801 │ UInt64 │ 1 │ - 803. │ 802 │ UInt64 │ 0 │ - 804. │ 803 │ UInt64 │ 1 │ - 805. │ 804 │ UInt64 │ 0 │ - 806. │ 805 │ UInt64 │ 1 │ - 807. │ 806 │ UInt64 │ 0 │ - 808. │ 807 │ UInt64 │ 1 │ - 809. │ 808 │ UInt64 │ 0 │ - 810. │ 809 │ UInt64 │ 1 │ - 811. │ 810 │ UInt64 │ 0 │ - 812. │ 811 │ UInt64 │ 1 │ - 813. │ 812 │ UInt64 │ 0 │ - 814. │ 813 │ UInt64 │ 1 │ - 815. │ 814 │ UInt64 │ 0 │ - 816. │ 815 │ UInt64 │ 1 │ - 817. │ 816 │ UInt64 │ 0 │ - 818. │ 817 │ UInt64 │ 1 │ - 819. │ 818 │ UInt64 │ 0 │ - 820. │ 819 │ UInt64 │ 1 │ - 821. │ 820 │ UInt64 │ 0 │ - 822. │ 821 │ UInt64 │ 1 │ - 823. │ 822 │ UInt64 │ 0 │ - 824. │ 823 │ UInt64 │ 1 │ - 825. │ 824 │ UInt64 │ 0 │ - 826. │ 825 │ UInt64 │ 1 │ - 827. │ 826 │ UInt64 │ 0 │ - 828. │ 827 │ UInt64 │ 1 │ - 829. │ 828 │ UInt64 │ 0 │ - 830. │ 829 │ UInt64 │ 1 │ - 831. │ 830 │ UInt64 │ 0 │ - 832. │ 831 │ UInt64 │ 1 │ - 833. │ 832 │ UInt64 │ 0 │ - 834. │ 833 │ UInt64 │ 1 │ - 835. │ 834 │ UInt64 │ 0 │ - 836. │ 835 │ UInt64 │ 1 │ - 837. │ 836 │ UInt64 │ 0 │ - 838. │ 837 │ UInt64 │ 1 │ - 839. │ 838 │ UInt64 │ 0 │ - 840. │ 839 │ UInt64 │ 1 │ - 841. │ 840 │ UInt64 │ 0 │ - 842. │ 841 │ UInt64 │ 1 │ - 843. │ 842 │ UInt64 │ 0 │ - 844. │ 843 │ UInt64 │ 1 │ - 845. │ 844 │ UInt64 │ 0 │ - 846. │ 845 │ UInt64 │ 1 │ - 847. │ 846 │ UInt64 │ 0 │ - 848. │ 847 │ UInt64 │ 1 │ - 849. │ 848 │ UInt64 │ 0 │ - 850. │ 849 │ UInt64 │ 1 │ - 851. │ 850 │ UInt64 │ 0 │ - 852. │ 851 │ UInt64 │ 1 │ - 853. │ 852 │ UInt64 │ 0 │ - 854. │ 853 │ UInt64 │ 1 │ - 855. │ 854 │ UInt64 │ 0 │ - 856. │ 855 │ UInt64 │ 1 │ - 857. │ 856 │ UInt64 │ 0 │ - 858. │ 857 │ UInt64 │ 1 │ - 859. │ 858 │ UInt64 │ 0 │ - 860. │ 859 │ UInt64 │ 1 │ - 861. │ 860 │ UInt64 │ 0 │ - 862. │ 861 │ UInt64 │ 1 │ - 863. │ 862 │ UInt64 │ 0 │ - 864. │ 863 │ UInt64 │ 1 │ - 865. │ 864 │ UInt64 │ 0 │ - 866. │ 865 │ UInt64 │ 1 │ - 867. │ 866 │ UInt64 │ 0 │ - 868. │ 867 │ UInt64 │ 1 │ - 869. │ 868 │ UInt64 │ 0 │ - 870. │ 869 │ UInt64 │ 1 │ - 871. │ 870 │ UInt64 │ 0 │ - 872. │ 871 │ UInt64 │ 1 │ - 873. │ 872 │ UInt64 │ 0 │ - 874. │ 873 │ UInt64 │ 1 │ - 875. │ 874 │ UInt64 │ 0 │ - 876. │ 875 │ UInt64 │ 1 │ - 877. │ 876 │ UInt64 │ 0 │ - 878. │ 877 │ UInt64 │ 1 │ - 879. │ 878 │ UInt64 │ 0 │ - 880. │ 879 │ UInt64 │ 1 │ - 881. │ 880 │ UInt64 │ 0 │ - 882. │ 881 │ UInt64 │ 1 │ - 883. │ 882 │ UInt64 │ 0 │ - 884. │ 883 │ UInt64 │ 1 │ - 885. │ 884 │ UInt64 │ 0 │ - 886. │ 885 │ UInt64 │ 1 │ - 887. │ 886 │ UInt64 │ 0 │ - 888. │ 887 │ UInt64 │ 1 │ - 889. │ 888 │ UInt64 │ 0 │ - 890. │ 889 │ UInt64 │ 1 │ - 891. │ 890 │ UInt64 │ 0 │ - 892. │ 891 │ UInt64 │ 1 │ - 893. │ 892 │ UInt64 │ 0 │ - 894. │ 893 │ UInt64 │ 1 │ - 895. │ 894 │ UInt64 │ 0 │ - 896. │ 895 │ UInt64 │ 1 │ - 897. │ 896 │ UInt64 │ 0 │ - 898. │ 897 │ UInt64 │ 1 │ - 899. │ 898 │ UInt64 │ 0 │ - 900. │ 899 │ UInt64 │ 1 │ - 901. │ 900 │ UInt64 │ 0 │ - 902. │ 901 │ UInt64 │ 1 │ - 903. │ 902 │ UInt64 │ 0 │ - 904. │ 903 │ UInt64 │ 1 │ - 905. │ 904 │ UInt64 │ 0 │ - 906. │ 905 │ UInt64 │ 1 │ - 907. │ 906 │ UInt64 │ 0 │ - 908. │ 907 │ UInt64 │ 1 │ - 909. │ 908 │ UInt64 │ 0 │ - 910. │ 909 │ UInt64 │ 1 │ - 911. │ 910 │ UInt64 │ 0 │ - 912. │ 911 │ UInt64 │ 1 │ - 913. │ 912 │ UInt64 │ 0 │ - 914. │ 913 │ UInt64 │ 1 │ - 915. │ 914 │ UInt64 │ 0 │ - 916. │ 915 │ UInt64 │ 1 │ - 917. │ 916 │ UInt64 │ 0 │ - 918. │ 917 │ UInt64 │ 1 │ - 919. │ 918 │ UInt64 │ 0 │ - 920. │ 919 │ UInt64 │ 1 │ - 921. │ 920 │ UInt64 │ 0 │ - 922. │ 921 │ UInt64 │ 1 │ - 923. │ 922 │ UInt64 │ 0 │ - 924. │ 923 │ UInt64 │ 1 │ - 925. │ 924 │ UInt64 │ 0 │ - 926. │ 925 │ UInt64 │ 1 │ - 927. │ 926 │ UInt64 │ 0 │ - 928. │ 927 │ UInt64 │ 1 │ - 929. │ 928 │ UInt64 │ 0 │ - 930. │ 929 │ UInt64 │ 1 │ - 931. │ 930 │ UInt64 │ 0 │ - 932. │ 931 │ UInt64 │ 1 │ - 933. │ 932 │ UInt64 │ 0 │ - 934. │ 933 │ UInt64 │ 1 │ - 935. │ 934 │ UInt64 │ 0 │ - 936. │ 935 │ UInt64 │ 1 │ - 937. │ 936 │ UInt64 │ 0 │ - 938. │ 937 │ UInt64 │ 1 │ - 939. │ 938 │ UInt64 │ 0 │ - 940. │ 939 │ UInt64 │ 1 │ - 941. │ 940 │ UInt64 │ 0 │ - 942. │ 941 │ UInt64 │ 1 │ - 943. │ 942 │ UInt64 │ 0 │ - 944. │ 943 │ UInt64 │ 1 │ - 945. │ 944 │ UInt64 │ 0 │ - 946. │ 945 │ UInt64 │ 1 │ - 947. │ 946 │ UInt64 │ 0 │ - 948. │ 947 │ UInt64 │ 1 │ - 949. │ 948 │ UInt64 │ 0 │ - 950. │ 949 │ UInt64 │ 1 │ - 951. │ 950 │ UInt64 │ 0 │ - 952. │ 951 │ UInt64 │ 1 │ - 953. │ 952 │ UInt64 │ 0 │ - 954. │ 953 │ UInt64 │ 1 │ - 955. │ 954 │ UInt64 │ 0 │ - 956. │ 955 │ UInt64 │ 1 │ - 957. │ 956 │ UInt64 │ 0 │ - 958. │ 957 │ UInt64 │ 1 │ - 959. │ 958 │ UInt64 │ 0 │ - 960. │ 959 │ UInt64 │ 1 │ - 961. │ 960 │ UInt64 │ 0 │ - 962. │ 961 │ UInt64 │ 1 │ - 963. │ 962 │ UInt64 │ 0 │ - 964. │ 963 │ UInt64 │ 1 │ - 965. │ 964 │ UInt64 │ 0 │ - 966. │ 965 │ UInt64 │ 1 │ - 967. │ 966 │ UInt64 │ 0 │ - 968. │ 967 │ UInt64 │ 1 │ - 969. │ 968 │ UInt64 │ 0 │ - 970. │ 969 │ UInt64 │ 1 │ - 971. │ 970 │ UInt64 │ 0 │ - 972. │ 971 │ UInt64 │ 1 │ - 973. │ 972 │ UInt64 │ 0 │ - 974. │ 973 │ UInt64 │ 1 │ - 975. │ 974 │ UInt64 │ 0 │ - 976. │ 975 │ UInt64 │ 1 │ - 977. │ 976 │ UInt64 │ 0 │ - 978. │ 977 │ UInt64 │ 1 │ - 979. │ 978 │ UInt64 │ 0 │ - 980. │ 979 │ UInt64 │ 1 │ - 981. │ 980 │ UInt64 │ 0 │ - 982. │ 981 │ UInt64 │ 1 │ - 983. │ 982 │ UInt64 │ 0 │ - 984. │ 983 │ UInt64 │ 1 │ - 985. │ 984 │ UInt64 │ 0 │ - 986. │ 985 │ UInt64 │ 1 │ - 987. │ 986 │ UInt64 │ 0 │ - 988. │ 987 │ UInt64 │ 1 │ - 989. │ 988 │ UInt64 │ 0 │ - 990. │ 989 │ UInt64 │ 1 │ - 991. │ 990 │ UInt64 │ 0 │ - 992. │ 991 │ UInt64 │ 1 │ - 993. │ 992 │ UInt64 │ 0 │ - 994. │ 993 │ UInt64 │ 1 │ - 995. │ 994 │ UInt64 │ 0 │ - 996. │ 995 │ UInt64 │ 1 │ - 997. │ 996 │ UInt64 │ 0 │ - 998. │ 997 │ UInt64 │ 1 │ - 999. │ 998 │ UInt64 │ 0 │ -1000. │ 999 │ UInt64 │ 1 │ - └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ - 100. │ 99 │ UInt64 │ 1 │ - 101. │ 100 │ UInt64 │ 0 │ - 102. │ 101 │ UInt64 │ 1 │ - 103. │ 102 │ UInt64 │ 0 │ - 104. │ 103 │ UInt64 │ 1 │ - 105. │ 104 │ UInt64 │ 0 │ - 106. │ 105 │ UInt64 │ 1 │ - 107. │ 106 │ UInt64 │ 0 │ - 108. │ 107 │ UInt64 │ 1 │ - 109. │ 108 │ UInt64 │ 0 │ - 110. │ 109 │ UInt64 │ 1 │ - 111. │ 110 │ UInt64 │ 0 │ - 112. │ 111 │ UInt64 │ 1 │ - 113. │ 112 │ UInt64 │ 0 │ - 114. │ 113 │ UInt64 │ 1 │ - 115. │ 114 │ UInt64 │ 0 │ - 116. │ 115 │ UInt64 │ 1 │ - 117. │ 116 │ UInt64 │ 0 │ - 118. │ 117 │ UInt64 │ 1 │ - 119. │ 118 │ UInt64 │ 0 │ - 120. │ 119 │ UInt64 │ 1 │ - 121. │ 120 │ UInt64 │ 0 │ - 122. │ 121 │ UInt64 │ 1 │ - 123. │ 122 │ UInt64 │ 0 │ - 124. │ 123 │ UInt64 │ 1 │ - 125. │ 124 │ UInt64 │ 0 │ - 126. │ 125 │ UInt64 │ 1 │ - 127. │ 126 │ UInt64 │ 0 │ - 128. │ 127 │ UInt64 │ 1 │ - 129. │ 128 │ UInt64 │ 0 │ - 130. │ 129 │ UInt64 │ 1 │ - 131. │ 130 │ UInt64 │ 0 │ - 132. │ 131 │ UInt64 │ 1 │ - 133. │ 132 │ UInt64 │ 0 │ - 134. │ 133 │ UInt64 │ 1 │ - 135. │ 134 │ UInt64 │ 0 │ - 136. │ 135 │ UInt64 │ 1 │ - 137. │ 136 │ UInt64 │ 0 │ - 138. │ 137 │ UInt64 │ 1 │ - 139. │ 138 │ UInt64 │ 0 │ - 140. │ 139 │ UInt64 │ 1 │ - 141. │ 140 │ UInt64 │ 0 │ - 142. │ 141 │ UInt64 │ 1 │ - 143. │ 142 │ UInt64 │ 0 │ - 144. │ 143 │ UInt64 │ 1 │ - 145. │ 144 │ UInt64 │ 0 │ - 146. │ 145 │ UInt64 │ 1 │ - 147. │ 146 │ UInt64 │ 0 │ - 148. │ 147 │ UInt64 │ 1 │ - 149. │ 148 │ UInt64 │ 0 │ - 150. │ 149 │ UInt64 │ 1 │ - 151. │ 150 │ UInt64 │ 0 │ - 152. │ 151 │ UInt64 │ 1 │ - 153. │ 152 │ UInt64 │ 0 │ - 154. │ 153 │ UInt64 │ 1 │ - 155. │ 154 │ UInt64 │ 0 │ - 156. │ 155 │ UInt64 │ 1 │ - 157. │ 156 │ UInt64 │ 0 │ - 158. │ 157 │ UInt64 │ 1 │ - 159. │ 158 │ UInt64 │ 0 │ - 160. │ 159 │ UInt64 │ 1 │ - 161. │ 160 │ UInt64 │ 0 │ - 162. │ 161 │ UInt64 │ 1 │ - 163. │ 162 │ UInt64 │ 0 │ - 164. │ 163 │ UInt64 │ 1 │ - 165. │ 164 │ UInt64 │ 0 │ - 166. │ 165 │ UInt64 │ 1 │ - 167. │ 166 │ UInt64 │ 0 │ - 168. │ 167 │ UInt64 │ 1 │ - 169. │ 168 │ UInt64 │ 0 │ - 170. │ 169 │ UInt64 │ 1 │ - 171. │ 170 │ UInt64 │ 0 │ - 172. │ 171 │ UInt64 │ 1 │ - 173. │ 172 │ UInt64 │ 0 │ - 174. │ 173 │ UInt64 │ 1 │ - 175. │ 174 │ UInt64 │ 0 │ - 176. │ 175 │ UInt64 │ 1 │ - 177. │ 176 │ UInt64 │ 0 │ - 178. │ 177 │ UInt64 │ 1 │ - 179. │ 178 │ UInt64 │ 0 │ - 180. │ 179 │ UInt64 │ 1 │ - 181. │ 180 │ UInt64 │ 0 │ - 182. │ 181 │ UInt64 │ 1 │ - 183. │ 182 │ UInt64 │ 0 │ - 184. │ 183 │ UInt64 │ 1 │ - 185. │ 184 │ UInt64 │ 0 │ - 186. │ 185 │ UInt64 │ 1 │ - 187. │ 186 │ UInt64 │ 0 │ - 188. │ 187 │ UInt64 │ 1 │ - 189. │ 188 │ UInt64 │ 0 │ - 190. │ 189 │ UInt64 │ 1 │ - 191. │ 190 │ UInt64 │ 0 │ - 192. │ 191 │ UInt64 │ 1 │ - 193. │ 192 │ UInt64 │ 0 │ - 194. │ 193 │ UInt64 │ 1 │ - 195. │ 194 │ UInt64 │ 0 │ - 196. │ 195 │ UInt64 │ 1 │ - 197. │ 196 │ UInt64 │ 0 │ - 198. │ 197 │ UInt64 │ 1 │ - 199. │ 198 │ UInt64 │ 0 │ - 200. │ 199 │ UInt64 │ 1 │ - 201. │ 200 │ UInt64 │ 0 │ - 202. │ 201 │ UInt64 │ 1 │ - 203. │ 202 │ UInt64 │ 0 │ - 204. │ 203 │ UInt64 │ 1 │ - 205. │ 204 │ UInt64 │ 0 │ - 206. │ 205 │ UInt64 │ 1 │ - 207. │ 206 │ UInt64 │ 0 │ - 208. │ 207 │ UInt64 │ 1 │ - 209. │ 208 │ UInt64 │ 0 │ - 210. │ 209 │ UInt64 │ 1 │ - 211. │ 210 │ UInt64 │ 0 │ - 212. │ 211 │ UInt64 │ 1 │ - 213. │ 212 │ UInt64 │ 0 │ - 214. │ 213 │ UInt64 │ 1 │ - 215. │ 214 │ UInt64 │ 0 │ - 216. │ 215 │ UInt64 │ 1 │ - 217. │ 216 │ UInt64 │ 0 │ - 218. │ 217 │ UInt64 │ 1 │ - 219. │ 218 │ UInt64 │ 0 │ - 220. │ 219 │ UInt64 │ 1 │ - 221. │ 220 │ UInt64 │ 0 │ - 222. │ 221 │ UInt64 │ 1 │ - 223. │ 222 │ UInt64 │ 0 │ - 224. │ 223 │ UInt64 │ 1 │ - 225. │ 224 │ UInt64 │ 0 │ - 226. │ 225 │ UInt64 │ 1 │ - 227. │ 226 │ UInt64 │ 0 │ - 228. │ 227 │ UInt64 │ 1 │ - 229. │ 228 │ UInt64 │ 0 │ - 230. │ 229 │ UInt64 │ 1 │ - 231. │ 230 │ UInt64 │ 0 │ - 232. │ 231 │ UInt64 │ 1 │ - 233. │ 232 │ UInt64 │ 0 │ - 234. │ 233 │ UInt64 │ 1 │ - 235. │ 234 │ UInt64 │ 0 │ - 236. │ 235 │ UInt64 │ 1 │ - 237. │ 236 │ UInt64 │ 0 │ - 238. │ 237 │ UInt64 │ 1 │ - 239. │ 238 │ UInt64 │ 0 │ - 240. │ 239 │ UInt64 │ 1 │ - 241. │ 240 │ UInt64 │ 0 │ - 242. │ 241 │ UInt64 │ 1 │ - 243. │ 242 │ UInt64 │ 0 │ - 244. │ 243 │ UInt64 │ 1 │ - 245. │ 244 │ UInt64 │ 0 │ - 246. │ 245 │ UInt64 │ 1 │ - 247. │ 246 │ UInt64 │ 0 │ - 248. │ 247 │ UInt64 │ 1 │ - 249. │ 248 │ UInt64 │ 0 │ - 250. │ 249 │ UInt64 │ 1 │ - 251. │ 250 │ UInt64 │ 0 │ - 252. │ 251 │ UInt64 │ 1 │ - 253. │ 252 │ UInt64 │ 0 │ - 254. │ 253 │ UInt64 │ 1 │ - 255. │ 254 │ UInt64 │ 0 │ - 256. │ 255 │ UInt64 │ 1 │ - 257. │ 256 │ UInt64 │ 0 │ - 258. │ 257 │ UInt64 │ 1 │ - 259. │ 258 │ UInt64 │ 0 │ - 260. │ 259 │ UInt64 │ 1 │ - 261. │ 260 │ UInt64 │ 0 │ - 262. │ 261 │ UInt64 │ 1 │ - 263. │ 262 │ UInt64 │ 0 │ - 264. │ 263 │ UInt64 │ 1 │ - 265. │ 264 │ UInt64 │ 0 │ - 266. │ 265 │ UInt64 │ 1 │ - 267. │ 266 │ UInt64 │ 0 │ - 268. │ 267 │ UInt64 │ 1 │ - 269. │ 268 │ UInt64 │ 0 │ - 270. │ 269 │ UInt64 │ 1 │ - 271. │ 270 │ UInt64 │ 0 │ - 272. │ 271 │ UInt64 │ 1 │ - 273. │ 272 │ UInt64 │ 0 │ - 274. │ 273 │ UInt64 │ 1 │ - 275. │ 274 │ UInt64 │ 0 │ - 276. │ 275 │ UInt64 │ 1 │ - 277. │ 276 │ UInt64 │ 0 │ - 278. │ 277 │ UInt64 │ 1 │ - 279. │ 278 │ UInt64 │ 0 │ - 280. │ 279 │ UInt64 │ 1 │ - 281. │ 280 │ UInt64 │ 0 │ - 282. │ 281 │ UInt64 │ 1 │ - 283. │ 282 │ UInt64 │ 0 │ - 284. │ 283 │ UInt64 │ 1 │ - 285. │ 284 │ UInt64 │ 0 │ - 286. │ 285 │ UInt64 │ 1 │ - 287. │ 286 │ UInt64 │ 0 │ - 288. │ 287 │ UInt64 │ 1 │ - 289. │ 288 │ UInt64 │ 0 │ - 290. │ 289 │ UInt64 │ 1 │ - 291. │ 290 │ UInt64 │ 0 │ - 292. │ 291 │ UInt64 │ 1 │ - 293. │ 292 │ UInt64 │ 0 │ - 294. │ 293 │ UInt64 │ 1 │ - 295. │ 294 │ UInt64 │ 0 │ - 296. │ 295 │ UInt64 │ 1 │ - 297. │ 296 │ UInt64 │ 0 │ - 298. │ 297 │ UInt64 │ 1 │ - 299. │ 298 │ UInt64 │ 0 │ - 300. │ 299 │ UInt64 │ 1 │ - 301. │ 300 │ UInt64 │ 0 │ - 302. │ 301 │ UInt64 │ 1 │ - 303. │ 302 │ UInt64 │ 0 │ - 304. │ 303 │ UInt64 │ 1 │ - 305. │ 304 │ UInt64 │ 0 │ - 306. │ 305 │ UInt64 │ 1 │ - 307. │ 306 │ UInt64 │ 0 │ - 308. │ 307 │ UInt64 │ 1 │ - 309. │ 308 │ UInt64 │ 0 │ - 310. │ 309 │ UInt64 │ 1 │ - 311. │ 310 │ UInt64 │ 0 │ - 312. │ 311 │ UInt64 │ 1 │ - 313. │ 312 │ UInt64 │ 0 │ - 314. │ 313 │ UInt64 │ 1 │ - 315. │ 314 │ UInt64 │ 0 │ - 316. │ 315 │ UInt64 │ 1 │ - 317. │ 316 │ UInt64 │ 0 │ - 318. │ 317 │ UInt64 │ 1 │ - 319. │ 318 │ UInt64 │ 0 │ - 320. │ 319 │ UInt64 │ 1 │ - 321. │ 320 │ UInt64 │ 0 │ - 322. │ 321 │ UInt64 │ 1 │ - 323. │ 322 │ UInt64 │ 0 │ - 324. │ 323 │ UInt64 │ 1 │ - 325. │ 324 │ UInt64 │ 0 │ - 326. │ 325 │ UInt64 │ 1 │ - 327. │ 326 │ UInt64 │ 0 │ - 328. │ 327 │ UInt64 │ 1 │ - 329. │ 328 │ UInt64 │ 0 │ - 330. │ 329 │ UInt64 │ 1 │ - 331. │ 330 │ UInt64 │ 0 │ - 332. │ 331 │ UInt64 │ 1 │ - 333. │ 332 │ UInt64 │ 0 │ - 334. │ 333 │ UInt64 │ 1 │ - 335. │ 334 │ UInt64 │ 0 │ - 336. │ 335 │ UInt64 │ 1 │ - 337. │ 336 │ UInt64 │ 0 │ - 338. │ 337 │ UInt64 │ 1 │ - 339. │ 338 │ UInt64 │ 0 │ - 340. │ 339 │ UInt64 │ 1 │ - 341. │ 340 │ UInt64 │ 0 │ - 342. │ 341 │ UInt64 │ 1 │ - 343. │ 342 │ UInt64 │ 0 │ - 344. │ 343 │ UInt64 │ 1 │ - 345. │ 344 │ UInt64 │ 0 │ - 346. │ 345 │ UInt64 │ 1 │ - 347. │ 346 │ UInt64 │ 0 │ - 348. │ 347 │ UInt64 │ 1 │ - 349. │ 348 │ UInt64 │ 0 │ - 350. │ 349 │ UInt64 │ 1 │ - 351. │ 350 │ UInt64 │ 0 │ - 352. │ 351 │ UInt64 │ 1 │ - 353. │ 352 │ UInt64 │ 0 │ - 354. │ 353 │ UInt64 │ 1 │ - 355. │ 354 │ UInt64 │ 0 │ - 356. │ 355 │ UInt64 │ 1 │ - 357. │ 356 │ UInt64 │ 0 │ - 358. │ 357 │ UInt64 │ 1 │ - 359. │ 358 │ UInt64 │ 0 │ - 360. │ 359 │ UInt64 │ 1 │ - 361. │ 360 │ UInt64 │ 0 │ - 362. │ 361 │ UInt64 │ 1 │ - 363. │ 362 │ UInt64 │ 0 │ - 364. │ 363 │ UInt64 │ 1 │ - 365. │ 364 │ UInt64 │ 0 │ - 366. │ 365 │ UInt64 │ 1 │ - 367. │ 366 │ UInt64 │ 0 │ - 368. │ 367 │ UInt64 │ 1 │ - 369. │ 368 │ UInt64 │ 0 │ - 370. │ 369 │ UInt64 │ 1 │ - 371. │ 370 │ UInt64 │ 0 │ - 372. │ 371 │ UInt64 │ 1 │ - 373. │ 372 │ UInt64 │ 0 │ - 374. │ 373 │ UInt64 │ 1 │ - 375. │ 374 │ UInt64 │ 0 │ - 376. │ 375 │ UInt64 │ 1 │ - 377. │ 376 │ UInt64 │ 0 │ - 378. │ 377 │ UInt64 │ 1 │ - 379. │ 378 │ UInt64 │ 0 │ - 380. │ 379 │ UInt64 │ 1 │ - 381. │ 380 │ UInt64 │ 0 │ - 382. │ 381 │ UInt64 │ 1 │ - 383. │ 382 │ UInt64 │ 0 │ - 384. │ 383 │ UInt64 │ 1 │ - 385. │ 384 │ UInt64 │ 0 │ - 386. │ 385 │ UInt64 │ 1 │ - 387. │ 386 │ UInt64 │ 0 │ - 388. │ 387 │ UInt64 │ 1 │ - 389. │ 388 │ UInt64 │ 0 │ - 390. │ 389 │ UInt64 │ 1 │ - 391. │ 390 │ UInt64 │ 0 │ - 392. │ 391 │ UInt64 │ 1 │ - 393. │ 392 │ UInt64 │ 0 │ - 394. │ 393 │ UInt64 │ 1 │ - 395. │ 394 │ UInt64 │ 0 │ - 396. │ 395 │ UInt64 │ 1 │ - 397. │ 396 │ UInt64 │ 0 │ - 398. │ 397 │ UInt64 │ 1 │ - 399. │ 398 │ UInt64 │ 0 │ - 400. │ 399 │ UInt64 │ 1 │ - 401. │ 400 │ UInt64 │ 0 │ - 402. │ 401 │ UInt64 │ 1 │ - 403. │ 402 │ UInt64 │ 0 │ - 404. │ 403 │ UInt64 │ 1 │ - 405. │ 404 │ UInt64 │ 0 │ - 406. │ 405 │ UInt64 │ 1 │ - 407. │ 406 │ UInt64 │ 0 │ - 408. │ 407 │ UInt64 │ 1 │ - 409. │ 408 │ UInt64 │ 0 │ - 410. │ 409 │ UInt64 │ 1 │ - 411. │ 410 │ UInt64 │ 0 │ - 412. │ 411 │ UInt64 │ 1 │ - 413. │ 412 │ UInt64 │ 0 │ - 414. │ 413 │ UInt64 │ 1 │ - 415. │ 414 │ UInt64 │ 0 │ - 416. │ 415 │ UInt64 │ 1 │ - 417. │ 416 │ UInt64 │ 0 │ - 418. │ 417 │ UInt64 │ 1 │ - 419. │ 418 │ UInt64 │ 0 │ - 420. │ 419 │ UInt64 │ 1 │ - 421. │ 420 │ UInt64 │ 0 │ - 422. │ 421 │ UInt64 │ 1 │ - 423. │ 422 │ UInt64 │ 0 │ - 424. │ 423 │ UInt64 │ 1 │ - 425. │ 424 │ UInt64 │ 0 │ - 426. │ 425 │ UInt64 │ 1 │ - 427. │ 426 │ UInt64 │ 0 │ - 428. │ 427 │ UInt64 │ 1 │ - 429. │ 428 │ UInt64 │ 0 │ - 430. │ 429 │ UInt64 │ 1 │ - 431. │ 430 │ UInt64 │ 0 │ - 432. │ 431 │ UInt64 │ 1 │ - 433. │ 432 │ UInt64 │ 0 │ - 434. │ 433 │ UInt64 │ 1 │ - 435. │ 434 │ UInt64 │ 0 │ - 436. │ 435 │ UInt64 │ 1 │ - 437. │ 436 │ UInt64 │ 0 │ - 438. │ 437 │ UInt64 │ 1 │ - 439. │ 438 │ UInt64 │ 0 │ - 440. │ 439 │ UInt64 │ 1 │ - 441. │ 440 │ UInt64 │ 0 │ - 442. │ 441 │ UInt64 │ 1 │ - 443. │ 442 │ UInt64 │ 0 │ - 444. │ 443 │ UInt64 │ 1 │ - 445. │ 444 │ UInt64 │ 0 │ - 446. │ 445 │ UInt64 │ 1 │ - 447. │ 446 │ UInt64 │ 0 │ - 448. │ 447 │ UInt64 │ 1 │ - 449. │ 448 │ UInt64 │ 0 │ - 450. │ 449 │ UInt64 │ 1 │ - 451. │ 450 │ UInt64 │ 0 │ - 452. │ 451 │ UInt64 │ 1 │ - 453. │ 452 │ UInt64 │ 0 │ - 454. │ 453 │ UInt64 │ 1 │ - 455. │ 454 │ UInt64 │ 0 │ - 456. │ 455 │ UInt64 │ 1 │ - 457. │ 456 │ UInt64 │ 0 │ - 458. │ 457 │ UInt64 │ 1 │ - 459. │ 458 │ UInt64 │ 0 │ - 460. │ 459 │ UInt64 │ 1 │ - 461. │ 460 │ UInt64 │ 0 │ - 462. │ 461 │ UInt64 │ 1 │ - 463. │ 462 │ UInt64 │ 0 │ - 464. │ 463 │ UInt64 │ 1 │ - 465. │ 464 │ UInt64 │ 0 │ - 466. │ 465 │ UInt64 │ 1 │ - 467. │ 466 │ UInt64 │ 0 │ - 468. │ 467 │ UInt64 │ 1 │ - 469. │ 468 │ UInt64 │ 0 │ - 470. │ 469 │ UInt64 │ 1 │ - 471. │ 470 │ UInt64 │ 0 │ - 472. │ 471 │ UInt64 │ 1 │ - 473. │ 472 │ UInt64 │ 0 │ - 474. │ 473 │ UInt64 │ 1 │ - 475. │ 474 │ UInt64 │ 0 │ - 476. │ 475 │ UInt64 │ 1 │ - 477. │ 476 │ UInt64 │ 0 │ - 478. │ 477 │ UInt64 │ 1 │ - 479. │ 478 │ UInt64 │ 0 │ - 480. │ 479 │ UInt64 │ 1 │ - 481. │ 480 │ UInt64 │ 0 │ - 482. │ 481 │ UInt64 │ 1 │ - 483. │ 482 │ UInt64 │ 0 │ - 484. │ 483 │ UInt64 │ 1 │ - 485. │ 484 │ UInt64 │ 0 │ - 486. │ 485 │ UInt64 │ 1 │ - 487. │ 486 │ UInt64 │ 0 │ - 488. │ 487 │ UInt64 │ 1 │ - 489. │ 488 │ UInt64 │ 0 │ - 490. │ 489 │ UInt64 │ 1 │ - 491. │ 490 │ UInt64 │ 0 │ - 492. │ 491 │ UInt64 │ 1 │ - 493. │ 492 │ UInt64 │ 0 │ - 494. │ 493 │ UInt64 │ 1 │ - 495. │ 494 │ UInt64 │ 0 │ - 496. │ 495 │ UInt64 │ 1 │ - 497. │ 496 │ UInt64 │ 0 │ - 498. │ 497 │ UInt64 │ 1 │ - 499. │ 498 │ UInt64 │ 0 │ - 500. │ 499 │ UInt64 │ 1 │ - 501. │ 500 │ UInt64 │ 0 │ - 502. │ 501 │ UInt64 │ 1 │ - 503. │ 502 │ UInt64 │ 0 │ - 504. │ 503 │ UInt64 │ 1 │ - 505. │ 504 │ UInt64 │ 0 │ - 506. │ 505 │ UInt64 │ 1 │ - 507. │ 506 │ UInt64 │ 0 │ - 508. │ 507 │ UInt64 │ 1 │ - 509. │ 508 │ UInt64 │ 0 │ - 510. │ 509 │ UInt64 │ 1 │ - 511. │ 510 │ UInt64 │ 0 │ - 512. │ 511 │ UInt64 │ 1 │ - 513. │ 512 │ UInt64 │ 0 │ - 514. │ 513 │ UInt64 │ 1 │ - 515. │ 514 │ UInt64 │ 0 │ - 516. │ 515 │ UInt64 │ 1 │ - 517. │ 516 │ UInt64 │ 0 │ - 518. │ 517 │ UInt64 │ 1 │ - 519. │ 518 │ UInt64 │ 0 │ - 520. │ 519 │ UInt64 │ 1 │ - 521. │ 520 │ UInt64 │ 0 │ - 522. │ 521 │ UInt64 │ 1 │ - 523. │ 522 │ UInt64 │ 0 │ - 524. │ 523 │ UInt64 │ 1 │ - 525. │ 524 │ UInt64 │ 0 │ - 526. │ 525 │ UInt64 │ 1 │ - 527. │ 526 │ UInt64 │ 0 │ - 528. │ 527 │ UInt64 │ 1 │ - 529. │ 528 │ UInt64 │ 0 │ - 530. │ 529 │ UInt64 │ 1 │ - 531. │ 530 │ UInt64 │ 0 │ - 532. │ 531 │ UInt64 │ 1 │ - 533. │ 532 │ UInt64 │ 0 │ - 534. │ 533 │ UInt64 │ 1 │ - 535. │ 534 │ UInt64 │ 0 │ - 536. │ 535 │ UInt64 │ 1 │ - 537. │ 536 │ UInt64 │ 0 │ - 538. │ 537 │ UInt64 │ 1 │ - 539. │ 538 │ UInt64 │ 0 │ - 540. │ 539 │ UInt64 │ 1 │ - 541. │ 540 │ UInt64 │ 0 │ - 542. │ 541 │ UInt64 │ 1 │ - 543. │ 542 │ UInt64 │ 0 │ - 544. │ 543 │ UInt64 │ 1 │ - 545. │ 544 │ UInt64 │ 0 │ - 546. │ 545 │ UInt64 │ 1 │ - 547. │ 546 │ UInt64 │ 0 │ - 548. │ 547 │ UInt64 │ 1 │ - 549. │ 548 │ UInt64 │ 0 │ - 550. │ 549 │ UInt64 │ 1 │ - 551. │ 550 │ UInt64 │ 0 │ - 552. │ 551 │ UInt64 │ 1 │ - 553. │ 552 │ UInt64 │ 0 │ - 554. │ 553 │ UInt64 │ 1 │ - 555. │ 554 │ UInt64 │ 0 │ - 556. │ 555 │ UInt64 │ 1 │ - 557. │ 556 │ UInt64 │ 0 │ - 558. │ 557 │ UInt64 │ 1 │ - 559. │ 558 │ UInt64 │ 0 │ - 560. │ 559 │ UInt64 │ 1 │ - 561. │ 560 │ UInt64 │ 0 │ - 562. │ 561 │ UInt64 │ 1 │ - 563. │ 562 │ UInt64 │ 0 │ - 564. │ 563 │ UInt64 │ 1 │ - 565. │ 564 │ UInt64 │ 0 │ - 566. │ 565 │ UInt64 │ 1 │ - 567. │ 566 │ UInt64 │ 0 │ - 568. │ 567 │ UInt64 │ 1 │ - 569. │ 568 │ UInt64 │ 0 │ - 570. │ 569 │ UInt64 │ 1 │ - 571. │ 570 │ UInt64 │ 0 │ - 572. │ 571 │ UInt64 │ 1 │ - 573. │ 572 │ UInt64 │ 0 │ - 574. │ 573 │ UInt64 │ 1 │ - 575. │ 574 │ UInt64 │ 0 │ - 576. │ 575 │ UInt64 │ 1 │ - 577. │ 576 │ UInt64 │ 0 │ - 578. │ 577 │ UInt64 │ 1 │ - 579. │ 578 │ UInt64 │ 0 │ - 580. │ 579 │ UInt64 │ 1 │ - 581. │ 580 │ UInt64 │ 0 │ - 582. │ 581 │ UInt64 │ 1 │ - 583. │ 582 │ UInt64 │ 0 │ - 584. │ 583 │ UInt64 │ 1 │ - 585. │ 584 │ UInt64 │ 0 │ - 586. │ 585 │ UInt64 │ 1 │ - 587. │ 586 │ UInt64 │ 0 │ - 588. │ 587 │ UInt64 │ 1 │ - 589. │ 588 │ UInt64 │ 0 │ - 590. │ 589 │ UInt64 │ 1 │ - 591. │ 590 │ UInt64 │ 0 │ - 592. │ 591 │ UInt64 │ 1 │ - 593. │ 592 │ UInt64 │ 0 │ - 594. │ 593 │ UInt64 │ 1 │ - 595. │ 594 │ UInt64 │ 0 │ - 596. │ 595 │ UInt64 │ 1 │ - 597. │ 596 │ UInt64 │ 0 │ - 598. │ 597 │ UInt64 │ 1 │ - 599. │ 598 │ UInt64 │ 0 │ - 600. │ 599 │ UInt64 │ 1 │ - 601. │ 600 │ UInt64 │ 0 │ - 602. │ 601 │ UInt64 │ 1 │ - 603. │ 602 │ UInt64 │ 0 │ - 604. │ 603 │ UInt64 │ 1 │ - 605. │ 604 │ UInt64 │ 0 │ - 606. │ 605 │ UInt64 │ 1 │ - 607. │ 606 │ UInt64 │ 0 │ - 608. │ 607 │ UInt64 │ 1 │ - 609. │ 608 │ UInt64 │ 0 │ - 610. │ 609 │ UInt64 │ 1 │ - 611. │ 610 │ UInt64 │ 0 │ - 612. │ 611 │ UInt64 │ 1 │ - 613. │ 612 │ UInt64 │ 0 │ - 614. │ 613 │ UInt64 │ 1 │ - 615. │ 614 │ UInt64 │ 0 │ - 616. │ 615 │ UInt64 │ 1 │ - 617. │ 616 │ UInt64 │ 0 │ - 618. │ 617 │ UInt64 │ 1 │ - 619. │ 618 │ UInt64 │ 0 │ - 620. │ 619 │ UInt64 │ 1 │ - 621. │ 620 │ UInt64 │ 0 │ - 622. │ 621 │ UInt64 │ 1 │ - 623. │ 622 │ UInt64 │ 0 │ - 624. │ 623 │ UInt64 │ 1 │ - 625. │ 624 │ UInt64 │ 0 │ - 626. │ 625 │ UInt64 │ 1 │ - 627. │ 626 │ UInt64 │ 0 │ - 628. │ 627 │ UInt64 │ 1 │ - 629. │ 628 │ UInt64 │ 0 │ - 630. │ 629 │ UInt64 │ 1 │ - 631. │ 630 │ UInt64 │ 0 │ - 632. │ 631 │ UInt64 │ 1 │ - 633. │ 632 │ UInt64 │ 0 │ - 634. │ 633 │ UInt64 │ 1 │ - 635. │ 634 │ UInt64 │ 0 │ - 636. │ 635 │ UInt64 │ 1 │ - 637. │ 636 │ UInt64 │ 0 │ - 638. │ 637 │ UInt64 │ 1 │ - 639. │ 638 │ UInt64 │ 0 │ - 640. │ 639 │ UInt64 │ 1 │ - 641. │ 640 │ UInt64 │ 0 │ - 642. │ 641 │ UInt64 │ 1 │ - 643. │ 642 │ UInt64 │ 0 │ - 644. │ 643 │ UInt64 │ 1 │ - 645. │ 644 │ UInt64 │ 0 │ - 646. │ 645 │ UInt64 │ 1 │ - 647. │ 646 │ UInt64 │ 0 │ - 648. │ 647 │ UInt64 │ 1 │ - 649. │ 648 │ UInt64 │ 0 │ - 650. │ 649 │ UInt64 │ 1 │ - 651. │ 650 │ UInt64 │ 0 │ - 652. │ 651 │ UInt64 │ 1 │ - 653. │ 652 │ UInt64 │ 0 │ - 654. │ 653 │ UInt64 │ 1 │ - 655. │ 654 │ UInt64 │ 0 │ - 656. │ 655 │ UInt64 │ 1 │ - 657. │ 656 │ UInt64 │ 0 │ - 658. │ 657 │ UInt64 │ 1 │ - 659. │ 658 │ UInt64 │ 0 │ - 660. │ 659 │ UInt64 │ 1 │ - 661. │ 660 │ UInt64 │ 0 │ - 662. │ 661 │ UInt64 │ 1 │ - 663. │ 662 │ UInt64 │ 0 │ - 664. │ 663 │ UInt64 │ 1 │ - 665. │ 664 │ UInt64 │ 0 │ - 666. │ 665 │ UInt64 │ 1 │ - 667. │ 666 │ UInt64 │ 0 │ - 668. │ 667 │ UInt64 │ 1 │ - 669. │ 668 │ UInt64 │ 0 │ - 670. │ 669 │ UInt64 │ 1 │ - 671. │ 670 │ UInt64 │ 0 │ - 672. │ 671 │ UInt64 │ 1 │ - 673. │ 672 │ UInt64 │ 0 │ - 674. │ 673 │ UInt64 │ 1 │ - 675. │ 674 │ UInt64 │ 0 │ - 676. │ 675 │ UInt64 │ 1 │ - 677. │ 676 │ UInt64 │ 0 │ - 678. │ 677 │ UInt64 │ 1 │ - 679. │ 678 │ UInt64 │ 0 │ - 680. │ 679 │ UInt64 │ 1 │ - 681. │ 680 │ UInt64 │ 0 │ - 682. │ 681 │ UInt64 │ 1 │ - 683. │ 682 │ UInt64 │ 0 │ - 684. │ 683 │ UInt64 │ 1 │ - 685. │ 684 │ UInt64 │ 0 │ - 686. │ 685 │ UInt64 │ 1 │ - 687. │ 686 │ UInt64 │ 0 │ - 688. │ 687 │ UInt64 │ 1 │ - 689. │ 688 │ UInt64 │ 0 │ - 690. │ 689 │ UInt64 │ 1 │ - 691. │ 690 │ UInt64 │ 0 │ - 692. │ 691 │ UInt64 │ 1 │ - 693. │ 692 │ UInt64 │ 0 │ - 694. │ 693 │ UInt64 │ 1 │ - 695. │ 694 │ UInt64 │ 0 │ - 696. │ 695 │ UInt64 │ 1 │ - 697. │ 696 │ UInt64 │ 0 │ - 698. │ 697 │ UInt64 │ 1 │ - 699. │ 698 │ UInt64 │ 0 │ - 700. │ 699 │ UInt64 │ 1 │ - 701. │ 700 │ UInt64 │ 0 │ - 702. │ 701 │ UInt64 │ 1 │ - 703. │ 702 │ UInt64 │ 0 │ - 704. │ 703 │ UInt64 │ 1 │ - 705. │ 704 │ UInt64 │ 0 │ - 706. │ 705 │ UInt64 │ 1 │ - 707. │ 706 │ UInt64 │ 0 │ - 708. │ 707 │ UInt64 │ 1 │ - 709. │ 708 │ UInt64 │ 0 │ - 710. │ 709 │ UInt64 │ 1 │ - 711. │ 710 │ UInt64 │ 0 │ - 712. │ 711 │ UInt64 │ 1 │ - 713. │ 712 │ UInt64 │ 0 │ - 714. │ 713 │ UInt64 │ 1 │ - 715. │ 714 │ UInt64 │ 0 │ - 716. │ 715 │ UInt64 │ 1 │ - 717. │ 716 │ UInt64 │ 0 │ - 718. │ 717 │ UInt64 │ 1 │ - 719. │ 718 │ UInt64 │ 0 │ - 720. │ 719 │ UInt64 │ 1 │ - 721. │ 720 │ UInt64 │ 0 │ - 722. │ 721 │ UInt64 │ 1 │ - 723. │ 722 │ UInt64 │ 0 │ - 724. │ 723 │ UInt64 │ 1 │ - 725. │ 724 │ UInt64 │ 0 │ - 726. │ 725 │ UInt64 │ 1 │ - 727. │ 726 │ UInt64 │ 0 │ - 728. │ 727 │ UInt64 │ 1 │ - 729. │ 728 │ UInt64 │ 0 │ - 730. │ 729 │ UInt64 │ 1 │ - 731. │ 730 │ UInt64 │ 0 │ - 732. │ 731 │ UInt64 │ 1 │ - 733. │ 732 │ UInt64 │ 0 │ - 734. │ 733 │ UInt64 │ 1 │ - 735. │ 734 │ UInt64 │ 0 │ - 736. │ 735 │ UInt64 │ 1 │ - 737. │ 736 │ UInt64 │ 0 │ - 738. │ 737 │ UInt64 │ 1 │ - 739. │ 738 │ UInt64 │ 0 │ - 740. │ 739 │ UInt64 │ 1 │ - 741. │ 740 │ UInt64 │ 0 │ - 742. │ 741 │ UInt64 │ 1 │ - 743. │ 742 │ UInt64 │ 0 │ - 744. │ 743 │ UInt64 │ 1 │ - 745. │ 744 │ UInt64 │ 0 │ - 746. │ 745 │ UInt64 │ 1 │ - 747. │ 746 │ UInt64 │ 0 │ - 748. │ 747 │ UInt64 │ 1 │ - 749. │ 748 │ UInt64 │ 0 │ - 750. │ 749 │ UInt64 │ 1 │ - 751. │ 750 │ UInt64 │ 0 │ - 752. │ 751 │ UInt64 │ 1 │ - 753. │ 752 │ UInt64 │ 0 │ - 754. │ 753 │ UInt64 │ 1 │ - 755. │ 754 │ UInt64 │ 0 │ - 756. │ 755 │ UInt64 │ 1 │ - 757. │ 756 │ UInt64 │ 0 │ - 758. │ 757 │ UInt64 │ 1 │ - 759. │ 758 │ UInt64 │ 0 │ - 760. │ 759 │ UInt64 │ 1 │ - 761. │ 760 │ UInt64 │ 0 │ - 762. │ 761 │ UInt64 │ 1 │ - 763. │ 762 │ UInt64 │ 0 │ - 764. │ 763 │ UInt64 │ 1 │ - 765. │ 764 │ UInt64 │ 0 │ - 766. │ 765 │ UInt64 │ 1 │ - 767. │ 766 │ UInt64 │ 0 │ - 768. │ 767 │ UInt64 │ 1 │ - 769. │ 768 │ UInt64 │ 0 │ - 770. │ 769 │ UInt64 │ 1 │ - 771. │ 770 │ UInt64 │ 0 │ - 772. │ 771 │ UInt64 │ 1 │ - 773. │ 772 │ UInt64 │ 0 │ - 774. │ 773 │ UInt64 │ 1 │ - 775. │ 774 │ UInt64 │ 0 │ - 776. │ 775 │ UInt64 │ 1 │ - 777. │ 776 │ UInt64 │ 0 │ - 778. │ 777 │ UInt64 │ 1 │ - 779. │ 778 │ UInt64 │ 0 │ - 780. │ 779 │ UInt64 │ 1 │ - 781. │ 780 │ UInt64 │ 0 │ - 782. │ 781 │ UInt64 │ 1 │ - 783. │ 782 │ UInt64 │ 0 │ - 784. │ 783 │ UInt64 │ 1 │ - 785. │ 784 │ UInt64 │ 0 │ - 786. │ 785 │ UInt64 │ 1 │ - 787. │ 786 │ UInt64 │ 0 │ - 788. │ 787 │ UInt64 │ 1 │ - 789. │ 788 │ UInt64 │ 0 │ - 790. │ 789 │ UInt64 │ 1 │ - 791. │ 790 │ UInt64 │ 0 │ - 792. │ 791 │ UInt64 │ 1 │ - 793. │ 792 │ UInt64 │ 0 │ - 794. │ 793 │ UInt64 │ 1 │ - 795. │ 794 │ UInt64 │ 0 │ - 796. │ 795 │ UInt64 │ 1 │ - 797. │ 796 │ UInt64 │ 0 │ - 798. │ 797 │ UInt64 │ 1 │ - 799. │ 798 │ UInt64 │ 0 │ - 800. │ 799 │ UInt64 │ 1 │ - 801. │ 800 │ UInt64 │ 0 │ - 802. │ 801 │ UInt64 │ 1 │ - 803. │ 802 │ UInt64 │ 0 │ - 804. │ 803 │ UInt64 │ 1 │ - 805. │ 804 │ UInt64 │ 0 │ - 806. │ 805 │ UInt64 │ 1 │ - 807. │ 806 │ UInt64 │ 0 │ - 808. │ 807 │ UInt64 │ 1 │ - 809. │ 808 │ UInt64 │ 0 │ - 810. │ 809 │ UInt64 │ 1 │ - 811. │ 810 │ UInt64 │ 0 │ - 812. │ 811 │ UInt64 │ 1 │ - 813. │ 812 │ UInt64 │ 0 │ - 814. │ 813 │ UInt64 │ 1 │ - 815. │ 814 │ UInt64 │ 0 │ - 816. │ 815 │ UInt64 │ 1 │ - 817. │ 816 │ UInt64 │ 0 │ - 818. │ 817 │ UInt64 │ 1 │ - 819. │ 818 │ UInt64 │ 0 │ - 820. │ 819 │ UInt64 │ 1 │ - 821. │ 820 │ UInt64 │ 0 │ - 822. │ 821 │ UInt64 │ 1 │ - 823. │ 822 │ UInt64 │ 0 │ - 824. │ 823 │ UInt64 │ 1 │ - 825. │ 824 │ UInt64 │ 0 │ - 826. │ 825 │ UInt64 │ 1 │ - 827. │ 826 │ UInt64 │ 0 │ - 828. │ 827 │ UInt64 │ 1 │ - 829. │ 828 │ UInt64 │ 0 │ - 830. │ 829 │ UInt64 │ 1 │ - 831. │ 830 │ UInt64 │ 0 │ - 832. │ 831 │ UInt64 │ 1 │ - 833. │ 832 │ UInt64 │ 0 │ - 834. │ 833 │ UInt64 │ 1 │ - 835. │ 834 │ UInt64 │ 0 │ - 836. │ 835 │ UInt64 │ 1 │ - 837. │ 836 │ UInt64 │ 0 │ - 838. │ 837 │ UInt64 │ 1 │ - 839. │ 838 │ UInt64 │ 0 │ - 840. │ 839 │ UInt64 │ 1 │ - 841. │ 840 │ UInt64 │ 0 │ - 842. │ 841 │ UInt64 │ 1 │ - 843. │ 842 │ UInt64 │ 0 │ - 844. │ 843 │ UInt64 │ 1 │ - 845. │ 844 │ UInt64 │ 0 │ - 846. │ 845 │ UInt64 │ 1 │ - 847. │ 846 │ UInt64 │ 0 │ - 848. │ 847 │ UInt64 │ 1 │ - 849. │ 848 │ UInt64 │ 0 │ - 850. │ 849 │ UInt64 │ 1 │ - 851. │ 850 │ UInt64 │ 0 │ - 852. │ 851 │ UInt64 │ 1 │ - 853. │ 852 │ UInt64 │ 0 │ - 854. │ 853 │ UInt64 │ 1 │ - 855. │ 854 │ UInt64 │ 0 │ - 856. │ 855 │ UInt64 │ 1 │ - 857. │ 856 │ UInt64 │ 0 │ - 858. │ 857 │ UInt64 │ 1 │ - 859. │ 858 │ UInt64 │ 0 │ - 860. │ 859 │ UInt64 │ 1 │ - 861. │ 860 │ UInt64 │ 0 │ - 862. │ 861 │ UInt64 │ 1 │ - 863. │ 862 │ UInt64 │ 0 │ - 864. │ 863 │ UInt64 │ 1 │ - 865. │ 864 │ UInt64 │ 0 │ - 866. │ 865 │ UInt64 │ 1 │ - 867. │ 866 │ UInt64 │ 0 │ - 868. │ 867 │ UInt64 │ 1 │ - 869. │ 868 │ UInt64 │ 0 │ - 870. │ 869 │ UInt64 │ 1 │ - 871. │ 870 │ UInt64 │ 0 │ - 872. │ 871 │ UInt64 │ 1 │ - 873. │ 872 │ UInt64 │ 0 │ - 874. │ 873 │ UInt64 │ 1 │ - 875. │ 874 │ UInt64 │ 0 │ - 876. │ 875 │ UInt64 │ 1 │ - 877. │ 876 │ UInt64 │ 0 │ - 878. │ 877 │ UInt64 │ 1 │ - 879. │ 878 │ UInt64 │ 0 │ - 880. │ 879 │ UInt64 │ 1 │ - 881. │ 880 │ UInt64 │ 0 │ - 882. │ 881 │ UInt64 │ 1 │ - 883. │ 882 │ UInt64 │ 0 │ - 884. │ 883 │ UInt64 │ 1 │ - 885. │ 884 │ UInt64 │ 0 │ - 886. │ 885 │ UInt64 │ 1 │ - 887. │ 886 │ UInt64 │ 0 │ - 888. │ 887 │ UInt64 │ 1 │ - 889. │ 888 │ UInt64 │ 0 │ - 890. │ 889 │ UInt64 │ 1 │ - 891. │ 890 │ UInt64 │ 0 │ - 892. │ 891 │ UInt64 │ 1 │ - 893. │ 892 │ UInt64 │ 0 │ - 894. │ 893 │ UInt64 │ 1 │ - 895. │ 894 │ UInt64 │ 0 │ - 896. │ 895 │ UInt64 │ 1 │ - 897. │ 896 │ UInt64 │ 0 │ - 898. │ 897 │ UInt64 │ 1 │ - 899. │ 898 │ UInt64 │ 0 │ - 900. │ 899 │ UInt64 │ 1 │ - 901. │ 900 │ UInt64 │ 0 │ - 902. │ 901 │ UInt64 │ 1 │ - 903. │ 902 │ UInt64 │ 0 │ - 904. │ 903 │ UInt64 │ 1 │ - 905. │ 904 │ UInt64 │ 0 │ - 906. │ 905 │ UInt64 │ 1 │ - 907. │ 906 │ UInt64 │ 0 │ - 908. │ 907 │ UInt64 │ 1 │ - 909. │ 908 │ UInt64 │ 0 │ - 910. │ 909 │ UInt64 │ 1 │ - 911. │ 910 │ UInt64 │ 0 │ - 912. │ 911 │ UInt64 │ 1 │ - 913. │ 912 │ UInt64 │ 0 │ - 914. │ 913 │ UInt64 │ 1 │ - 915. │ 914 │ UInt64 │ 0 │ - 916. │ 915 │ UInt64 │ 1 │ - 917. │ 916 │ UInt64 │ 0 │ - 918. │ 917 │ UInt64 │ 1 │ - 919. │ 918 │ UInt64 │ 0 │ - 920. │ 919 │ UInt64 │ 1 │ - 921. │ 920 │ UInt64 │ 0 │ - 922. │ 921 │ UInt64 │ 1 │ - 923. │ 922 │ UInt64 │ 0 │ - 924. │ 923 │ UInt64 │ 1 │ - 925. │ 924 │ UInt64 │ 0 │ - 926. │ 925 │ UInt64 │ 1 │ - 927. │ 926 │ UInt64 │ 0 │ - 928. │ 927 │ UInt64 │ 1 │ - 929. │ 928 │ UInt64 │ 0 │ - 930. │ 929 │ UInt64 │ 1 │ - 931. │ 930 │ UInt64 │ 0 │ - 932. │ 931 │ UInt64 │ 1 │ - 933. │ 932 │ UInt64 │ 0 │ - 934. │ 933 │ UInt64 │ 1 │ - 935. │ 934 │ UInt64 │ 0 │ - 936. │ 935 │ UInt64 │ 1 │ - 937. │ 936 │ UInt64 │ 0 │ - 938. │ 937 │ UInt64 │ 1 │ - 939. │ 938 │ UInt64 │ 0 │ - 940. │ 939 │ UInt64 │ 1 │ - 941. │ 940 │ UInt64 │ 0 │ - 942. │ 941 │ UInt64 │ 1 │ - 943. │ 942 │ UInt64 │ 0 │ - 944. │ 943 │ UInt64 │ 1 │ - 945. │ 944 │ UInt64 │ 0 │ - 946. │ 945 │ UInt64 │ 1 │ - 947. │ 946 │ UInt64 │ 0 │ - 948. │ 947 │ UInt64 │ 1 │ - 949. │ 948 │ UInt64 │ 0 │ - 950. │ 949 │ UInt64 │ 1 │ - 951. │ 950 │ UInt64 │ 0 │ - 952. │ 951 │ UInt64 │ 1 │ - 953. │ 952 │ UInt64 │ 0 │ - 954. │ 953 │ UInt64 │ 1 │ - 955. │ 954 │ UInt64 │ 0 │ - 956. │ 955 │ UInt64 │ 1 │ - 957. │ 956 │ UInt64 │ 0 │ - 958. │ 957 │ UInt64 │ 1 │ - 959. │ 958 │ UInt64 │ 0 │ - 960. │ 959 │ UInt64 │ 1 │ - 961. │ 960 │ UInt64 │ 0 │ - 962. │ 961 │ UInt64 │ 1 │ - 963. │ 962 │ UInt64 │ 0 │ - 964. │ 963 │ UInt64 │ 1 │ - 965. │ 964 │ UInt64 │ 0 │ - 966. │ 965 │ UInt64 │ 1 │ - 967. │ 966 │ UInt64 │ 0 │ - 968. │ 967 │ UInt64 │ 1 │ - 969. │ 968 │ UInt64 │ 0 │ - 970. │ 969 │ UInt64 │ 1 │ - 971. │ 970 │ UInt64 │ 0 │ - 972. │ 971 │ UInt64 │ 1 │ - 973. │ 972 │ UInt64 │ 0 │ - 974. │ 973 │ UInt64 │ 1 │ - 975. │ 974 │ UInt64 │ 0 │ - 976. │ 975 │ UInt64 │ 1 │ - 977. │ 976 │ UInt64 │ 0 │ - 978. │ 977 │ UInt64 │ 1 │ - 979. │ 978 │ UInt64 │ 0 │ - 980. │ 979 │ UInt64 │ 1 │ - 981. │ 980 │ UInt64 │ 0 │ - 982. │ 981 │ UInt64 │ 1 │ - 983. │ 982 │ UInt64 │ 0 │ - 984. │ 983 │ UInt64 │ 1 │ - 985. │ 984 │ UInt64 │ 0 │ - 986. │ 985 │ UInt64 │ 1 │ - 987. │ 986 │ UInt64 │ 0 │ - 988. │ 987 │ UInt64 │ 1 │ - 989. │ 988 │ UInt64 │ 0 │ - 990. │ 989 │ UInt64 │ 1 │ - 991. │ 990 │ UInt64 │ 0 │ - 992. │ 991 │ UInt64 │ 1 │ - 993. │ 992 │ UInt64 │ 0 │ - 994. │ 993 │ UInt64 │ 1 │ - 995. │ 994 │ UInt64 │ 0 │ - 996. │ 995 │ UInt64 │ 1 │ - 997. │ 996 │ UInt64 │ 0 │ - 998. │ 997 │ UInt64 │ 1 │ - 999. │ 998 │ UInt64 │ 0 │ -1000. │ 999 │ UInt64 │ 1 │ - └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ - 100. │ 99 │ UInt64 │ 1 │ - 101. │ 100 │ UInt64 │ 0 │ - 102. │ 101 │ UInt64 │ 1 │ - 103. │ 102 │ UInt64 │ 0 │ - 104. │ 103 │ UInt64 │ 1 │ - 105. │ 104 │ UInt64 │ 0 │ - 106. │ 105 │ UInt64 │ 1 │ - 107. │ 106 │ UInt64 │ 0 │ - 108. │ 107 │ UInt64 │ 1 │ - 109. │ 108 │ UInt64 │ 0 │ - 110. │ 109 │ UInt64 │ 1 │ - 111. │ 110 │ UInt64 │ 0 │ - 112. │ 111 │ UInt64 │ 1 │ - 113. │ 112 │ UInt64 │ 0 │ - 114. │ 113 │ UInt64 │ 1 │ - 115. │ 114 │ UInt64 │ 0 │ - 116. │ 115 │ UInt64 │ 1 │ - 117. │ 116 │ UInt64 │ 0 │ - 118. │ 117 │ UInt64 │ 1 │ - 119. │ 118 │ UInt64 │ 0 │ - 120. │ 119 │ UInt64 │ 1 │ - 121. │ 120 │ UInt64 │ 0 │ - 122. │ 121 │ UInt64 │ 1 │ - 123. │ 122 │ UInt64 │ 0 │ - 124. │ 123 │ UInt64 │ 1 │ - 125. │ 124 │ UInt64 │ 0 │ - 126. │ 125 │ UInt64 │ 1 │ - 127. │ 126 │ UInt64 │ 0 │ - 128. │ 127 │ UInt64 │ 1 │ - 129. │ 128 │ UInt64 │ 0 │ - 130. │ 129 │ UInt64 │ 1 │ - 131. │ 130 │ UInt64 │ 0 │ - 132. │ 131 │ UInt64 │ 1 │ - 133. │ 132 │ UInt64 │ 0 │ - 134. │ 133 │ UInt64 │ 1 │ - 135. │ 134 │ UInt64 │ 0 │ - 136. │ 135 │ UInt64 │ 1 │ - 137. │ 136 │ UInt64 │ 0 │ - 138. │ 137 │ UInt64 │ 1 │ - 139. │ 138 │ UInt64 │ 0 │ - 140. │ 139 │ UInt64 │ 1 │ - 141. │ 140 │ UInt64 │ 0 │ - 142. │ 141 │ UInt64 │ 1 │ - 143. │ 142 │ UInt64 │ 0 │ - 144. │ 143 │ UInt64 │ 1 │ - 145. │ 144 │ UInt64 │ 0 │ - 146. │ 145 │ UInt64 │ 1 │ - 147. │ 146 │ UInt64 │ 0 │ - 148. │ 147 │ UInt64 │ 1 │ - 149. │ 148 │ UInt64 │ 0 │ - 150. │ 149 │ UInt64 │ 1 │ - 151. │ 150 │ UInt64 │ 0 │ - 152. │ 151 │ UInt64 │ 1 │ - 153. │ 152 │ UInt64 │ 0 │ - 154. │ 153 │ UInt64 │ 1 │ - 155. │ 154 │ UInt64 │ 0 │ - 156. │ 155 │ UInt64 │ 1 │ - 157. │ 156 │ UInt64 │ 0 │ - 158. │ 157 │ UInt64 │ 1 │ - 159. │ 158 │ UInt64 │ 0 │ - 160. │ 159 │ UInt64 │ 1 │ - 161. │ 160 │ UInt64 │ 0 │ - 162. │ 161 │ UInt64 │ 1 │ - 163. │ 162 │ UInt64 │ 0 │ - 164. │ 163 │ UInt64 │ 1 │ - 165. │ 164 │ UInt64 │ 0 │ - 166. │ 165 │ UInt64 │ 1 │ - 167. │ 166 │ UInt64 │ 0 │ - 168. │ 167 │ UInt64 │ 1 │ - 169. │ 168 │ UInt64 │ 0 │ - 170. │ 169 │ UInt64 │ 1 │ - 171. │ 170 │ UInt64 │ 0 │ - 172. │ 171 │ UInt64 │ 1 │ - 173. │ 172 │ UInt64 │ 0 │ - 174. │ 173 │ UInt64 │ 1 │ - 175. │ 174 │ UInt64 │ 0 │ - 176. │ 175 │ UInt64 │ 1 │ - 177. │ 176 │ UInt64 │ 0 │ - 178. │ 177 │ UInt64 │ 1 │ - 179. │ 178 │ UInt64 │ 0 │ - 180. │ 179 │ UInt64 │ 1 │ - 181. │ 180 │ UInt64 │ 0 │ - 182. │ 181 │ UInt64 │ 1 │ - 183. │ 182 │ UInt64 │ 0 │ - 184. │ 183 │ UInt64 │ 1 │ - 185. │ 184 │ UInt64 │ 0 │ - 186. │ 185 │ UInt64 │ 1 │ - 187. │ 186 │ UInt64 │ 0 │ - 188. │ 187 │ UInt64 │ 1 │ - 189. │ 188 │ UInt64 │ 0 │ - 190. │ 189 │ UInt64 │ 1 │ - 191. │ 190 │ UInt64 │ 0 │ - 192. │ 191 │ UInt64 │ 1 │ - 193. │ 192 │ UInt64 │ 0 │ - 194. │ 193 │ UInt64 │ 1 │ - 195. │ 194 │ UInt64 │ 0 │ - 196. │ 195 │ UInt64 │ 1 │ - 197. │ 196 │ UInt64 │ 0 │ - 198. │ 197 │ UInt64 │ 1 │ - 199. │ 198 │ UInt64 │ 0 │ - 200. │ 199 │ UInt64 │ 1 │ - 201. │ 200 │ UInt64 │ 0 │ - 202. │ 201 │ UInt64 │ 1 │ - 203. │ 202 │ UInt64 │ 0 │ - 204. │ 203 │ UInt64 │ 1 │ - 205. │ 204 │ UInt64 │ 0 │ - 206. │ 205 │ UInt64 │ 1 │ - 207. │ 206 │ UInt64 │ 0 │ - 208. │ 207 │ UInt64 │ 1 │ - 209. │ 208 │ UInt64 │ 0 │ - 210. │ 209 │ UInt64 │ 1 │ - 211. │ 210 │ UInt64 │ 0 │ - 212. │ 211 │ UInt64 │ 1 │ - 213. │ 212 │ UInt64 │ 0 │ - 214. │ 213 │ UInt64 │ 1 │ - 215. │ 214 │ UInt64 │ 0 │ - 216. │ 215 │ UInt64 │ 1 │ - 217. │ 216 │ UInt64 │ 0 │ - 218. │ 217 │ UInt64 │ 1 │ - 219. │ 218 │ UInt64 │ 0 │ - 220. │ 219 │ UInt64 │ 1 │ - 221. │ 220 │ UInt64 │ 0 │ - 222. │ 221 │ UInt64 │ 1 │ - 223. │ 222 │ UInt64 │ 0 │ - 224. │ 223 │ UInt64 │ 1 │ - 225. │ 224 │ UInt64 │ 0 │ - 226. │ 225 │ UInt64 │ 1 │ - 227. │ 226 │ UInt64 │ 0 │ - 228. │ 227 │ UInt64 │ 1 │ - 229. │ 228 │ UInt64 │ 0 │ - 230. │ 229 │ UInt64 │ 1 │ - 231. │ 230 │ UInt64 │ 0 │ - 232. │ 231 │ UInt64 │ 1 │ - 233. │ 232 │ UInt64 │ 0 │ - 234. │ 233 │ UInt64 │ 1 │ - 235. │ 234 │ UInt64 │ 0 │ - 236. │ 235 │ UInt64 │ 1 │ - 237. │ 236 │ UInt64 │ 0 │ - 238. │ 237 │ UInt64 │ 1 │ - 239. │ 238 │ UInt64 │ 0 │ - 240. │ 239 │ UInt64 │ 1 │ - 241. │ 240 │ UInt64 │ 0 │ - 242. │ 241 │ UInt64 │ 1 │ - 243. │ 242 │ UInt64 │ 0 │ - 244. │ 243 │ UInt64 │ 1 │ - 245. │ 244 │ UInt64 │ 0 │ - 246. │ 245 │ UInt64 │ 1 │ - 247. │ 246 │ UInt64 │ 0 │ - 248. │ 247 │ UInt64 │ 1 │ - 249. │ 248 │ UInt64 │ 0 │ - 250. │ 249 │ UInt64 │ 1 │ - 251. │ 250 │ UInt64 │ 0 │ - 252. │ 251 │ UInt64 │ 1 │ - 253. │ 252 │ UInt64 │ 0 │ - 254. │ 253 │ UInt64 │ 1 │ - 255. │ 254 │ UInt64 │ 0 │ - 256. │ 255 │ UInt64 │ 1 │ - 257. │ 256 │ UInt64 │ 0 │ - 258. │ 257 │ UInt64 │ 1 │ - 259. │ 258 │ UInt64 │ 0 │ - 260. │ 259 │ UInt64 │ 1 │ - 261. │ 260 │ UInt64 │ 0 │ - 262. │ 261 │ UInt64 │ 1 │ - 263. │ 262 │ UInt64 │ 0 │ - 264. │ 263 │ UInt64 │ 1 │ - 265. │ 264 │ UInt64 │ 0 │ - 266. │ 265 │ UInt64 │ 1 │ - 267. │ 266 │ UInt64 │ 0 │ - 268. │ 267 │ UInt64 │ 1 │ - 269. │ 268 │ UInt64 │ 0 │ - 270. │ 269 │ UInt64 │ 1 │ - 271. │ 270 │ UInt64 │ 0 │ - 272. │ 271 │ UInt64 │ 1 │ - 273. │ 272 │ UInt64 │ 0 │ - 274. │ 273 │ UInt64 │ 1 │ - 275. │ 274 │ UInt64 │ 0 │ - 276. │ 275 │ UInt64 │ 1 │ - 277. │ 276 │ UInt64 │ 0 │ - 278. │ 277 │ UInt64 │ 1 │ - 279. │ 278 │ UInt64 │ 0 │ - 280. │ 279 │ UInt64 │ 1 │ - 281. │ 280 │ UInt64 │ 0 │ - 282. │ 281 │ UInt64 │ 1 │ - 283. │ 282 │ UInt64 │ 0 │ - 284. │ 283 │ UInt64 │ 1 │ - 285. │ 284 │ UInt64 │ 0 │ - 286. │ 285 │ UInt64 │ 1 │ - 287. │ 286 │ UInt64 │ 0 │ - 288. │ 287 │ UInt64 │ 1 │ - 289. │ 288 │ UInt64 │ 0 │ - 290. │ 289 │ UInt64 │ 1 │ - 291. │ 290 │ UInt64 │ 0 │ - 292. │ 291 │ UInt64 │ 1 │ - 293. │ 292 │ UInt64 │ 0 │ - 294. │ 293 │ UInt64 │ 1 │ - 295. │ 294 │ UInt64 │ 0 │ - 296. │ 295 │ UInt64 │ 1 │ - 297. │ 296 │ UInt64 │ 0 │ - 298. │ 297 │ UInt64 │ 1 │ - 299. │ 298 │ UInt64 │ 0 │ - 300. │ 299 │ UInt64 │ 1 │ - 301. │ 300 │ UInt64 │ 0 │ - 302. │ 301 │ UInt64 │ 1 │ - 303. │ 302 │ UInt64 │ 0 │ - 304. │ 303 │ UInt64 │ 1 │ - 305. │ 304 │ UInt64 │ 0 │ - 306. │ 305 │ UInt64 │ 1 │ - 307. │ 306 │ UInt64 │ 0 │ - 308. │ 307 │ UInt64 │ 1 │ - 309. │ 308 │ UInt64 │ 0 │ - 310. │ 309 │ UInt64 │ 1 │ - 311. │ 310 │ UInt64 │ 0 │ - 312. │ 311 │ UInt64 │ 1 │ - 313. │ 312 │ UInt64 │ 0 │ - 314. │ 313 │ UInt64 │ 1 │ - 315. │ 314 │ UInt64 │ 0 │ - 316. │ 315 │ UInt64 │ 1 │ - 317. │ 316 │ UInt64 │ 0 │ - 318. │ 317 │ UInt64 │ 1 │ - 319. │ 318 │ UInt64 │ 0 │ - 320. │ 319 │ UInt64 │ 1 │ - 321. │ 320 │ UInt64 │ 0 │ - 322. │ 321 │ UInt64 │ 1 │ - 323. │ 322 │ UInt64 │ 0 │ - 324. │ 323 │ UInt64 │ 1 │ - 325. │ 324 │ UInt64 │ 0 │ - 326. │ 325 │ UInt64 │ 1 │ - 327. │ 326 │ UInt64 │ 0 │ - 328. │ 327 │ UInt64 │ 1 │ - 329. │ 328 │ UInt64 │ 0 │ - 330. │ 329 │ UInt64 │ 1 │ - 331. │ 330 │ UInt64 │ 0 │ - 332. │ 331 │ UInt64 │ 1 │ - 333. │ 332 │ UInt64 │ 0 │ - 334. │ 333 │ UInt64 │ 1 │ - 335. │ 334 │ UInt64 │ 0 │ - 336. │ 335 │ UInt64 │ 1 │ - 337. │ 336 │ UInt64 │ 0 │ - 338. │ 337 │ UInt64 │ 1 │ - 339. │ 338 │ UInt64 │ 0 │ - 340. │ 339 │ UInt64 │ 1 │ - 341. │ 340 │ UInt64 │ 0 │ - 342. │ 341 │ UInt64 │ 1 │ - 343. │ 342 │ UInt64 │ 0 │ - 344. │ 343 │ UInt64 │ 1 │ - 345. │ 344 │ UInt64 │ 0 │ - 346. │ 345 │ UInt64 │ 1 │ - 347. │ 346 │ UInt64 │ 0 │ - 348. │ 347 │ UInt64 │ 1 │ - 349. │ 348 │ UInt64 │ 0 │ - 350. │ 349 │ UInt64 │ 1 │ - 351. │ 350 │ UInt64 │ 0 │ - 352. │ 351 │ UInt64 │ 1 │ - 353. │ 352 │ UInt64 │ 0 │ - 354. │ 353 │ UInt64 │ 1 │ - 355. │ 354 │ UInt64 │ 0 │ - 356. │ 355 │ UInt64 │ 1 │ - 357. │ 356 │ UInt64 │ 0 │ - 358. │ 357 │ UInt64 │ 1 │ - 359. │ 358 │ UInt64 │ 0 │ - 360. │ 359 │ UInt64 │ 1 │ - 361. │ 360 │ UInt64 │ 0 │ - 362. │ 361 │ UInt64 │ 1 │ - 363. │ 362 │ UInt64 │ 0 │ - 364. │ 363 │ UInt64 │ 1 │ - 365. │ 364 │ UInt64 │ 0 │ - 366. │ 365 │ UInt64 │ 1 │ - 367. │ 366 │ UInt64 │ 0 │ - 368. │ 367 │ UInt64 │ 1 │ - 369. │ 368 │ UInt64 │ 0 │ - 370. │ 369 │ UInt64 │ 1 │ - 371. │ 370 │ UInt64 │ 0 │ - 372. │ 371 │ UInt64 │ 1 │ - 373. │ 372 │ UInt64 │ 0 │ - 374. │ 373 │ UInt64 │ 1 │ - 375. │ 374 │ UInt64 │ 0 │ - 376. │ 375 │ UInt64 │ 1 │ - 377. │ 376 │ UInt64 │ 0 │ - 378. │ 377 │ UInt64 │ 1 │ - 379. │ 378 │ UInt64 │ 0 │ - 380. │ 379 │ UInt64 │ 1 │ - 381. │ 380 │ UInt64 │ 0 │ - 382. │ 381 │ UInt64 │ 1 │ - 383. │ 382 │ UInt64 │ 0 │ - 384. │ 383 │ UInt64 │ 1 │ - 385. │ 384 │ UInt64 │ 0 │ - 386. │ 385 │ UInt64 │ 1 │ - 387. │ 386 │ UInt64 │ 0 │ - 388. │ 387 │ UInt64 │ 1 │ - 389. │ 388 │ UInt64 │ 0 │ - 390. │ 389 │ UInt64 │ 1 │ - 391. │ 390 │ UInt64 │ 0 │ - 392. │ 391 │ UInt64 │ 1 │ - 393. │ 392 │ UInt64 │ 0 │ - 394. │ 393 │ UInt64 │ 1 │ - 395. │ 394 │ UInt64 │ 0 │ - 396. │ 395 │ UInt64 │ 1 │ - 397. │ 396 │ UInt64 │ 0 │ - 398. │ 397 │ UInt64 │ 1 │ - 399. │ 398 │ UInt64 │ 0 │ - 400. │ 399 │ UInt64 │ 1 │ - 401. │ 400 │ UInt64 │ 0 │ - 402. │ 401 │ UInt64 │ 1 │ - 403. │ 402 │ UInt64 │ 0 │ - 404. │ 403 │ UInt64 │ 1 │ - 405. │ 404 │ UInt64 │ 0 │ - 406. │ 405 │ UInt64 │ 1 │ - 407. │ 406 │ UInt64 │ 0 │ - 408. │ 407 │ UInt64 │ 1 │ - 409. │ 408 │ UInt64 │ 0 │ - 410. │ 409 │ UInt64 │ 1 │ - 411. │ 410 │ UInt64 │ 0 │ - 412. │ 411 │ UInt64 │ 1 │ - 413. │ 412 │ UInt64 │ 0 │ - 414. │ 413 │ UInt64 │ 1 │ - 415. │ 414 │ UInt64 │ 0 │ - 416. │ 415 │ UInt64 │ 1 │ - 417. │ 416 │ UInt64 │ 0 │ - 418. │ 417 │ UInt64 │ 1 │ - 419. │ 418 │ UInt64 │ 0 │ - 420. │ 419 │ UInt64 │ 1 │ - 421. │ 420 │ UInt64 │ 0 │ - 422. │ 421 │ UInt64 │ 1 │ - 423. │ 422 │ UInt64 │ 0 │ - 424. │ 423 │ UInt64 │ 1 │ - 425. │ 424 │ UInt64 │ 0 │ - 426. │ 425 │ UInt64 │ 1 │ - 427. │ 426 │ UInt64 │ 0 │ - 428. │ 427 │ UInt64 │ 1 │ - 429. │ 428 │ UInt64 │ 0 │ - 430. │ 429 │ UInt64 │ 1 │ - 431. │ 430 │ UInt64 │ 0 │ - 432. │ 431 │ UInt64 │ 1 │ - 433. │ 432 │ UInt64 │ 0 │ - 434. │ 433 │ UInt64 │ 1 │ - 435. │ 434 │ UInt64 │ 0 │ - 436. │ 435 │ UInt64 │ 1 │ - 437. │ 436 │ UInt64 │ 0 │ - 438. │ 437 │ UInt64 │ 1 │ - 439. │ 438 │ UInt64 │ 0 │ - 440. │ 439 │ UInt64 │ 1 │ - 441. │ 440 │ UInt64 │ 0 │ - 442. │ 441 │ UInt64 │ 1 │ - 443. │ 442 │ UInt64 │ 0 │ - 444. │ 443 │ UInt64 │ 1 │ - 445. │ 444 │ UInt64 │ 0 │ - 446. │ 445 │ UInt64 │ 1 │ - 447. │ 446 │ UInt64 │ 0 │ - 448. │ 447 │ UInt64 │ 1 │ - 449. │ 448 │ UInt64 │ 0 │ - 450. │ 449 │ UInt64 │ 1 │ - 451. │ 450 │ UInt64 │ 0 │ - 452. │ 451 │ UInt64 │ 1 │ - 453. │ 452 │ UInt64 │ 0 │ - 454. │ 453 │ UInt64 │ 1 │ - 455. │ 454 │ UInt64 │ 0 │ - 456. │ 455 │ UInt64 │ 1 │ - 457. │ 456 │ UInt64 │ 0 │ - 458. │ 457 │ UInt64 │ 1 │ - 459. │ 458 │ UInt64 │ 0 │ - 460. │ 459 │ UInt64 │ 1 │ - 461. │ 460 │ UInt64 │ 0 │ - 462. │ 461 │ UInt64 │ 1 │ - 463. │ 462 │ UInt64 │ 0 │ - 464. │ 463 │ UInt64 │ 1 │ - 465. │ 464 │ UInt64 │ 0 │ - 466. │ 465 │ UInt64 │ 1 │ - 467. │ 466 │ UInt64 │ 0 │ - 468. │ 467 │ UInt64 │ 1 │ - 469. │ 468 │ UInt64 │ 0 │ - 470. │ 469 │ UInt64 │ 1 │ - 471. │ 470 │ UInt64 │ 0 │ - 472. │ 471 │ UInt64 │ 1 │ - 473. │ 472 │ UInt64 │ 0 │ - 474. │ 473 │ UInt64 │ 1 │ - 475. │ 474 │ UInt64 │ 0 │ - 476. │ 475 │ UInt64 │ 1 │ - 477. │ 476 │ UInt64 │ 0 │ - 478. │ 477 │ UInt64 │ 1 │ - 479. │ 478 │ UInt64 │ 0 │ - 480. │ 479 │ UInt64 │ 1 │ - 481. │ 480 │ UInt64 │ 0 │ - 482. │ 481 │ UInt64 │ 1 │ - 483. │ 482 │ UInt64 │ 0 │ - 484. │ 483 │ UInt64 │ 1 │ - 485. │ 484 │ UInt64 │ 0 │ - 486. │ 485 │ UInt64 │ 1 │ - 487. │ 486 │ UInt64 │ 0 │ - 488. │ 487 │ UInt64 │ 1 │ - 489. │ 488 │ UInt64 │ 0 │ - 490. │ 489 │ UInt64 │ 1 │ - 491. │ 490 │ UInt64 │ 0 │ - 492. │ 491 │ UInt64 │ 1 │ - 493. │ 492 │ UInt64 │ 0 │ - 494. │ 493 │ UInt64 │ 1 │ - 495. │ 494 │ UInt64 │ 0 │ - 496. │ 495 │ UInt64 │ 1 │ - 497. │ 496 │ UInt64 │ 0 │ - 498. │ 497 │ UInt64 │ 1 │ - 499. │ 498 │ UInt64 │ 0 │ - 500. │ 499 │ UInt64 │ 1 │ - 501. │ 500 │ UInt64 │ 0 │ - 502. │ 501 │ UInt64 │ 1 │ - 503. │ 502 │ UInt64 │ 0 │ - 504. │ 503 │ UInt64 │ 1 │ - 505. │ 504 │ UInt64 │ 0 │ - 506. │ 505 │ UInt64 │ 1 │ - 507. │ 506 │ UInt64 │ 0 │ - 508. │ 507 │ UInt64 │ 1 │ - 509. │ 508 │ UInt64 │ 0 │ - 510. │ 509 │ UInt64 │ 1 │ - 511. │ 510 │ UInt64 │ 0 │ - 512. │ 511 │ UInt64 │ 1 │ - 513. │ 512 │ UInt64 │ 0 │ - 514. │ 513 │ UInt64 │ 1 │ - 515. │ 514 │ UInt64 │ 0 │ - 516. │ 515 │ UInt64 │ 1 │ - 517. │ 516 │ UInt64 │ 0 │ - 518. │ 517 │ UInt64 │ 1 │ - 519. │ 518 │ UInt64 │ 0 │ - 520. │ 519 │ UInt64 │ 1 │ - 521. │ 520 │ UInt64 │ 0 │ - 522. │ 521 │ UInt64 │ 1 │ - 523. │ 522 │ UInt64 │ 0 │ - 524. │ 523 │ UInt64 │ 1 │ - 525. │ 524 │ UInt64 │ 0 │ - 526. │ 525 │ UInt64 │ 1 │ - 527. │ 526 │ UInt64 │ 0 │ - 528. │ 527 │ UInt64 │ 1 │ - 529. │ 528 │ UInt64 │ 0 │ - 530. │ 529 │ UInt64 │ 1 │ - 531. │ 530 │ UInt64 │ 0 │ - 532. │ 531 │ UInt64 │ 1 │ - 533. │ 532 │ UInt64 │ 0 │ - 534. │ 533 │ UInt64 │ 1 │ - 535. │ 534 │ UInt64 │ 0 │ - 536. │ 535 │ UInt64 │ 1 │ - 537. │ 536 │ UInt64 │ 0 │ - 538. │ 537 │ UInt64 │ 1 │ - 539. │ 538 │ UInt64 │ 0 │ - 540. │ 539 │ UInt64 │ 1 │ - 541. │ 540 │ UInt64 │ 0 │ - 542. │ 541 │ UInt64 │ 1 │ - 543. │ 542 │ UInt64 │ 0 │ - 544. │ 543 │ UInt64 │ 1 │ - 545. │ 544 │ UInt64 │ 0 │ - 546. │ 545 │ UInt64 │ 1 │ - 547. │ 546 │ UInt64 │ 0 │ - 548. │ 547 │ UInt64 │ 1 │ - 549. │ 548 │ UInt64 │ 0 │ - 550. │ 549 │ UInt64 │ 1 │ - 551. │ 550 │ UInt64 │ 0 │ - 552. │ 551 │ UInt64 │ 1 │ - 553. │ 552 │ UInt64 │ 0 │ - 554. │ 553 │ UInt64 │ 1 │ - 555. │ 554 │ UInt64 │ 0 │ - 556. │ 555 │ UInt64 │ 1 │ - 557. │ 556 │ UInt64 │ 0 │ - 558. │ 557 │ UInt64 │ 1 │ - 559. │ 558 │ UInt64 │ 0 │ - 560. │ 559 │ UInt64 │ 1 │ - 561. │ 560 │ UInt64 │ 0 │ - 562. │ 561 │ UInt64 │ 1 │ - 563. │ 562 │ UInt64 │ 0 │ - 564. │ 563 │ UInt64 │ 1 │ - 565. │ 564 │ UInt64 │ 0 │ - 566. │ 565 │ UInt64 │ 1 │ - 567. │ 566 │ UInt64 │ 0 │ - 568. │ 567 │ UInt64 │ 1 │ - 569. │ 568 │ UInt64 │ 0 │ - 570. │ 569 │ UInt64 │ 1 │ - 571. │ 570 │ UInt64 │ 0 │ - 572. │ 571 │ UInt64 │ 1 │ - 573. │ 572 │ UInt64 │ 0 │ - 574. │ 573 │ UInt64 │ 1 │ - 575. │ 574 │ UInt64 │ 0 │ - 576. │ 575 │ UInt64 │ 1 │ - 577. │ 576 │ UInt64 │ 0 │ - 578. │ 577 │ UInt64 │ 1 │ - 579. │ 578 │ UInt64 │ 0 │ - 580. │ 579 │ UInt64 │ 1 │ - 581. │ 580 │ UInt64 │ 0 │ - 582. │ 581 │ UInt64 │ 1 │ - 583. │ 582 │ UInt64 │ 0 │ - 584. │ 583 │ UInt64 │ 1 │ - 585. │ 584 │ UInt64 │ 0 │ - 586. │ 585 │ UInt64 │ 1 │ - 587. │ 586 │ UInt64 │ 0 │ - 588. │ 587 │ UInt64 │ 1 │ - 589. │ 588 │ UInt64 │ 0 │ - 590. │ 589 │ UInt64 │ 1 │ - 591. │ 590 │ UInt64 │ 0 │ - 592. │ 591 │ UInt64 │ 1 │ - 593. │ 592 │ UInt64 │ 0 │ - 594. │ 593 │ UInt64 │ 1 │ - 595. │ 594 │ UInt64 │ 0 │ - 596. │ 595 │ UInt64 │ 1 │ - 597. │ 596 │ UInt64 │ 0 │ - 598. │ 597 │ UInt64 │ 1 │ - 599. │ 598 │ UInt64 │ 0 │ - 600. │ 599 │ UInt64 │ 1 │ - 601. │ 600 │ UInt64 │ 0 │ - 602. │ 601 │ UInt64 │ 1 │ - 603. │ 602 │ UInt64 │ 0 │ - 604. │ 603 │ UInt64 │ 1 │ - 605. │ 604 │ UInt64 │ 0 │ - 606. │ 605 │ UInt64 │ 1 │ - 607. │ 606 │ UInt64 │ 0 │ - 608. │ 607 │ UInt64 │ 1 │ - 609. │ 608 │ UInt64 │ 0 │ - 610. │ 609 │ UInt64 │ 1 │ - 611. │ 610 │ UInt64 │ 0 │ - 612. │ 611 │ UInt64 │ 1 │ - 613. │ 612 │ UInt64 │ 0 │ - 614. │ 613 │ UInt64 │ 1 │ - 615. │ 614 │ UInt64 │ 0 │ - 616. │ 615 │ UInt64 │ 1 │ - 617. │ 616 │ UInt64 │ 0 │ - 618. │ 617 │ UInt64 │ 1 │ - 619. │ 618 │ UInt64 │ 0 │ - 620. │ 619 │ UInt64 │ 1 │ - 621. │ 620 │ UInt64 │ 0 │ - 622. │ 621 │ UInt64 │ 1 │ - 623. │ 622 │ UInt64 │ 0 │ - 624. │ 623 │ UInt64 │ 1 │ - 625. │ 624 │ UInt64 │ 0 │ - 626. │ 625 │ UInt64 │ 1 │ - 627. │ 626 │ UInt64 │ 0 │ - 628. │ 627 │ UInt64 │ 1 │ - 629. │ 628 │ UInt64 │ 0 │ - 630. │ 629 │ UInt64 │ 1 │ - 631. │ 630 │ UInt64 │ 0 │ - 632. │ 631 │ UInt64 │ 1 │ - 633. │ 632 │ UInt64 │ 0 │ - 634. │ 633 │ UInt64 │ 1 │ - 635. │ 634 │ UInt64 │ 0 │ - 636. │ 635 │ UInt64 │ 1 │ - 637. │ 636 │ UInt64 │ 0 │ - 638. │ 637 │ UInt64 │ 1 │ - 639. │ 638 │ UInt64 │ 0 │ - 640. │ 639 │ UInt64 │ 1 │ - 641. │ 640 │ UInt64 │ 0 │ - 642. │ 641 │ UInt64 │ 1 │ - 643. │ 642 │ UInt64 │ 0 │ - 644. │ 643 │ UInt64 │ 1 │ - 645. │ 644 │ UInt64 │ 0 │ - 646. │ 645 │ UInt64 │ 1 │ - 647. │ 646 │ UInt64 │ 0 │ - 648. │ 647 │ UInt64 │ 1 │ - 649. │ 648 │ UInt64 │ 0 │ - 650. │ 649 │ UInt64 │ 1 │ - 651. │ 650 │ UInt64 │ 0 │ - 652. │ 651 │ UInt64 │ 1 │ - 653. │ 652 │ UInt64 │ 0 │ - 654. │ 653 │ UInt64 │ 1 │ - 655. │ 654 │ UInt64 │ 0 │ - 656. │ 655 │ UInt64 │ 1 │ - 657. │ 656 │ UInt64 │ 0 │ - 658. │ 657 │ UInt64 │ 1 │ - 659. │ 658 │ UInt64 │ 0 │ - 660. │ 659 │ UInt64 │ 1 │ - 661. │ 660 │ UInt64 │ 0 │ - 662. │ 661 │ UInt64 │ 1 │ - 663. │ 662 │ UInt64 │ 0 │ - 664. │ 663 │ UInt64 │ 1 │ - 665. │ 664 │ UInt64 │ 0 │ - 666. │ 665 │ UInt64 │ 1 │ - 667. │ 666 │ UInt64 │ 0 │ - 668. │ 667 │ UInt64 │ 1 │ - 669. │ 668 │ UInt64 │ 0 │ - 670. │ 669 │ UInt64 │ 1 │ - 671. │ 670 │ UInt64 │ 0 │ - 672. │ 671 │ UInt64 │ 1 │ - 673. │ 672 │ UInt64 │ 0 │ - 674. │ 673 │ UInt64 │ 1 │ - 675. │ 674 │ UInt64 │ 0 │ - 676. │ 675 │ UInt64 │ 1 │ - 677. │ 676 │ UInt64 │ 0 │ - 678. │ 677 │ UInt64 │ 1 │ - 679. │ 678 │ UInt64 │ 0 │ - 680. │ 679 │ UInt64 │ 1 │ - 681. │ 680 │ UInt64 │ 0 │ - 682. │ 681 │ UInt64 │ 1 │ - 683. │ 682 │ UInt64 │ 0 │ - 684. │ 683 │ UInt64 │ 1 │ - 685. │ 684 │ UInt64 │ 0 │ - 686. │ 685 │ UInt64 │ 1 │ - 687. │ 686 │ UInt64 │ 0 │ - 688. │ 687 │ UInt64 │ 1 │ - 689. │ 688 │ UInt64 │ 0 │ - 690. │ 689 │ UInt64 │ 1 │ - 691. │ 690 │ UInt64 │ 0 │ - 692. │ 691 │ UInt64 │ 1 │ - 693. │ 692 │ UInt64 │ 0 │ - 694. │ 693 │ UInt64 │ 1 │ - 695. │ 694 │ UInt64 │ 0 │ - 696. │ 695 │ UInt64 │ 1 │ - 697. │ 696 │ UInt64 │ 0 │ - 698. │ 697 │ UInt64 │ 1 │ - 699. │ 698 │ UInt64 │ 0 │ - 700. │ 699 │ UInt64 │ 1 │ - 701. │ 700 │ UInt64 │ 0 │ - 702. │ 701 │ UInt64 │ 1 │ - 703. │ 702 │ UInt64 │ 0 │ - 704. │ 703 │ UInt64 │ 1 │ - 705. │ 704 │ UInt64 │ 0 │ - 706. │ 705 │ UInt64 │ 1 │ - 707. │ 706 │ UInt64 │ 0 │ - 708. │ 707 │ UInt64 │ 1 │ - 709. │ 708 │ UInt64 │ 0 │ - 710. │ 709 │ UInt64 │ 1 │ - 711. │ 710 │ UInt64 │ 0 │ - 712. │ 711 │ UInt64 │ 1 │ - 713. │ 712 │ UInt64 │ 0 │ - 714. │ 713 │ UInt64 │ 1 │ - 715. │ 714 │ UInt64 │ 0 │ - 716. │ 715 │ UInt64 │ 1 │ - 717. │ 716 │ UInt64 │ 0 │ - 718. │ 717 │ UInt64 │ 1 │ - 719. │ 718 │ UInt64 │ 0 │ - 720. │ 719 │ UInt64 │ 1 │ - 721. │ 720 │ UInt64 │ 0 │ - 722. │ 721 │ UInt64 │ 1 │ - 723. │ 722 │ UInt64 │ 0 │ - 724. │ 723 │ UInt64 │ 1 │ - 725. │ 724 │ UInt64 │ 0 │ - 726. │ 725 │ UInt64 │ 1 │ - 727. │ 726 │ UInt64 │ 0 │ - 728. │ 727 │ UInt64 │ 1 │ - 729. │ 728 │ UInt64 │ 0 │ - 730. │ 729 │ UInt64 │ 1 │ - 731. │ 730 │ UInt64 │ 0 │ - 732. │ 731 │ UInt64 │ 1 │ - 733. │ 732 │ UInt64 │ 0 │ - 734. │ 733 │ UInt64 │ 1 │ - 735. │ 734 │ UInt64 │ 0 │ - 736. │ 735 │ UInt64 │ 1 │ - 737. │ 736 │ UInt64 │ 0 │ - 738. │ 737 │ UInt64 │ 1 │ - 739. │ 738 │ UInt64 │ 0 │ - 740. │ 739 │ UInt64 │ 1 │ - 741. │ 740 │ UInt64 │ 0 │ - 742. │ 741 │ UInt64 │ 1 │ - 743. │ 742 │ UInt64 │ 0 │ - 744. │ 743 │ UInt64 │ 1 │ - 745. │ 744 │ UInt64 │ 0 │ - 746. │ 745 │ UInt64 │ 1 │ - 747. │ 746 │ UInt64 │ 0 │ - 748. │ 747 │ UInt64 │ 1 │ - 749. │ 748 │ UInt64 │ 0 │ - 750. │ 749 │ UInt64 │ 1 │ - 751. │ 750 │ UInt64 │ 0 │ - 752. │ 751 │ UInt64 │ 1 │ - 753. │ 752 │ UInt64 │ 0 │ - 754. │ 753 │ UInt64 │ 1 │ - 755. │ 754 │ UInt64 │ 0 │ - 756. │ 755 │ UInt64 │ 1 │ - 757. │ 756 │ UInt64 │ 0 │ - 758. │ 757 │ UInt64 │ 1 │ - 759. │ 758 │ UInt64 │ 0 │ - 760. │ 759 │ UInt64 │ 1 │ - 761. │ 760 │ UInt64 │ 0 │ - 762. │ 761 │ UInt64 │ 1 │ - 763. │ 762 │ UInt64 │ 0 │ - 764. │ 763 │ UInt64 │ 1 │ - 765. │ 764 │ UInt64 │ 0 │ - 766. │ 765 │ UInt64 │ 1 │ - 767. │ 766 │ UInt64 │ 0 │ - 768. │ 767 │ UInt64 │ 1 │ - 769. │ 768 │ UInt64 │ 0 │ - 770. │ 769 │ UInt64 │ 1 │ - 771. │ 770 │ UInt64 │ 0 │ - 772. │ 771 │ UInt64 │ 1 │ - 773. │ 772 │ UInt64 │ 0 │ - 774. │ 773 │ UInt64 │ 1 │ - 775. │ 774 │ UInt64 │ 0 │ - 776. │ 775 │ UInt64 │ 1 │ - 777. │ 776 │ UInt64 │ 0 │ - 778. │ 777 │ UInt64 │ 1 │ - 779. │ 778 │ UInt64 │ 0 │ - 780. │ 779 │ UInt64 │ 1 │ - 781. │ 780 │ UInt64 │ 0 │ - 782. │ 781 │ UInt64 │ 1 │ - 783. │ 782 │ UInt64 │ 0 │ - 784. │ 783 │ UInt64 │ 1 │ - 785. │ 784 │ UInt64 │ 0 │ - 786. │ 785 │ UInt64 │ 1 │ - 787. │ 786 │ UInt64 │ 0 │ - 788. │ 787 │ UInt64 │ 1 │ - 789. │ 788 │ UInt64 │ 0 │ - 790. │ 789 │ UInt64 │ 1 │ - 791. │ 790 │ UInt64 │ 0 │ - 792. │ 791 │ UInt64 │ 1 │ - 793. │ 792 │ UInt64 │ 0 │ - 794. │ 793 │ UInt64 │ 1 │ - 795. │ 794 │ UInt64 │ 0 │ - 796. │ 795 │ UInt64 │ 1 │ - 797. │ 796 │ UInt64 │ 0 │ - 798. │ 797 │ UInt64 │ 1 │ - 799. │ 798 │ UInt64 │ 0 │ - 800. │ 799 │ UInt64 │ 1 │ - 801. │ 800 │ UInt64 │ 0 │ - 802. │ 801 │ UInt64 │ 1 │ - 803. │ 802 │ UInt64 │ 0 │ - 804. │ 803 │ UInt64 │ 1 │ - 805. │ 804 │ UInt64 │ 0 │ - 806. │ 805 │ UInt64 │ 1 │ - 807. │ 806 │ UInt64 │ 0 │ - 808. │ 807 │ UInt64 │ 1 │ - 809. │ 808 │ UInt64 │ 0 │ - 810. │ 809 │ UInt64 │ 1 │ - 811. │ 810 │ UInt64 │ 0 │ - 812. │ 811 │ UInt64 │ 1 │ - 813. │ 812 │ UInt64 │ 0 │ - 814. │ 813 │ UInt64 │ 1 │ - 815. │ 814 │ UInt64 │ 0 │ - 816. │ 815 │ UInt64 │ 1 │ - 817. │ 816 │ UInt64 │ 0 │ - 818. │ 817 │ UInt64 │ 1 │ - 819. │ 818 │ UInt64 │ 0 │ - 820. │ 819 │ UInt64 │ 1 │ - 821. │ 820 │ UInt64 │ 0 │ - 822. │ 821 │ UInt64 │ 1 │ - 823. │ 822 │ UInt64 │ 0 │ - 824. │ 823 │ UInt64 │ 1 │ - 825. │ 824 │ UInt64 │ 0 │ - 826. │ 825 │ UInt64 │ 1 │ - 827. │ 826 │ UInt64 │ 0 │ - 828. │ 827 │ UInt64 │ 1 │ - 829. │ 828 │ UInt64 │ 0 │ - 830. │ 829 │ UInt64 │ 1 │ - 831. │ 830 │ UInt64 │ 0 │ - 832. │ 831 │ UInt64 │ 1 │ - 833. │ 832 │ UInt64 │ 0 │ - 834. │ 833 │ UInt64 │ 1 │ - 835. │ 834 │ UInt64 │ 0 │ - 836. │ 835 │ UInt64 │ 1 │ - 837. │ 836 │ UInt64 │ 0 │ - 838. │ 837 │ UInt64 │ 1 │ - 839. │ 838 │ UInt64 │ 0 │ - 840. │ 839 │ UInt64 │ 1 │ - 841. │ 840 │ UInt64 │ 0 │ - 842. │ 841 │ UInt64 │ 1 │ - 843. │ 842 │ UInt64 │ 0 │ - 844. │ 843 │ UInt64 │ 1 │ - 845. │ 844 │ UInt64 │ 0 │ - 846. │ 845 │ UInt64 │ 1 │ - 847. │ 846 │ UInt64 │ 0 │ - 848. │ 847 │ UInt64 │ 1 │ - 849. │ 848 │ UInt64 │ 0 │ - 850. │ 849 │ UInt64 │ 1 │ - 851. │ 850 │ UInt64 │ 0 │ - 852. │ 851 │ UInt64 │ 1 │ - 853. │ 852 │ UInt64 │ 0 │ - 854. │ 853 │ UInt64 │ 1 │ - 855. │ 854 │ UInt64 │ 0 │ - 856. │ 855 │ UInt64 │ 1 │ - 857. │ 856 │ UInt64 │ 0 │ - 858. │ 857 │ UInt64 │ 1 │ - 859. │ 858 │ UInt64 │ 0 │ - 860. │ 859 │ UInt64 │ 1 │ - 861. │ 860 │ UInt64 │ 0 │ - 862. │ 861 │ UInt64 │ 1 │ - 863. │ 862 │ UInt64 │ 0 │ - 864. │ 863 │ UInt64 │ 1 │ - 865. │ 864 │ UInt64 │ 0 │ - 866. │ 865 │ UInt64 │ 1 │ - 867. │ 866 │ UInt64 │ 0 │ - 868. │ 867 │ UInt64 │ 1 │ - 869. │ 868 │ UInt64 │ 0 │ - 870. │ 869 │ UInt64 │ 1 │ - 871. │ 870 │ UInt64 │ 0 │ - 872. │ 871 │ UInt64 │ 1 │ - 873. │ 872 │ UInt64 │ 0 │ - 874. │ 873 │ UInt64 │ 1 │ - 875. │ 874 │ UInt64 │ 0 │ - 876. │ 875 │ UInt64 │ 1 │ - 877. │ 876 │ UInt64 │ 0 │ - 878. │ 877 │ UInt64 │ 1 │ - 879. │ 878 │ UInt64 │ 0 │ - 880. │ 879 │ UInt64 │ 1 │ - 881. │ 880 │ UInt64 │ 0 │ - 882. │ 881 │ UInt64 │ 1 │ - 883. │ 882 │ UInt64 │ 0 │ - 884. │ 883 │ UInt64 │ 1 │ - 885. │ 884 │ UInt64 │ 0 │ - 886. │ 885 │ UInt64 │ 1 │ - 887. │ 886 │ UInt64 │ 0 │ - 888. │ 887 │ UInt64 │ 1 │ - 889. │ 888 │ UInt64 │ 0 │ - 890. │ 889 │ UInt64 │ 1 │ - 891. │ 890 │ UInt64 │ 0 │ - 892. │ 891 │ UInt64 │ 1 │ - 893. │ 892 │ UInt64 │ 0 │ - 894. │ 893 │ UInt64 │ 1 │ - 895. │ 894 │ UInt64 │ 0 │ - 896. │ 895 │ UInt64 │ 1 │ - 897. │ 896 │ UInt64 │ 0 │ - 898. │ 897 │ UInt64 │ 1 │ - 899. │ 898 │ UInt64 │ 0 │ - 900. │ 899 │ UInt64 │ 1 │ - 901. │ 900 │ UInt64 │ 0 │ - 902. │ 901 │ UInt64 │ 1 │ - 903. │ 902 │ UInt64 │ 0 │ - 904. │ 903 │ UInt64 │ 1 │ - 905. │ 904 │ UInt64 │ 0 │ - 906. │ 905 │ UInt64 │ 1 │ - 907. │ 906 │ UInt64 │ 0 │ - 908. │ 907 │ UInt64 │ 1 │ - 909. │ 908 │ UInt64 │ 0 │ - 910. │ 909 │ UInt64 │ 1 │ - 911. │ 910 │ UInt64 │ 0 │ - 912. │ 911 │ UInt64 │ 1 │ - 913. │ 912 │ UInt64 │ 0 │ - 914. │ 913 │ UInt64 │ 1 │ - 915. │ 914 │ UInt64 │ 0 │ - 916. │ 915 │ UInt64 │ 1 │ - 917. │ 916 │ UInt64 │ 0 │ - 918. │ 917 │ UInt64 │ 1 │ - 919. │ 918 │ UInt64 │ 0 │ - 920. │ 919 │ UInt64 │ 1 │ - 921. │ 920 │ UInt64 │ 0 │ - 922. │ 921 │ UInt64 │ 1 │ - 923. │ 922 │ UInt64 │ 0 │ - 924. │ 923 │ UInt64 │ 1 │ - 925. │ 924 │ UInt64 │ 0 │ - 926. │ 925 │ UInt64 │ 1 │ - 927. │ 926 │ UInt64 │ 0 │ - 928. │ 927 │ UInt64 │ 1 │ - 929. │ 928 │ UInt64 │ 0 │ - 930. │ 929 │ UInt64 │ 1 │ - 931. │ 930 │ UInt64 │ 0 │ - 932. │ 931 │ UInt64 │ 1 │ - 933. │ 932 │ UInt64 │ 0 │ - 934. │ 933 │ UInt64 │ 1 │ - 935. │ 934 │ UInt64 │ 0 │ - 936. │ 935 │ UInt64 │ 1 │ - 937. │ 936 │ UInt64 │ 0 │ - 938. │ 937 │ UInt64 │ 1 │ - 939. │ 938 │ UInt64 │ 0 │ - 940. │ 939 │ UInt64 │ 1 │ - 941. │ 940 │ UInt64 │ 0 │ - 942. │ 941 │ UInt64 │ 1 │ - 943. │ 942 │ UInt64 │ 0 │ - 944. │ 943 │ UInt64 │ 1 │ - 945. │ 944 │ UInt64 │ 0 │ - 946. │ 945 │ UInt64 │ 1 │ - 947. │ 946 │ UInt64 │ 0 │ - 948. │ 947 │ UInt64 │ 1 │ - 949. │ 948 │ UInt64 │ 0 │ - 950. │ 949 │ UInt64 │ 1 │ - 951. │ 950 │ UInt64 │ 0 │ - 952. │ 951 │ UInt64 │ 1 │ - 953. │ 952 │ UInt64 │ 0 │ - 954. │ 953 │ UInt64 │ 1 │ - 955. │ 954 │ UInt64 │ 0 │ - 956. │ 955 │ UInt64 │ 1 │ - 957. │ 956 │ UInt64 │ 0 │ - 958. │ 957 │ UInt64 │ 1 │ - 959. │ 958 │ UInt64 │ 0 │ - 960. │ 959 │ UInt64 │ 1 │ - 961. │ 960 │ UInt64 │ 0 │ - 962. │ 961 │ UInt64 │ 1 │ - 963. │ 962 │ UInt64 │ 0 │ - 964. │ 963 │ UInt64 │ 1 │ - 965. │ 964 │ UInt64 │ 0 │ - 966. │ 965 │ UInt64 │ 1 │ - 967. │ 966 │ UInt64 │ 0 │ - 968. │ 967 │ UInt64 │ 1 │ - 969. │ 968 │ UInt64 │ 0 │ - 970. │ 969 │ UInt64 │ 1 │ - 971. │ 970 │ UInt64 │ 0 │ - 972. │ 971 │ UInt64 │ 1 │ - 973. │ 972 │ UInt64 │ 0 │ - 974. │ 973 │ UInt64 │ 1 │ - 975. │ 974 │ UInt64 │ 0 │ - 976. │ 975 │ UInt64 │ 1 │ - 977. │ 976 │ UInt64 │ 0 │ - 978. │ 977 │ UInt64 │ 1 │ - 979. │ 978 │ UInt64 │ 0 │ - 980. │ 979 │ UInt64 │ 1 │ - 981. │ 980 │ UInt64 │ 0 │ - 982. │ 981 │ UInt64 │ 1 │ - 983. │ 982 │ UInt64 │ 0 │ - 984. │ 983 │ UInt64 │ 1 │ - 985. │ 984 │ UInt64 │ 0 │ - 986. │ 985 │ UInt64 │ 1 │ - 987. │ 986 │ UInt64 │ 0 │ - 988. │ 987 │ UInt64 │ 1 │ - 989. │ 988 │ UInt64 │ 0 │ - 990. │ 989 │ UInt64 │ 1 │ - 991. │ 990 │ UInt64 │ 0 │ - 992. │ 991 │ UInt64 │ 1 │ - 993. │ 992 │ UInt64 │ 0 │ - 994. │ 993 │ UInt64 │ 1 │ - 995. │ 994 │ UInt64 │ 0 │ - 996. │ 995 │ UInt64 │ 1 │ - 997. │ 996 │ UInt64 │ 0 │ - 998. │ 997 │ UInt64 │ 1 │ - 999. │ 998 │ UInt64 │ 0 │ -1000. │ 999 │ UInt64 │ 1 │ - └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ - 100. │ 99 │ UInt64 │ 1 │ - 101. │ 100 │ UInt64 │ 0 │ - 102. │ 101 │ UInt64 │ 1 │ - 103. │ 102 │ UInt64 │ 0 │ - 104. │ 103 │ UInt64 │ 1 │ - 105. │ 104 │ UInt64 │ 0 │ - 106. │ 105 │ UInt64 │ 1 │ - 107. │ 106 │ UInt64 │ 0 │ - 108. │ 107 │ UInt64 │ 1 │ - 109. │ 108 │ UInt64 │ 0 │ - 110. │ 109 │ UInt64 │ 1 │ - 111. │ 110 │ UInt64 │ 0 │ - 112. │ 111 │ UInt64 │ 1 │ - 113. │ 112 │ UInt64 │ 0 │ - 114. │ 113 │ UInt64 │ 1 │ - 115. │ 114 │ UInt64 │ 0 │ - 116. │ 115 │ UInt64 │ 1 │ - 117. │ 116 │ UInt64 │ 0 │ - 118. │ 117 │ UInt64 │ 1 │ - 119. │ 118 │ UInt64 │ 0 │ - 120. │ 119 │ UInt64 │ 1 │ - 121. │ 120 │ UInt64 │ 0 │ - 122. │ 121 │ UInt64 │ 1 │ - 123. │ 122 │ UInt64 │ 0 │ - 124. │ 123 │ UInt64 │ 1 │ - 125. │ 124 │ UInt64 │ 0 │ - 126. │ 125 │ UInt64 │ 1 │ - 127. │ 126 │ UInt64 │ 0 │ - 128. │ 127 │ UInt64 │ 1 │ - 129. │ 128 │ UInt64 │ 0 │ - 130. │ 129 │ UInt64 │ 1 │ - 131. │ 130 │ UInt64 │ 0 │ - 132. │ 131 │ UInt64 │ 1 │ - 133. │ 132 │ UInt64 │ 0 │ - 134. │ 133 │ UInt64 │ 1 │ - 135. │ 134 │ UInt64 │ 0 │ - 136. │ 135 │ UInt64 │ 1 │ - 137. │ 136 │ UInt64 │ 0 │ - 138. │ 137 │ UInt64 │ 1 │ - 139. │ 138 │ UInt64 │ 0 │ - 140. │ 139 │ UInt64 │ 1 │ - 141. │ 140 │ UInt64 │ 0 │ - 142. │ 141 │ UInt64 │ 1 │ - 143. │ 142 │ UInt64 │ 0 │ - 144. │ 143 │ UInt64 │ 1 │ - 145. │ 144 │ UInt64 │ 0 │ - 146. │ 145 │ UInt64 │ 1 │ - 147. │ 146 │ UInt64 │ 0 │ - 148. │ 147 │ UInt64 │ 1 │ - 149. │ 148 │ UInt64 │ 0 │ - 150. │ 149 │ UInt64 │ 1 │ - 151. │ 150 │ UInt64 │ 0 │ - 152. │ 151 │ UInt64 │ 1 │ - 153. │ 152 │ UInt64 │ 0 │ - 154. │ 153 │ UInt64 │ 1 │ - 155. │ 154 │ UInt64 │ 0 │ - 156. │ 155 │ UInt64 │ 1 │ - 157. │ 156 │ UInt64 │ 0 │ - 158. │ 157 │ UInt64 │ 1 │ - 159. │ 158 │ UInt64 │ 0 │ - 160. │ 159 │ UInt64 │ 1 │ - 161. │ 160 │ UInt64 │ 0 │ - 162. │ 161 │ UInt64 │ 1 │ - 163. │ 162 │ UInt64 │ 0 │ - 164. │ 163 │ UInt64 │ 1 │ - 165. │ 164 │ UInt64 │ 0 │ - 166. │ 165 │ UInt64 │ 1 │ - 167. │ 166 │ UInt64 │ 0 │ - 168. │ 167 │ UInt64 │ 1 │ - 169. │ 168 │ UInt64 │ 0 │ - 170. │ 169 │ UInt64 │ 1 │ - 171. │ 170 │ UInt64 │ 0 │ - 172. │ 171 │ UInt64 │ 1 │ - 173. │ 172 │ UInt64 │ 0 │ - 174. │ 173 │ UInt64 │ 1 │ - 175. │ 174 │ UInt64 │ 0 │ - 176. │ 175 │ UInt64 │ 1 │ - 177. │ 176 │ UInt64 │ 0 │ - 178. │ 177 │ UInt64 │ 1 │ - 179. │ 178 │ UInt64 │ 0 │ - 180. │ 179 │ UInt64 │ 1 │ - 181. │ 180 │ UInt64 │ 0 │ - 182. │ 181 │ UInt64 │ 1 │ - 183. │ 182 │ UInt64 │ 0 │ - 184. │ 183 │ UInt64 │ 1 │ - 185. │ 184 │ UInt64 │ 0 │ - 186. │ 185 │ UInt64 │ 1 │ - 187. │ 186 │ UInt64 │ 0 │ - 188. │ 187 │ UInt64 │ 1 │ - 189. │ 188 │ UInt64 │ 0 │ - 190. │ 189 │ UInt64 │ 1 │ - 191. │ 190 │ UInt64 │ 0 │ - 192. │ 191 │ UInt64 │ 1 │ - 193. │ 192 │ UInt64 │ 0 │ - 194. │ 193 │ UInt64 │ 1 │ - 195. │ 194 │ UInt64 │ 0 │ - 196. │ 195 │ UInt64 │ 1 │ - 197. │ 196 │ UInt64 │ 0 │ - 198. │ 197 │ UInt64 │ 1 │ - 199. │ 198 │ UInt64 │ 0 │ - 200. │ 199 │ UInt64 │ 1 │ - 201. │ 200 │ UInt64 │ 0 │ - 202. │ 201 │ UInt64 │ 1 │ - 203. │ 202 │ UInt64 │ 0 │ - 204. │ 203 │ UInt64 │ 1 │ - 205. │ 204 │ UInt64 │ 0 │ - 206. │ 205 │ UInt64 │ 1 │ - 207. │ 206 │ UInt64 │ 0 │ - 208. │ 207 │ UInt64 │ 1 │ - 209. │ 208 │ UInt64 │ 0 │ - 210. │ 209 │ UInt64 │ 1 │ - 211. │ 210 │ UInt64 │ 0 │ - 212. │ 211 │ UInt64 │ 1 │ - 213. │ 212 │ UInt64 │ 0 │ - 214. │ 213 │ UInt64 │ 1 │ - 215. │ 214 │ UInt64 │ 0 │ - 216. │ 215 │ UInt64 │ 1 │ - 217. │ 216 │ UInt64 │ 0 │ - 218. │ 217 │ UInt64 │ 1 │ - 219. │ 218 │ UInt64 │ 0 │ - 220. │ 219 │ UInt64 │ 1 │ - 221. │ 220 │ UInt64 │ 0 │ - 222. │ 221 │ UInt64 │ 1 │ - 223. │ 222 │ UInt64 │ 0 │ - 224. │ 223 │ UInt64 │ 1 │ - 225. │ 224 │ UInt64 │ 0 │ - 226. │ 225 │ UInt64 │ 1 │ - 227. │ 226 │ UInt64 │ 0 │ - 228. │ 227 │ UInt64 │ 1 │ - 229. │ 228 │ UInt64 │ 0 │ - 230. │ 229 │ UInt64 │ 1 │ - 231. │ 230 │ UInt64 │ 0 │ - 232. │ 231 │ UInt64 │ 1 │ - 233. │ 232 │ UInt64 │ 0 │ - 234. │ 233 │ UInt64 │ 1 │ - 235. │ 234 │ UInt64 │ 0 │ - 236. │ 235 │ UInt64 │ 1 │ - 237. │ 236 │ UInt64 │ 0 │ - 238. │ 237 │ UInt64 │ 1 │ - 239. │ 238 │ UInt64 │ 0 │ - 240. │ 239 │ UInt64 │ 1 │ - 241. │ 240 │ UInt64 │ 0 │ - 242. │ 241 │ UInt64 │ 1 │ - 243. │ 242 │ UInt64 │ 0 │ - 244. │ 243 │ UInt64 │ 1 │ - 245. │ 244 │ UInt64 │ 0 │ - 246. │ 245 │ UInt64 │ 1 │ - 247. │ 246 │ UInt64 │ 0 │ - 248. │ 247 │ UInt64 │ 1 │ - 249. │ 248 │ UInt64 │ 0 │ - 250. │ 249 │ UInt64 │ 1 │ - 251. │ 250 │ UInt64 │ 0 │ - 252. │ 251 │ UInt64 │ 1 │ - 253. │ 252 │ UInt64 │ 0 │ - 254. │ 253 │ UInt64 │ 1 │ - 255. │ 254 │ UInt64 │ 0 │ - 256. │ 255 │ UInt64 │ 1 │ - 257. │ 256 │ UInt64 │ 0 │ - 258. │ 257 │ UInt64 │ 1 │ - 259. │ 258 │ UInt64 │ 0 │ - 260. │ 259 │ UInt64 │ 1 │ - 261. │ 260 │ UInt64 │ 0 │ - 262. │ 261 │ UInt64 │ 1 │ - 263. │ 262 │ UInt64 │ 0 │ - 264. │ 263 │ UInt64 │ 1 │ - 265. │ 264 │ UInt64 │ 0 │ - 266. │ 265 │ UInt64 │ 1 │ - 267. │ 266 │ UInt64 │ 0 │ - 268. │ 267 │ UInt64 │ 1 │ - 269. │ 268 │ UInt64 │ 0 │ - 270. │ 269 │ UInt64 │ 1 │ - 271. │ 270 │ UInt64 │ 0 │ - 272. │ 271 │ UInt64 │ 1 │ - 273. │ 272 │ UInt64 │ 0 │ - 274. │ 273 │ UInt64 │ 1 │ - 275. │ 274 │ UInt64 │ 0 │ - 276. │ 275 │ UInt64 │ 1 │ - 277. │ 276 │ UInt64 │ 0 │ - 278. │ 277 │ UInt64 │ 1 │ - 279. │ 278 │ UInt64 │ 0 │ - 280. │ 279 │ UInt64 │ 1 │ - 281. │ 280 │ UInt64 │ 0 │ - 282. │ 281 │ UInt64 │ 1 │ - 283. │ 282 │ UInt64 │ 0 │ - 284. │ 283 │ UInt64 │ 1 │ - 285. │ 284 │ UInt64 │ 0 │ - 286. │ 285 │ UInt64 │ 1 │ - 287. │ 286 │ UInt64 │ 0 │ - 288. │ 287 │ UInt64 │ 1 │ - 289. │ 288 │ UInt64 │ 0 │ - 290. │ 289 │ UInt64 │ 1 │ - 291. │ 290 │ UInt64 │ 0 │ - 292. │ 291 │ UInt64 │ 1 │ - 293. │ 292 │ UInt64 │ 0 │ - 294. │ 293 │ UInt64 │ 1 │ - 295. │ 294 │ UInt64 │ 0 │ - 296. │ 295 │ UInt64 │ 1 │ - 297. │ 296 │ UInt64 │ 0 │ - 298. │ 297 │ UInt64 │ 1 │ - 299. │ 298 │ UInt64 │ 0 │ - 300. │ 299 │ UInt64 │ 1 │ - 301. │ 300 │ UInt64 │ 0 │ - 302. │ 301 │ UInt64 │ 1 │ - 303. │ 302 │ UInt64 │ 0 │ - 304. │ 303 │ UInt64 │ 1 │ - 305. │ 304 │ UInt64 │ 0 │ - 306. │ 305 │ UInt64 │ 1 │ - 307. │ 306 │ UInt64 │ 0 │ - 308. │ 307 │ UInt64 │ 1 │ - 309. │ 308 │ UInt64 │ 0 │ - 310. │ 309 │ UInt64 │ 1 │ - 311. │ 310 │ UInt64 │ 0 │ - 312. │ 311 │ UInt64 │ 1 │ - 313. │ 312 │ UInt64 │ 0 │ - 314. │ 313 │ UInt64 │ 1 │ - 315. │ 314 │ UInt64 │ 0 │ - 316. │ 315 │ UInt64 │ 1 │ - 317. │ 316 │ UInt64 │ 0 │ - 318. │ 317 │ UInt64 │ 1 │ - 319. │ 318 │ UInt64 │ 0 │ - 320. │ 319 │ UInt64 │ 1 │ - 321. │ 320 │ UInt64 │ 0 │ - 322. │ 321 │ UInt64 │ 1 │ - 323. │ 322 │ UInt64 │ 0 │ - 324. │ 323 │ UInt64 │ 1 │ - 325. │ 324 │ UInt64 │ 0 │ - 326. │ 325 │ UInt64 │ 1 │ - 327. │ 326 │ UInt64 │ 0 │ - 328. │ 327 │ UInt64 │ 1 │ - 329. │ 328 │ UInt64 │ 0 │ - 330. │ 329 │ UInt64 │ 1 │ - 331. │ 330 │ UInt64 │ 0 │ - 332. │ 331 │ UInt64 │ 1 │ - 333. │ 332 │ UInt64 │ 0 │ - 334. │ 333 │ UInt64 │ 1 │ - 335. │ 334 │ UInt64 │ 0 │ - 336. │ 335 │ UInt64 │ 1 │ - 337. │ 336 │ UInt64 │ 0 │ - 338. │ 337 │ UInt64 │ 1 │ - 339. │ 338 │ UInt64 │ 0 │ - 340. │ 339 │ UInt64 │ 1 │ - 341. │ 340 │ UInt64 │ 0 │ - 342. │ 341 │ UInt64 │ 1 │ - 343. │ 342 │ UInt64 │ 0 │ - 344. │ 343 │ UInt64 │ 1 │ - 345. │ 344 │ UInt64 │ 0 │ - 346. │ 345 │ UInt64 │ 1 │ - 347. │ 346 │ UInt64 │ 0 │ - 348. │ 347 │ UInt64 │ 1 │ - 349. │ 348 │ UInt64 │ 0 │ - 350. │ 349 │ UInt64 │ 1 │ - 351. │ 350 │ UInt64 │ 0 │ - 352. │ 351 │ UInt64 │ 1 │ - 353. │ 352 │ UInt64 │ 0 │ - 354. │ 353 │ UInt64 │ 1 │ - 355. │ 354 │ UInt64 │ 0 │ - 356. │ 355 │ UInt64 │ 1 │ - 357. │ 356 │ UInt64 │ 0 │ - 358. │ 357 │ UInt64 │ 1 │ - 359. │ 358 │ UInt64 │ 0 │ - 360. │ 359 │ UInt64 │ 1 │ - 361. │ 360 │ UInt64 │ 0 │ - 362. │ 361 │ UInt64 │ 1 │ - 363. │ 362 │ UInt64 │ 0 │ - 364. │ 363 │ UInt64 │ 1 │ - 365. │ 364 │ UInt64 │ 0 │ - 366. │ 365 │ UInt64 │ 1 │ - 367. │ 366 │ UInt64 │ 0 │ - 368. │ 367 │ UInt64 │ 1 │ - 369. │ 368 │ UInt64 │ 0 │ - 370. │ 369 │ UInt64 │ 1 │ - 371. │ 370 │ UInt64 │ 0 │ - 372. │ 371 │ UInt64 │ 1 │ - 373. │ 372 │ UInt64 │ 0 │ - 374. │ 373 │ UInt64 │ 1 │ - 375. │ 374 │ UInt64 │ 0 │ - 376. │ 375 │ UInt64 │ 1 │ - 377. │ 376 │ UInt64 │ 0 │ - 378. │ 377 │ UInt64 │ 1 │ - 379. │ 378 │ UInt64 │ 0 │ - 380. │ 379 │ UInt64 │ 1 │ - 381. │ 380 │ UInt64 │ 0 │ - 382. │ 381 │ UInt64 │ 1 │ - 383. │ 382 │ UInt64 │ 0 │ - 384. │ 383 │ UInt64 │ 1 │ - 385. │ 384 │ UInt64 │ 0 │ - 386. │ 385 │ UInt64 │ 1 │ - 387. │ 386 │ UInt64 │ 0 │ - 388. │ 387 │ UInt64 │ 1 │ - 389. │ 388 │ UInt64 │ 0 │ - 390. │ 389 │ UInt64 │ 1 │ - 391. │ 390 │ UInt64 │ 0 │ - 392. │ 391 │ UInt64 │ 1 │ - 393. │ 392 │ UInt64 │ 0 │ - 394. │ 393 │ UInt64 │ 1 │ - 395. │ 394 │ UInt64 │ 0 │ - 396. │ 395 │ UInt64 │ 1 │ - 397. │ 396 │ UInt64 │ 0 │ - 398. │ 397 │ UInt64 │ 1 │ - 399. │ 398 │ UInt64 │ 0 │ - 400. │ 399 │ UInt64 │ 1 │ - 401. │ 400 │ UInt64 │ 0 │ - 402. │ 401 │ UInt64 │ 1 │ - 403. │ 402 │ UInt64 │ 0 │ - 404. │ 403 │ UInt64 │ 1 │ - 405. │ 404 │ UInt64 │ 0 │ - 406. │ 405 │ UInt64 │ 1 │ - 407. │ 406 │ UInt64 │ 0 │ - 408. │ 407 │ UInt64 │ 1 │ - 409. │ 408 │ UInt64 │ 0 │ - 410. │ 409 │ UInt64 │ 1 │ - 411. │ 410 │ UInt64 │ 0 │ - 412. │ 411 │ UInt64 │ 1 │ - 413. │ 412 │ UInt64 │ 0 │ - 414. │ 413 │ UInt64 │ 1 │ - 415. │ 414 │ UInt64 │ 0 │ - 416. │ 415 │ UInt64 │ 1 │ - 417. │ 416 │ UInt64 │ 0 │ - 418. │ 417 │ UInt64 │ 1 │ - 419. │ 418 │ UInt64 │ 0 │ - 420. │ 419 │ UInt64 │ 1 │ - 421. │ 420 │ UInt64 │ 0 │ - 422. │ 421 │ UInt64 │ 1 │ - 423. │ 422 │ UInt64 │ 0 │ - 424. │ 423 │ UInt64 │ 1 │ - 425. │ 424 │ UInt64 │ 0 │ - 426. │ 425 │ UInt64 │ 1 │ - 427. │ 426 │ UInt64 │ 0 │ - 428. │ 427 │ UInt64 │ 1 │ - 429. │ 428 │ UInt64 │ 0 │ - 430. │ 429 │ UInt64 │ 1 │ - 431. │ 430 │ UInt64 │ 0 │ - 432. │ 431 │ UInt64 │ 1 │ - 433. │ 432 │ UInt64 │ 0 │ - 434. │ 433 │ UInt64 │ 1 │ - 435. │ 434 │ UInt64 │ 0 │ - 436. │ 435 │ UInt64 │ 1 │ - 437. │ 436 │ UInt64 │ 0 │ - 438. │ 437 │ UInt64 │ 1 │ - 439. │ 438 │ UInt64 │ 0 │ - 440. │ 439 │ UInt64 │ 1 │ - 441. │ 440 │ UInt64 │ 0 │ - 442. │ 441 │ UInt64 │ 1 │ - 443. │ 442 │ UInt64 │ 0 │ - 444. │ 443 │ UInt64 │ 1 │ - 445. │ 444 │ UInt64 │ 0 │ - 446. │ 445 │ UInt64 │ 1 │ - 447. │ 446 │ UInt64 │ 0 │ - 448. │ 447 │ UInt64 │ 1 │ - 449. │ 448 │ UInt64 │ 0 │ - 450. │ 449 │ UInt64 │ 1 │ - 451. │ 450 │ UInt64 │ 0 │ - 452. │ 451 │ UInt64 │ 1 │ - 453. │ 452 │ UInt64 │ 0 │ - 454. │ 453 │ UInt64 │ 1 │ - 455. │ 454 │ UInt64 │ 0 │ - 456. │ 455 │ UInt64 │ 1 │ - 457. │ 456 │ UInt64 │ 0 │ - 458. │ 457 │ UInt64 │ 1 │ - 459. │ 458 │ UInt64 │ 0 │ - 460. │ 459 │ UInt64 │ 1 │ - 461. │ 460 │ UInt64 │ 0 │ - 462. │ 461 │ UInt64 │ 1 │ - 463. │ 462 │ UInt64 │ 0 │ - 464. │ 463 │ UInt64 │ 1 │ - 465. │ 464 │ UInt64 │ 0 │ - 466. │ 465 │ UInt64 │ 1 │ - 467. │ 466 │ UInt64 │ 0 │ - 468. │ 467 │ UInt64 │ 1 │ - 469. │ 468 │ UInt64 │ 0 │ - 470. │ 469 │ UInt64 │ 1 │ - 471. │ 470 │ UInt64 │ 0 │ - 472. │ 471 │ UInt64 │ 1 │ - 473. │ 472 │ UInt64 │ 0 │ - 474. │ 473 │ UInt64 │ 1 │ - 475. │ 474 │ UInt64 │ 0 │ - 476. │ 475 │ UInt64 │ 1 │ - 477. │ 476 │ UInt64 │ 0 │ - 478. │ 477 │ UInt64 │ 1 │ - 479. │ 478 │ UInt64 │ 0 │ - 480. │ 479 │ UInt64 │ 1 │ - 481. │ 480 │ UInt64 │ 0 │ - 482. │ 481 │ UInt64 │ 1 │ - 483. │ 482 │ UInt64 │ 0 │ - 484. │ 483 │ UInt64 │ 1 │ - 485. │ 484 │ UInt64 │ 0 │ - 486. │ 485 │ UInt64 │ 1 │ - 487. │ 486 │ UInt64 │ 0 │ - 488. │ 487 │ UInt64 │ 1 │ - 489. │ 488 │ UInt64 │ 0 │ - 490. │ 489 │ UInt64 │ 1 │ - 491. │ 490 │ UInt64 │ 0 │ - 492. │ 491 │ UInt64 │ 1 │ - 493. │ 492 │ UInt64 │ 0 │ - 494. │ 493 │ UInt64 │ 1 │ - 495. │ 494 │ UInt64 │ 0 │ - 496. │ 495 │ UInt64 │ 1 │ - 497. │ 496 │ UInt64 │ 0 │ - 498. │ 497 │ UInt64 │ 1 │ - 499. │ 498 │ UInt64 │ 0 │ - 500. │ 499 │ UInt64 │ 1 │ - 501. │ 500 │ UInt64 │ 0 │ - 502. │ 501 │ UInt64 │ 1 │ - 503. │ 502 │ UInt64 │ 0 │ - 504. │ 503 │ UInt64 │ 1 │ - 505. │ 504 │ UInt64 │ 0 │ - 506. │ 505 │ UInt64 │ 1 │ - 507. │ 506 │ UInt64 │ 0 │ - 508. │ 507 │ UInt64 │ 1 │ - 509. │ 508 │ UInt64 │ 0 │ - 510. │ 509 │ UInt64 │ 1 │ - 511. │ 510 │ UInt64 │ 0 │ - 512. │ 511 │ UInt64 │ 1 │ - 513. │ 512 │ UInt64 │ 0 │ - 514. │ 513 │ UInt64 │ 1 │ - 515. │ 514 │ UInt64 │ 0 │ - 516. │ 515 │ UInt64 │ 1 │ - 517. │ 516 │ UInt64 │ 0 │ - 518. │ 517 │ UInt64 │ 1 │ - 519. │ 518 │ UInt64 │ 0 │ - 520. │ 519 │ UInt64 │ 1 │ - 521. │ 520 │ UInt64 │ 0 │ - 522. │ 521 │ UInt64 │ 1 │ - 523. │ 522 │ UInt64 │ 0 │ - 524. │ 523 │ UInt64 │ 1 │ - 525. │ 524 │ UInt64 │ 0 │ - 526. │ 525 │ UInt64 │ 1 │ - 527. │ 526 │ UInt64 │ 0 │ - 528. │ 527 │ UInt64 │ 1 │ - 529. │ 528 │ UInt64 │ 0 │ - 530. │ 529 │ UInt64 │ 1 │ - 531. │ 530 │ UInt64 │ 0 │ - 532. │ 531 │ UInt64 │ 1 │ - 533. │ 532 │ UInt64 │ 0 │ - 534. │ 533 │ UInt64 │ 1 │ - 535. │ 534 │ UInt64 │ 0 │ - 536. │ 535 │ UInt64 │ 1 │ - 537. │ 536 │ UInt64 │ 0 │ - 538. │ 537 │ UInt64 │ 1 │ - 539. │ 538 │ UInt64 │ 0 │ - 540. │ 539 │ UInt64 │ 1 │ - 541. │ 540 │ UInt64 │ 0 │ - 542. │ 541 │ UInt64 │ 1 │ - 543. │ 542 │ UInt64 │ 0 │ - 544. │ 543 │ UInt64 │ 1 │ - 545. │ 544 │ UInt64 │ 0 │ - 546. │ 545 │ UInt64 │ 1 │ - 547. │ 546 │ UInt64 │ 0 │ - 548. │ 547 │ UInt64 │ 1 │ - 549. │ 548 │ UInt64 │ 0 │ - 550. │ 549 │ UInt64 │ 1 │ - 551. │ 550 │ UInt64 │ 0 │ - 552. │ 551 │ UInt64 │ 1 │ - 553. │ 552 │ UInt64 │ 0 │ - 554. │ 553 │ UInt64 │ 1 │ - 555. │ 554 │ UInt64 │ 0 │ - 556. │ 555 │ UInt64 │ 1 │ - 557. │ 556 │ UInt64 │ 0 │ - 558. │ 557 │ UInt64 │ 1 │ - 559. │ 558 │ UInt64 │ 0 │ - 560. │ 559 │ UInt64 │ 1 │ - 561. │ 560 │ UInt64 │ 0 │ - 562. │ 561 │ UInt64 │ 1 │ - 563. │ 562 │ UInt64 │ 0 │ - 564. │ 563 │ UInt64 │ 1 │ - 565. │ 564 │ UInt64 │ 0 │ - 566. │ 565 │ UInt64 │ 1 │ - 567. │ 566 │ UInt64 │ 0 │ - 568. │ 567 │ UInt64 │ 1 │ - 569. │ 568 │ UInt64 │ 0 │ - 570. │ 569 │ UInt64 │ 1 │ - 571. │ 570 │ UInt64 │ 0 │ - 572. │ 571 │ UInt64 │ 1 │ - 573. │ 572 │ UInt64 │ 0 │ - 574. │ 573 │ UInt64 │ 1 │ - 575. │ 574 │ UInt64 │ 0 │ - 576. │ 575 │ UInt64 │ 1 │ - 577. │ 576 │ UInt64 │ 0 │ - 578. │ 577 │ UInt64 │ 1 │ - 579. │ 578 │ UInt64 │ 0 │ - 580. │ 579 │ UInt64 │ 1 │ - 581. │ 580 │ UInt64 │ 0 │ - 582. │ 581 │ UInt64 │ 1 │ - 583. │ 582 │ UInt64 │ 0 │ - 584. │ 583 │ UInt64 │ 1 │ - 585. │ 584 │ UInt64 │ 0 │ - 586. │ 585 │ UInt64 │ 1 │ - 587. │ 586 │ UInt64 │ 0 │ - 588. │ 587 │ UInt64 │ 1 │ - 589. │ 588 │ UInt64 │ 0 │ - 590. │ 589 │ UInt64 │ 1 │ - 591. │ 590 │ UInt64 │ 0 │ - 592. │ 591 │ UInt64 │ 1 │ - 593. │ 592 │ UInt64 │ 0 │ - 594. │ 593 │ UInt64 │ 1 │ - 595. │ 594 │ UInt64 │ 0 │ - 596. │ 595 │ UInt64 │ 1 │ - 597. │ 596 │ UInt64 │ 0 │ - 598. │ 597 │ UInt64 │ 1 │ - 599. │ 598 │ UInt64 │ 0 │ - 600. │ 599 │ UInt64 │ 1 │ - 601. │ 600 │ UInt64 │ 0 │ - 602. │ 601 │ UInt64 │ 1 │ - 603. │ 602 │ UInt64 │ 0 │ - 604. │ 603 │ UInt64 │ 1 │ - 605. │ 604 │ UInt64 │ 0 │ - 606. │ 605 │ UInt64 │ 1 │ - 607. │ 606 │ UInt64 │ 0 │ - 608. │ 607 │ UInt64 │ 1 │ - 609. │ 608 │ UInt64 │ 0 │ - 610. │ 609 │ UInt64 │ 1 │ - 611. │ 610 │ UInt64 │ 0 │ - 612. │ 611 │ UInt64 │ 1 │ - 613. │ 612 │ UInt64 │ 0 │ - 614. │ 613 │ UInt64 │ 1 │ - 615. │ 614 │ UInt64 │ 0 │ - 616. │ 615 │ UInt64 │ 1 │ - 617. │ 616 │ UInt64 │ 0 │ - 618. │ 617 │ UInt64 │ 1 │ - 619. │ 618 │ UInt64 │ 0 │ - 620. │ 619 │ UInt64 │ 1 │ - 621. │ 620 │ UInt64 │ 0 │ - 622. │ 621 │ UInt64 │ 1 │ - 623. │ 622 │ UInt64 │ 0 │ - 624. │ 623 │ UInt64 │ 1 │ - 625. │ 624 │ UInt64 │ 0 │ - 626. │ 625 │ UInt64 │ 1 │ - 627. │ 626 │ UInt64 │ 0 │ - 628. │ 627 │ UInt64 │ 1 │ - 629. │ 628 │ UInt64 │ 0 │ - 630. │ 629 │ UInt64 │ 1 │ - 631. │ 630 │ UInt64 │ 0 │ - 632. │ 631 │ UInt64 │ 1 │ - 633. │ 632 │ UInt64 │ 0 │ - 634. │ 633 │ UInt64 │ 1 │ - 635. │ 634 │ UInt64 │ 0 │ - 636. │ 635 │ UInt64 │ 1 │ - 637. │ 636 │ UInt64 │ 0 │ - 638. │ 637 │ UInt64 │ 1 │ - 639. │ 638 │ UInt64 │ 0 │ - 640. │ 639 │ UInt64 │ 1 │ - 641. │ 640 │ UInt64 │ 0 │ - 642. │ 641 │ UInt64 │ 1 │ - 643. │ 642 │ UInt64 │ 0 │ - 644. │ 643 │ UInt64 │ 1 │ - 645. │ 644 │ UInt64 │ 0 │ - 646. │ 645 │ UInt64 │ 1 │ - 647. │ 646 │ UInt64 │ 0 │ - 648. │ 647 │ UInt64 │ 1 │ - 649. │ 648 │ UInt64 │ 0 │ - 650. │ 649 │ UInt64 │ 1 │ - 651. │ 650 │ UInt64 │ 0 │ - 652. │ 651 │ UInt64 │ 1 │ - 653. │ 652 │ UInt64 │ 0 │ - 654. │ 653 │ UInt64 │ 1 │ - 655. │ 654 │ UInt64 │ 0 │ - 656. │ 655 │ UInt64 │ 1 │ - 657. │ 656 │ UInt64 │ 0 │ - 658. │ 657 │ UInt64 │ 1 │ - 659. │ 658 │ UInt64 │ 0 │ - 660. │ 659 │ UInt64 │ 1 │ - 661. │ 660 │ UInt64 │ 0 │ - 662. │ 661 │ UInt64 │ 1 │ - 663. │ 662 │ UInt64 │ 0 │ - 664. │ 663 │ UInt64 │ 1 │ - 665. │ 664 │ UInt64 │ 0 │ - 666. │ 665 │ UInt64 │ 1 │ - 667. │ 666 │ UInt64 │ 0 │ - 668. │ 667 │ UInt64 │ 1 │ - 669. │ 668 │ UInt64 │ 0 │ - 670. │ 669 │ UInt64 │ 1 │ - 671. │ 670 │ UInt64 │ 0 │ - 672. │ 671 │ UInt64 │ 1 │ - 673. │ 672 │ UInt64 │ 0 │ - 674. │ 673 │ UInt64 │ 1 │ - 675. │ 674 │ UInt64 │ 0 │ - 676. │ 675 │ UInt64 │ 1 │ - 677. │ 676 │ UInt64 │ 0 │ - 678. │ 677 │ UInt64 │ 1 │ - 679. │ 678 │ UInt64 │ 0 │ - 680. │ 679 │ UInt64 │ 1 │ - 681. │ 680 │ UInt64 │ 0 │ - 682. │ 681 │ UInt64 │ 1 │ - 683. │ 682 │ UInt64 │ 0 │ - 684. │ 683 │ UInt64 │ 1 │ - 685. │ 684 │ UInt64 │ 0 │ - 686. │ 685 │ UInt64 │ 1 │ - 687. │ 686 │ UInt64 │ 0 │ - 688. │ 687 │ UInt64 │ 1 │ - 689. │ 688 │ UInt64 │ 0 │ - 690. │ 689 │ UInt64 │ 1 │ - 691. │ 690 │ UInt64 │ 0 │ - 692. │ 691 │ UInt64 │ 1 │ - 693. │ 692 │ UInt64 │ 0 │ - 694. │ 693 │ UInt64 │ 1 │ - 695. │ 694 │ UInt64 │ 0 │ - 696. │ 695 │ UInt64 │ 1 │ - 697. │ 696 │ UInt64 │ 0 │ - 698. │ 697 │ UInt64 │ 1 │ - 699. │ 698 │ UInt64 │ 0 │ - 700. │ 699 │ UInt64 │ 1 │ - 701. │ 700 │ UInt64 │ 0 │ - 702. │ 701 │ UInt64 │ 1 │ - 703. │ 702 │ UInt64 │ 0 │ - 704. │ 703 │ UInt64 │ 1 │ - 705. │ 704 │ UInt64 │ 0 │ - 706. │ 705 │ UInt64 │ 1 │ - 707. │ 706 │ UInt64 │ 0 │ - 708. │ 707 │ UInt64 │ 1 │ - 709. │ 708 │ UInt64 │ 0 │ - 710. │ 709 │ UInt64 │ 1 │ - 711. │ 710 │ UInt64 │ 0 │ - 712. │ 711 │ UInt64 │ 1 │ - 713. │ 712 │ UInt64 │ 0 │ - 714. │ 713 │ UInt64 │ 1 │ - 715. │ 714 │ UInt64 │ 0 │ - 716. │ 715 │ UInt64 │ 1 │ - 717. │ 716 │ UInt64 │ 0 │ - 718. │ 717 │ UInt64 │ 1 │ - 719. │ 718 │ UInt64 │ 0 │ - 720. │ 719 │ UInt64 │ 1 │ - 721. │ 720 │ UInt64 │ 0 │ - 722. │ 721 │ UInt64 │ 1 │ - 723. │ 722 │ UInt64 │ 0 │ - 724. │ 723 │ UInt64 │ 1 │ - 725. │ 724 │ UInt64 │ 0 │ - 726. │ 725 │ UInt64 │ 1 │ - 727. │ 726 │ UInt64 │ 0 │ - 728. │ 727 │ UInt64 │ 1 │ - 729. │ 728 │ UInt64 │ 0 │ - 730. │ 729 │ UInt64 │ 1 │ - 731. │ 730 │ UInt64 │ 0 │ - 732. │ 731 │ UInt64 │ 1 │ - 733. │ 732 │ UInt64 │ 0 │ - 734. │ 733 │ UInt64 │ 1 │ - 735. │ 734 │ UInt64 │ 0 │ - 736. │ 735 │ UInt64 │ 1 │ - 737. │ 736 │ UInt64 │ 0 │ - 738. │ 737 │ UInt64 │ 1 │ - 739. │ 738 │ UInt64 │ 0 │ - 740. │ 739 │ UInt64 │ 1 │ - 741. │ 740 │ UInt64 │ 0 │ - 742. │ 741 │ UInt64 │ 1 │ - 743. │ 742 │ UInt64 │ 0 │ - 744. │ 743 │ UInt64 │ 1 │ - 745. │ 744 │ UInt64 │ 0 │ - 746. │ 745 │ UInt64 │ 1 │ - 747. │ 746 │ UInt64 │ 0 │ - 748. │ 747 │ UInt64 │ 1 │ - 749. │ 748 │ UInt64 │ 0 │ - 750. │ 749 │ UInt64 │ 1 │ - 751. │ 750 │ UInt64 │ 0 │ - 752. │ 751 │ UInt64 │ 1 │ - 753. │ 752 │ UInt64 │ 0 │ - 754. │ 753 │ UInt64 │ 1 │ - 755. │ 754 │ UInt64 │ 0 │ - 756. │ 755 │ UInt64 │ 1 │ - 757. │ 756 │ UInt64 │ 0 │ - 758. │ 757 │ UInt64 │ 1 │ - 759. │ 758 │ UInt64 │ 0 │ - 760. │ 759 │ UInt64 │ 1 │ - 761. │ 760 │ UInt64 │ 0 │ - 762. │ 761 │ UInt64 │ 1 │ - 763. │ 762 │ UInt64 │ 0 │ - 764. │ 763 │ UInt64 │ 1 │ - 765. │ 764 │ UInt64 │ 0 │ - 766. │ 765 │ UInt64 │ 1 │ - 767. │ 766 │ UInt64 │ 0 │ - 768. │ 767 │ UInt64 │ 1 │ - 769. │ 768 │ UInt64 │ 0 │ - 770. │ 769 │ UInt64 │ 1 │ - 771. │ 770 │ UInt64 │ 0 │ - 772. │ 771 │ UInt64 │ 1 │ - 773. │ 772 │ UInt64 │ 0 │ - 774. │ 773 │ UInt64 │ 1 │ - 775. │ 774 │ UInt64 │ 0 │ - 776. │ 775 │ UInt64 │ 1 │ - 777. │ 776 │ UInt64 │ 0 │ - 778. │ 777 │ UInt64 │ 1 │ - 779. │ 778 │ UInt64 │ 0 │ - 780. │ 779 │ UInt64 │ 1 │ - 781. │ 780 │ UInt64 │ 0 │ - 782. │ 781 │ UInt64 │ 1 │ - 783. │ 782 │ UInt64 │ 0 │ - 784. │ 783 │ UInt64 │ 1 │ - 785. │ 784 │ UInt64 │ 0 │ - 786. │ 785 │ UInt64 │ 1 │ - 787. │ 786 │ UInt64 │ 0 │ - 788. │ 787 │ UInt64 │ 1 │ - 789. │ 788 │ UInt64 │ 0 │ - 790. │ 789 │ UInt64 │ 1 │ - 791. │ 790 │ UInt64 │ 0 │ - 792. │ 791 │ UInt64 │ 1 │ - 793. │ 792 │ UInt64 │ 0 │ - 794. │ 793 │ UInt64 │ 1 │ - 795. │ 794 │ UInt64 │ 0 │ - 796. │ 795 │ UInt64 │ 1 │ - 797. │ 796 │ UInt64 │ 0 │ - 798. │ 797 │ UInt64 │ 1 │ - 799. │ 798 │ UInt64 │ 0 │ - 800. │ 799 │ UInt64 │ 1 │ - 801. │ 800 │ UInt64 │ 0 │ - 802. │ 801 │ UInt64 │ 1 │ - 803. │ 802 │ UInt64 │ 0 │ - 804. │ 803 │ UInt64 │ 1 │ - 805. │ 804 │ UInt64 │ 0 │ - 806. │ 805 │ UInt64 │ 1 │ - 807. │ 806 │ UInt64 │ 0 │ - 808. │ 807 │ UInt64 │ 1 │ - 809. │ 808 │ UInt64 │ 0 │ - 810. │ 809 │ UInt64 │ 1 │ - 811. │ 810 │ UInt64 │ 0 │ - 812. │ 811 │ UInt64 │ 1 │ - 813. │ 812 │ UInt64 │ 0 │ - 814. │ 813 │ UInt64 │ 1 │ - 815. │ 814 │ UInt64 │ 0 │ - 816. │ 815 │ UInt64 │ 1 │ - 817. │ 816 │ UInt64 │ 0 │ - 818. │ 817 │ UInt64 │ 1 │ - 819. │ 818 │ UInt64 │ 0 │ - 820. │ 819 │ UInt64 │ 1 │ - 821. │ 820 │ UInt64 │ 0 │ - 822. │ 821 │ UInt64 │ 1 │ - 823. │ 822 │ UInt64 │ 0 │ - 824. │ 823 │ UInt64 │ 1 │ - 825. │ 824 │ UInt64 │ 0 │ - 826. │ 825 │ UInt64 │ 1 │ - 827. │ 826 │ UInt64 │ 0 │ - 828. │ 827 │ UInt64 │ 1 │ - 829. │ 828 │ UInt64 │ 0 │ - 830. │ 829 │ UInt64 │ 1 │ - 831. │ 830 │ UInt64 │ 0 │ - 832. │ 831 │ UInt64 │ 1 │ - 833. │ 832 │ UInt64 │ 0 │ - 834. │ 833 │ UInt64 │ 1 │ - 835. │ 834 │ UInt64 │ 0 │ - 836. │ 835 │ UInt64 │ 1 │ - 837. │ 836 │ UInt64 │ 0 │ - 838. │ 837 │ UInt64 │ 1 │ - 839. │ 838 │ UInt64 │ 0 │ - 840. │ 839 │ UInt64 │ 1 │ - 841. │ 840 │ UInt64 │ 0 │ - 842. │ 841 │ UInt64 │ 1 │ - 843. │ 842 │ UInt64 │ 0 │ - 844. │ 843 │ UInt64 │ 1 │ - 845. │ 844 │ UInt64 │ 0 │ - 846. │ 845 │ UInt64 │ 1 │ - 847. │ 846 │ UInt64 │ 0 │ - 848. │ 847 │ UInt64 │ 1 │ - 849. │ 848 │ UInt64 │ 0 │ - 850. │ 849 │ UInt64 │ 1 │ - 851. │ 850 │ UInt64 │ 0 │ - 852. │ 851 │ UInt64 │ 1 │ - 853. │ 852 │ UInt64 │ 0 │ - 854. │ 853 │ UInt64 │ 1 │ - 855. │ 854 │ UInt64 │ 0 │ - 856. │ 855 │ UInt64 │ 1 │ - 857. │ 856 │ UInt64 │ 0 │ - 858. │ 857 │ UInt64 │ 1 │ - 859. │ 858 │ UInt64 │ 0 │ - 860. │ 859 │ UInt64 │ 1 │ - 861. │ 860 │ UInt64 │ 0 │ - 862. │ 861 │ UInt64 │ 1 │ - 863. │ 862 │ UInt64 │ 0 │ - 864. │ 863 │ UInt64 │ 1 │ - 865. │ 864 │ UInt64 │ 0 │ - 866. │ 865 │ UInt64 │ 1 │ - 867. │ 866 │ UInt64 │ 0 │ - 868. │ 867 │ UInt64 │ 1 │ - 869. │ 868 │ UInt64 │ 0 │ - 870. │ 869 │ UInt64 │ 1 │ - 871. │ 870 │ UInt64 │ 0 │ - 872. │ 871 │ UInt64 │ 1 │ - 873. │ 872 │ UInt64 │ 0 │ - 874. │ 873 │ UInt64 │ 1 │ - 875. │ 874 │ UInt64 │ 0 │ - 876. │ 875 │ UInt64 │ 1 │ - 877. │ 876 │ UInt64 │ 0 │ - 878. │ 877 │ UInt64 │ 1 │ - 879. │ 878 │ UInt64 │ 0 │ - 880. │ 879 │ UInt64 │ 1 │ - 881. │ 880 │ UInt64 │ 0 │ - 882. │ 881 │ UInt64 │ 1 │ - 883. │ 882 │ UInt64 │ 0 │ - 884. │ 883 │ UInt64 │ 1 │ - 885. │ 884 │ UInt64 │ 0 │ - 886. │ 885 │ UInt64 │ 1 │ - 887. │ 886 │ UInt64 │ 0 │ - 888. │ 887 │ UInt64 │ 1 │ - 889. │ 888 │ UInt64 │ 0 │ - 890. │ 889 │ UInt64 │ 1 │ - 891. │ 890 │ UInt64 │ 0 │ - 892. │ 891 │ UInt64 │ 1 │ - 893. │ 892 │ UInt64 │ 0 │ - 894. │ 893 │ UInt64 │ 1 │ - 895. │ 894 │ UInt64 │ 0 │ - 896. │ 895 │ UInt64 │ 1 │ - 897. │ 896 │ UInt64 │ 0 │ - 898. │ 897 │ UInt64 │ 1 │ - 899. │ 898 │ UInt64 │ 0 │ - 900. │ 899 │ UInt64 │ 1 │ - 901. │ 900 │ UInt64 │ 0 │ - 902. │ 901 │ UInt64 │ 1 │ - 903. │ 902 │ UInt64 │ 0 │ - 904. │ 903 │ UInt64 │ 1 │ - 905. │ 904 │ UInt64 │ 0 │ - 906. │ 905 │ UInt64 │ 1 │ - 907. │ 906 │ UInt64 │ 0 │ - 908. │ 907 │ UInt64 │ 1 │ - 909. │ 908 │ UInt64 │ 0 │ - 910. │ 909 │ UInt64 │ 1 │ - 911. │ 910 │ UInt64 │ 0 │ - 912. │ 911 │ UInt64 │ 1 │ - 913. │ 912 │ UInt64 │ 0 │ - 914. │ 913 │ UInt64 │ 1 │ - 915. │ 914 │ UInt64 │ 0 │ - 916. │ 915 │ UInt64 │ 1 │ - 917. │ 916 │ UInt64 │ 0 │ - 918. │ 917 │ UInt64 │ 1 │ - 919. │ 918 │ UInt64 │ 0 │ - 920. │ 919 │ UInt64 │ 1 │ - 921. │ 920 │ UInt64 │ 0 │ - 922. │ 921 │ UInt64 │ 1 │ - 923. │ 922 │ UInt64 │ 0 │ - 924. │ 923 │ UInt64 │ 1 │ - 925. │ 924 │ UInt64 │ 0 │ - 926. │ 925 │ UInt64 │ 1 │ - 927. │ 926 │ UInt64 │ 0 │ - 928. │ 927 │ UInt64 │ 1 │ - 929. │ 928 │ UInt64 │ 0 │ - 930. │ 929 │ UInt64 │ 1 │ - 931. │ 930 │ UInt64 │ 0 │ - 932. │ 931 │ UInt64 │ 1 │ - 933. │ 932 │ UInt64 │ 0 │ - 934. │ 933 │ UInt64 │ 1 │ - 935. │ 934 │ UInt64 │ 0 │ - 936. │ 935 │ UInt64 │ 1 │ - 937. │ 936 │ UInt64 │ 0 │ - 938. │ 937 │ UInt64 │ 1 │ - 939. │ 938 │ UInt64 │ 0 │ - 940. │ 939 │ UInt64 │ 1 │ - 941. │ 940 │ UInt64 │ 0 │ - 942. │ 941 │ UInt64 │ 1 │ - 943. │ 942 │ UInt64 │ 0 │ - 944. │ 943 │ UInt64 │ 1 │ - 945. │ 944 │ UInt64 │ 0 │ - 946. │ 945 │ UInt64 │ 1 │ - 947. │ 946 │ UInt64 │ 0 │ - 948. │ 947 │ UInt64 │ 1 │ - 949. │ 948 │ UInt64 │ 0 │ - 950. │ 949 │ UInt64 │ 1 │ - 951. │ 950 │ UInt64 │ 0 │ - 952. │ 951 │ UInt64 │ 1 │ - 953. │ 952 │ UInt64 │ 0 │ - 954. │ 953 │ UInt64 │ 1 │ - 955. │ 954 │ UInt64 │ 0 │ - 956. │ 955 │ UInt64 │ 1 │ - 957. │ 956 │ UInt64 │ 0 │ - 958. │ 957 │ UInt64 │ 1 │ - 959. │ 958 │ UInt64 │ 0 │ - 960. │ 959 │ UInt64 │ 1 │ - 961. │ 960 │ UInt64 │ 0 │ - 962. │ 961 │ UInt64 │ 1 │ - 963. │ 962 │ UInt64 │ 0 │ - 964. │ 963 │ UInt64 │ 1 │ - 965. │ 964 │ UInt64 │ 0 │ - 966. │ 965 │ UInt64 │ 1 │ - 967. │ 966 │ UInt64 │ 0 │ - 968. │ 967 │ UInt64 │ 1 │ - 969. │ 968 │ UInt64 │ 0 │ - 970. │ 969 │ UInt64 │ 1 │ - 971. │ 970 │ UInt64 │ 0 │ - 972. │ 971 │ UInt64 │ 1 │ - 973. │ 972 │ UInt64 │ 0 │ - 974. │ 973 │ UInt64 │ 1 │ - 975. │ 974 │ UInt64 │ 0 │ - 976. │ 975 │ UInt64 │ 1 │ - 977. │ 976 │ UInt64 │ 0 │ - 978. │ 977 │ UInt64 │ 1 │ - 979. │ 978 │ UInt64 │ 0 │ - 980. │ 979 │ UInt64 │ 1 │ - 981. │ 980 │ UInt64 │ 0 │ - 982. │ 981 │ UInt64 │ 1 │ - 983. │ 982 │ UInt64 │ 0 │ - 984. │ 983 │ UInt64 │ 1 │ - 985. │ 984 │ UInt64 │ 0 │ - 986. │ 985 │ UInt64 │ 1 │ - 987. │ 986 │ UInt64 │ 0 │ - 988. │ 987 │ UInt64 │ 1 │ - 989. │ 988 │ UInt64 │ 0 │ - 990. │ 989 │ UInt64 │ 1 │ - 991. │ 990 │ UInt64 │ 0 │ - 992. │ 991 │ UInt64 │ 1 │ - 993. │ 992 │ UInt64 │ 0 │ - 994. │ 993 │ UInt64 │ 1 │ - 995. │ 994 │ UInt64 │ 0 │ - 996. │ 995 │ UInt64 │ 1 │ - 997. │ 996 │ UInt64 │ 0 │ - 998. │ 997 │ UInt64 │ 1 │ - 999. │ 998 │ UInt64 │ 0 │ -1000. │ 999 │ UInt64 │ 1 │ - └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ - number toTypeName(number) mod(number, 2) + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ +49. │ 48 │ UInt64 │ 0 │ + └────────┴────────────────────┴────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +10. │ 9 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +100. │ 99 │ UInt64 │ 1 │ + └────────┴────────────────────┴────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +100. │ 99 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +100. │ 99 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +100. │ 99 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +100. │ 99 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 2. │ 1 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 3. │ 2 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 4. │ 3 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 5. │ 4 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 6. │ 5 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 7. │ 6 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 8. │ 7 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 9. │ 8 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 10. │ 9 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 11. │ 10 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 12. │ 11 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 13. │ 12 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 14. │ 13 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 15. │ 14 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 16. │ 15 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 17. │ 16 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 18. │ 17 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 19. │ 18 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 20. │ 19 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 21. │ 20 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 22. │ 21 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 23. │ 22 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 24. │ 23 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 25. │ 24 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 26. │ 25 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 27. │ 26 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 28. │ 27 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 29. │ 28 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 30. │ 29 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 31. │ 30 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 32. │ 31 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 33. │ 32 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 34. │ 33 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 35. │ 34 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 36. │ 35 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 37. │ 36 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 38. │ 37 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 39. │ 38 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 40. │ 39 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 41. │ 40 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 42. │ 41 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 43. │ 42 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 44. │ 43 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 45. │ 44 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 46. │ 45 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 47. │ 46 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 48. │ 47 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 49. │ 48 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 50. │ 49 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 51. │ 50 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 52. │ 51 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 53. │ 52 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 54. │ 53 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 55. │ 54 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 56. │ 55 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 57. │ 56 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 58. │ 57 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 59. │ 58 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 60. │ 59 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 61. │ 60 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 62. │ 61 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 63. │ 62 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 64. │ 63 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 65. │ 64 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 66. │ 65 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 67. │ 66 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 68. │ 67 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 69. │ 68 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 70. │ 69 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 71. │ 70 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 72. │ 71 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 73. │ 72 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 74. │ 73 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 75. │ 74 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 76. │ 75 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 77. │ 76 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 78. │ 77 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 79. │ 78 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 80. │ 79 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 81. │ 80 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 82. │ 81 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 83. │ 82 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 84. │ 83 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 85. │ 84 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 86. │ 85 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 87. │ 86 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 88. │ 87 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 89. │ 88 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 90. │ 89 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 91. │ 90 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 92. │ 91 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 93. │ 92 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 94. │ 93 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 95. │ 94 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 96. │ 95 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 97. │ 96 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ + 98. │ 97 │ UInt64 │ 1 │ + ├────────┼────────────────────┼────────────────┤ + 99. │ 98 │ UInt64 │ 0 │ + ├────────┼────────────────────┼────────────────┤ +100. │ 99 │ UInt64 │ 1 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ +100. │ 99 │ UInt64 │ 1 │ + └────────┴────────────────────┴────────────────┘ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ +100. │ 99 │ UInt64 │ 1 │ + └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ +100. │ 99 │ UInt64 │ 1 │ + └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ +100. │ 99 │ UInt64 │ 1 │ + └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ + ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ + 1. │ 0 │ UInt64 │ 0 │ + 2. │ 1 │ UInt64 │ 1 │ + 3. │ 2 │ UInt64 │ 0 │ + 4. │ 3 │ UInt64 │ 1 │ + 5. │ 4 │ UInt64 │ 0 │ + 6. │ 5 │ UInt64 │ 1 │ + 7. │ 6 │ UInt64 │ 0 │ + 8. │ 7 │ UInt64 │ 1 │ + 9. │ 8 │ UInt64 │ 0 │ + 10. │ 9 │ UInt64 │ 1 │ + 11. │ 10 │ UInt64 │ 0 │ + 12. │ 11 │ UInt64 │ 1 │ + 13. │ 12 │ UInt64 │ 0 │ + 14. │ 13 │ UInt64 │ 1 │ + 15. │ 14 │ UInt64 │ 0 │ + 16. │ 15 │ UInt64 │ 1 │ + 17. │ 16 │ UInt64 │ 0 │ + 18. │ 17 │ UInt64 │ 1 │ + 19. │ 18 │ UInt64 │ 0 │ + 20. │ 19 │ UInt64 │ 1 │ + 21. │ 20 │ UInt64 │ 0 │ + 22. │ 21 │ UInt64 │ 1 │ + 23. │ 22 │ UInt64 │ 0 │ + 24. │ 23 │ UInt64 │ 1 │ + 25. │ 24 │ UInt64 │ 0 │ + 26. │ 25 │ UInt64 │ 1 │ + 27. │ 26 │ UInt64 │ 0 │ + 28. │ 27 │ UInt64 │ 1 │ + 29. │ 28 │ UInt64 │ 0 │ + 30. │ 29 │ UInt64 │ 1 │ + 31. │ 30 │ UInt64 │ 0 │ + 32. │ 31 │ UInt64 │ 1 │ + 33. │ 32 │ UInt64 │ 0 │ + 34. │ 33 │ UInt64 │ 1 │ + 35. │ 34 │ UInt64 │ 0 │ + 36. │ 35 │ UInt64 │ 1 │ + 37. │ 36 │ UInt64 │ 0 │ + 38. │ 37 │ UInt64 │ 1 │ + 39. │ 38 │ UInt64 │ 0 │ + 40. │ 39 │ UInt64 │ 1 │ + 41. │ 40 │ UInt64 │ 0 │ + 42. │ 41 │ UInt64 │ 1 │ + 43. │ 42 │ UInt64 │ 0 │ + 44. │ 43 │ UInt64 │ 1 │ + 45. │ 44 │ UInt64 │ 0 │ + 46. │ 45 │ UInt64 │ 1 │ + 47. │ 46 │ UInt64 │ 0 │ + 48. │ 47 │ UInt64 │ 1 │ + 49. │ 48 │ UInt64 │ 0 │ + 50. │ 49 │ UInt64 │ 1 │ + 51. │ 50 │ UInt64 │ 0 │ + 52. │ 51 │ UInt64 │ 1 │ + 53. │ 52 │ UInt64 │ 0 │ + 54. │ 53 │ UInt64 │ 1 │ + 55. │ 54 │ UInt64 │ 0 │ + 56. │ 55 │ UInt64 │ 1 │ + 57. │ 56 │ UInt64 │ 0 │ + 58. │ 57 │ UInt64 │ 1 │ + 59. │ 58 │ UInt64 │ 0 │ + 60. │ 59 │ UInt64 │ 1 │ + 61. │ 60 │ UInt64 │ 0 │ + 62. │ 61 │ UInt64 │ 1 │ + 63. │ 62 │ UInt64 │ 0 │ + 64. │ 63 │ UInt64 │ 1 │ + 65. │ 64 │ UInt64 │ 0 │ + 66. │ 65 │ UInt64 │ 1 │ + 67. │ 66 │ UInt64 │ 0 │ + 68. │ 67 │ UInt64 │ 1 │ + 69. │ 68 │ UInt64 │ 0 │ + 70. │ 69 │ UInt64 │ 1 │ + 71. │ 70 │ UInt64 │ 0 │ + 72. │ 71 │ UInt64 │ 1 │ + 73. │ 72 │ UInt64 │ 0 │ + 74. │ 73 │ UInt64 │ 1 │ + 75. │ 74 │ UInt64 │ 0 │ + 76. │ 75 │ UInt64 │ 1 │ + 77. │ 76 │ UInt64 │ 0 │ + 78. │ 77 │ UInt64 │ 1 │ + 79. │ 78 │ UInt64 │ 0 │ + 80. │ 79 │ UInt64 │ 1 │ + 81. │ 80 │ UInt64 │ 0 │ + 82. │ 81 │ UInt64 │ 1 │ + 83. │ 82 │ UInt64 │ 0 │ + 84. │ 83 │ UInt64 │ 1 │ + 85. │ 84 │ UInt64 │ 0 │ + 86. │ 85 │ UInt64 │ 1 │ + 87. │ 86 │ UInt64 │ 0 │ + 88. │ 87 │ UInt64 │ 1 │ + 89. │ 88 │ UInt64 │ 0 │ + 90. │ 89 │ UInt64 │ 1 │ + 91. │ 90 │ UInt64 │ 0 │ + 92. │ 91 │ UInt64 │ 1 │ + 93. │ 92 │ UInt64 │ 0 │ + 94. │ 93 │ UInt64 │ 1 │ + 95. │ 94 │ UInt64 │ 0 │ + 96. │ 95 │ UInt64 │ 1 │ + 97. │ 96 │ UInt64 │ 0 │ + 98. │ 97 │ UInt64 │ 1 │ + 99. │ 98 │ UInt64 │ 0 │ +100. │ 99 │ UInt64 │ 1 │ + └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ + number toTypeName(number) mod(number, 2) - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 - 100. 99 UInt64 1 - 101. 100 UInt64 0 - 102. 101 UInt64 1 - 103. 102 UInt64 0 - 104. 103 UInt64 1 - 105. 104 UInt64 0 - 106. 105 UInt64 1 - 107. 106 UInt64 0 - 108. 107 UInt64 1 - 109. 108 UInt64 0 - 110. 109 UInt64 1 - 111. 110 UInt64 0 - 112. 111 UInt64 1 - 113. 112 UInt64 0 - 114. 113 UInt64 1 - 115. 114 UInt64 0 - 116. 115 UInt64 1 - 117. 116 UInt64 0 - 118. 117 UInt64 1 - 119. 118 UInt64 0 - 120. 119 UInt64 1 - 121. 120 UInt64 0 - 122. 121 UInt64 1 - 123. 122 UInt64 0 - 124. 123 UInt64 1 - 125. 124 UInt64 0 - 126. 125 UInt64 1 - 127. 126 UInt64 0 - 128. 127 UInt64 1 - 129. 128 UInt64 0 - 130. 129 UInt64 1 - 131. 130 UInt64 0 - 132. 131 UInt64 1 - 133. 132 UInt64 0 - 134. 133 UInt64 1 - 135. 134 UInt64 0 - 136. 135 UInt64 1 - 137. 136 UInt64 0 - 138. 137 UInt64 1 - 139. 138 UInt64 0 - 140. 139 UInt64 1 - 141. 140 UInt64 0 - 142. 141 UInt64 1 - 143. 142 UInt64 0 - 144. 143 UInt64 1 - 145. 144 UInt64 0 - 146. 145 UInt64 1 - 147. 146 UInt64 0 - 148. 147 UInt64 1 - 149. 148 UInt64 0 - 150. 149 UInt64 1 - 151. 150 UInt64 0 - 152. 151 UInt64 1 - 153. 152 UInt64 0 - 154. 153 UInt64 1 - 155. 154 UInt64 0 - 156. 155 UInt64 1 - 157. 156 UInt64 0 - 158. 157 UInt64 1 - 159. 158 UInt64 0 - 160. 159 UInt64 1 - 161. 160 UInt64 0 - 162. 161 UInt64 1 - 163. 162 UInt64 0 - 164. 163 UInt64 1 - 165. 164 UInt64 0 - 166. 165 UInt64 1 - 167. 166 UInt64 0 - 168. 167 UInt64 1 - 169. 168 UInt64 0 - 170. 169 UInt64 1 - 171. 170 UInt64 0 - 172. 171 UInt64 1 - 173. 172 UInt64 0 - 174. 173 UInt64 1 - 175. 174 UInt64 0 - 176. 175 UInt64 1 - 177. 176 UInt64 0 - 178. 177 UInt64 1 - 179. 178 UInt64 0 - 180. 179 UInt64 1 - 181. 180 UInt64 0 - 182. 181 UInt64 1 - 183. 182 UInt64 0 - 184. 183 UInt64 1 - 185. 184 UInt64 0 - 186. 185 UInt64 1 - 187. 186 UInt64 0 - 188. 187 UInt64 1 - 189. 188 UInt64 0 - 190. 189 UInt64 1 - 191. 190 UInt64 0 - 192. 191 UInt64 1 - 193. 192 UInt64 0 - 194. 193 UInt64 1 - 195. 194 UInt64 0 - 196. 195 UInt64 1 - 197. 196 UInt64 0 - 198. 197 UInt64 1 - 199. 198 UInt64 0 - 200. 199 UInt64 1 - 201. 200 UInt64 0 - 202. 201 UInt64 1 - 203. 202 UInt64 0 - 204. 203 UInt64 1 - 205. 204 UInt64 0 - 206. 205 UInt64 1 - 207. 206 UInt64 0 - 208. 207 UInt64 1 - 209. 208 UInt64 0 - 210. 209 UInt64 1 - 211. 210 UInt64 0 - 212. 211 UInt64 1 - 213. 212 UInt64 0 - 214. 213 UInt64 1 - 215. 214 UInt64 0 - 216. 215 UInt64 1 - 217. 216 UInt64 0 - 218. 217 UInt64 1 - 219. 218 UInt64 0 - 220. 219 UInt64 1 - 221. 220 UInt64 0 - 222. 221 UInt64 1 - 223. 222 UInt64 0 - 224. 223 UInt64 1 - 225. 224 UInt64 0 - 226. 225 UInt64 1 - 227. 226 UInt64 0 - 228. 227 UInt64 1 - 229. 228 UInt64 0 - 230. 229 UInt64 1 - 231. 230 UInt64 0 - 232. 231 UInt64 1 - 233. 232 UInt64 0 - 234. 233 UInt64 1 - 235. 234 UInt64 0 - 236. 235 UInt64 1 - 237. 236 UInt64 0 - 238. 237 UInt64 1 - 239. 238 UInt64 0 - 240. 239 UInt64 1 - 241. 240 UInt64 0 - 242. 241 UInt64 1 - 243. 242 UInt64 0 - 244. 243 UInt64 1 - 245. 244 UInt64 0 - 246. 245 UInt64 1 - 247. 246 UInt64 0 - 248. 247 UInt64 1 - 249. 248 UInt64 0 - 250. 249 UInt64 1 - 251. 250 UInt64 0 - 252. 251 UInt64 1 - 253. 252 UInt64 0 - 254. 253 UInt64 1 - 255. 254 UInt64 0 - 256. 255 UInt64 1 - 257. 256 UInt64 0 - 258. 257 UInt64 1 - 259. 258 UInt64 0 - 260. 259 UInt64 1 - 261. 260 UInt64 0 - 262. 261 UInt64 1 - 263. 262 UInt64 0 - 264. 263 UInt64 1 - 265. 264 UInt64 0 - 266. 265 UInt64 1 - 267. 266 UInt64 0 - 268. 267 UInt64 1 - 269. 268 UInt64 0 - 270. 269 UInt64 1 - 271. 270 UInt64 0 - 272. 271 UInt64 1 - 273. 272 UInt64 0 - 274. 273 UInt64 1 - 275. 274 UInt64 0 - 276. 275 UInt64 1 - 277. 276 UInt64 0 - 278. 277 UInt64 1 - 279. 278 UInt64 0 - 280. 279 UInt64 1 - 281. 280 UInt64 0 - 282. 281 UInt64 1 - 283. 282 UInt64 0 - 284. 283 UInt64 1 - 285. 284 UInt64 0 - 286. 285 UInt64 1 - 287. 286 UInt64 0 - 288. 287 UInt64 1 - 289. 288 UInt64 0 - 290. 289 UInt64 1 - 291. 290 UInt64 0 - 292. 291 UInt64 1 - 293. 292 UInt64 0 - 294. 293 UInt64 1 - 295. 294 UInt64 0 - 296. 295 UInt64 1 - 297. 296 UInt64 0 - 298. 297 UInt64 1 - 299. 298 UInt64 0 - 300. 299 UInt64 1 - 301. 300 UInt64 0 - 302. 301 UInt64 1 - 303. 302 UInt64 0 - 304. 303 UInt64 1 - 305. 304 UInt64 0 - 306. 305 UInt64 1 - 307. 306 UInt64 0 - 308. 307 UInt64 1 - 309. 308 UInt64 0 - 310. 309 UInt64 1 - 311. 310 UInt64 0 - 312. 311 UInt64 1 - 313. 312 UInt64 0 - 314. 313 UInt64 1 - 315. 314 UInt64 0 - 316. 315 UInt64 1 - 317. 316 UInt64 0 - 318. 317 UInt64 1 - 319. 318 UInt64 0 - 320. 319 UInt64 1 - 321. 320 UInt64 0 - 322. 321 UInt64 1 - 323. 322 UInt64 0 - 324. 323 UInt64 1 - 325. 324 UInt64 0 - 326. 325 UInt64 1 - 327. 326 UInt64 0 - 328. 327 UInt64 1 - 329. 328 UInt64 0 - 330. 329 UInt64 1 - 331. 330 UInt64 0 - 332. 331 UInt64 1 - 333. 332 UInt64 0 - 334. 333 UInt64 1 - 335. 334 UInt64 0 - 336. 335 UInt64 1 - 337. 336 UInt64 0 - 338. 337 UInt64 1 - 339. 338 UInt64 0 - 340. 339 UInt64 1 - 341. 340 UInt64 0 - 342. 341 UInt64 1 - 343. 342 UInt64 0 - 344. 343 UInt64 1 - 345. 344 UInt64 0 - 346. 345 UInt64 1 - 347. 346 UInt64 0 - 348. 347 UInt64 1 - 349. 348 UInt64 0 - 350. 349 UInt64 1 - 351. 350 UInt64 0 - 352. 351 UInt64 1 - 353. 352 UInt64 0 - 354. 353 UInt64 1 - 355. 354 UInt64 0 - 356. 355 UInt64 1 - 357. 356 UInt64 0 - 358. 357 UInt64 1 - 359. 358 UInt64 0 - 360. 359 UInt64 1 - 361. 360 UInt64 0 - 362. 361 UInt64 1 - 363. 362 UInt64 0 - 364. 363 UInt64 1 - 365. 364 UInt64 0 - 366. 365 UInt64 1 - 367. 366 UInt64 0 - 368. 367 UInt64 1 - 369. 368 UInt64 0 - 370. 369 UInt64 1 - 371. 370 UInt64 0 - 372. 371 UInt64 1 - 373. 372 UInt64 0 - 374. 373 UInt64 1 - 375. 374 UInt64 0 - 376. 375 UInt64 1 - 377. 376 UInt64 0 - 378. 377 UInt64 1 - 379. 378 UInt64 0 - 380. 379 UInt64 1 - 381. 380 UInt64 0 - 382. 381 UInt64 1 - 383. 382 UInt64 0 - 384. 383 UInt64 1 - 385. 384 UInt64 0 - 386. 385 UInt64 1 - 387. 386 UInt64 0 - 388. 387 UInt64 1 - 389. 388 UInt64 0 - 390. 389 UInt64 1 - 391. 390 UInt64 0 - 392. 391 UInt64 1 - 393. 392 UInt64 0 - 394. 393 UInt64 1 - 395. 394 UInt64 0 - 396. 395 UInt64 1 - 397. 396 UInt64 0 - 398. 397 UInt64 1 - 399. 398 UInt64 0 - 400. 399 UInt64 1 - 401. 400 UInt64 0 - 402. 401 UInt64 1 - 403. 402 UInt64 0 - 404. 403 UInt64 1 - 405. 404 UInt64 0 - 406. 405 UInt64 1 - 407. 406 UInt64 0 - 408. 407 UInt64 1 - 409. 408 UInt64 0 - 410. 409 UInt64 1 - 411. 410 UInt64 0 - 412. 411 UInt64 1 - 413. 412 UInt64 0 - 414. 413 UInt64 1 - 415. 414 UInt64 0 - 416. 415 UInt64 1 - 417. 416 UInt64 0 - 418. 417 UInt64 1 - 419. 418 UInt64 0 - 420. 419 UInt64 1 - 421. 420 UInt64 0 - 422. 421 UInt64 1 - 423. 422 UInt64 0 - 424. 423 UInt64 1 - 425. 424 UInt64 0 - 426. 425 UInt64 1 - 427. 426 UInt64 0 - 428. 427 UInt64 1 - 429. 428 UInt64 0 - 430. 429 UInt64 1 - 431. 430 UInt64 0 - 432. 431 UInt64 1 - 433. 432 UInt64 0 - 434. 433 UInt64 1 - 435. 434 UInt64 0 - 436. 435 UInt64 1 - 437. 436 UInt64 0 - 438. 437 UInt64 1 - 439. 438 UInt64 0 - 440. 439 UInt64 1 - 441. 440 UInt64 0 - 442. 441 UInt64 1 - 443. 442 UInt64 0 - 444. 443 UInt64 1 - 445. 444 UInt64 0 - 446. 445 UInt64 1 - 447. 446 UInt64 0 - 448. 447 UInt64 1 - 449. 448 UInt64 0 - 450. 449 UInt64 1 - 451. 450 UInt64 0 - 452. 451 UInt64 1 - 453. 452 UInt64 0 - 454. 453 UInt64 1 - 455. 454 UInt64 0 - 456. 455 UInt64 1 - 457. 456 UInt64 0 - 458. 457 UInt64 1 - 459. 458 UInt64 0 - 460. 459 UInt64 1 - 461. 460 UInt64 0 - 462. 461 UInt64 1 - 463. 462 UInt64 0 - 464. 463 UInt64 1 - 465. 464 UInt64 0 - 466. 465 UInt64 1 - 467. 466 UInt64 0 - 468. 467 UInt64 1 - 469. 468 UInt64 0 - 470. 469 UInt64 1 - 471. 470 UInt64 0 - 472. 471 UInt64 1 - 473. 472 UInt64 0 - 474. 473 UInt64 1 - 475. 474 UInt64 0 - 476. 475 UInt64 1 - 477. 476 UInt64 0 - 478. 477 UInt64 1 - 479. 478 UInt64 0 - 480. 479 UInt64 1 - 481. 480 UInt64 0 - 482. 481 UInt64 1 - 483. 482 UInt64 0 - 484. 483 UInt64 1 - 485. 484 UInt64 0 - 486. 485 UInt64 1 - 487. 486 UInt64 0 - 488. 487 UInt64 1 - 489. 488 UInt64 0 - 490. 489 UInt64 1 - 491. 490 UInt64 0 - 492. 491 UInt64 1 - 493. 492 UInt64 0 - 494. 493 UInt64 1 - 495. 494 UInt64 0 - 496. 495 UInt64 1 - 497. 496 UInt64 0 - 498. 497 UInt64 1 - 499. 498 UInt64 0 - 500. 499 UInt64 1 - 501. 500 UInt64 0 - 502. 501 UInt64 1 - 503. 502 UInt64 0 - 504. 503 UInt64 1 - 505. 504 UInt64 0 - 506. 505 UInt64 1 - 507. 506 UInt64 0 - 508. 507 UInt64 1 - 509. 508 UInt64 0 - 510. 509 UInt64 1 - 511. 510 UInt64 0 - 512. 511 UInt64 1 - 513. 512 UInt64 0 - 514. 513 UInt64 1 - 515. 514 UInt64 0 - 516. 515 UInt64 1 - 517. 516 UInt64 0 - 518. 517 UInt64 1 - 519. 518 UInt64 0 - 520. 519 UInt64 1 - 521. 520 UInt64 0 - 522. 521 UInt64 1 - 523. 522 UInt64 0 - 524. 523 UInt64 1 - 525. 524 UInt64 0 - 526. 525 UInt64 1 - 527. 526 UInt64 0 - 528. 527 UInt64 1 - 529. 528 UInt64 0 - 530. 529 UInt64 1 - 531. 530 UInt64 0 - 532. 531 UInt64 1 - 533. 532 UInt64 0 - 534. 533 UInt64 1 - 535. 534 UInt64 0 - 536. 535 UInt64 1 - 537. 536 UInt64 0 - 538. 537 UInt64 1 - 539. 538 UInt64 0 - 540. 539 UInt64 1 - 541. 540 UInt64 0 - 542. 541 UInt64 1 - 543. 542 UInt64 0 - 544. 543 UInt64 1 - 545. 544 UInt64 0 - 546. 545 UInt64 1 - 547. 546 UInt64 0 - 548. 547 UInt64 1 - 549. 548 UInt64 0 - 550. 549 UInt64 1 - 551. 550 UInt64 0 - 552. 551 UInt64 1 - 553. 552 UInt64 0 - 554. 553 UInt64 1 - 555. 554 UInt64 0 - 556. 555 UInt64 1 - 557. 556 UInt64 0 - 558. 557 UInt64 1 - 559. 558 UInt64 0 - 560. 559 UInt64 1 - 561. 560 UInt64 0 - 562. 561 UInt64 1 - 563. 562 UInt64 0 - 564. 563 UInt64 1 - 565. 564 UInt64 0 - 566. 565 UInt64 1 - 567. 566 UInt64 0 - 568. 567 UInt64 1 - 569. 568 UInt64 0 - 570. 569 UInt64 1 - 571. 570 UInt64 0 - 572. 571 UInt64 1 - 573. 572 UInt64 0 - 574. 573 UInt64 1 - 575. 574 UInt64 0 - 576. 575 UInt64 1 - 577. 576 UInt64 0 - 578. 577 UInt64 1 - 579. 578 UInt64 0 - 580. 579 UInt64 1 - 581. 580 UInt64 0 - 582. 581 UInt64 1 - 583. 582 UInt64 0 - 584. 583 UInt64 1 - 585. 584 UInt64 0 - 586. 585 UInt64 1 - 587. 586 UInt64 0 - 588. 587 UInt64 1 - 589. 588 UInt64 0 - 590. 589 UInt64 1 - 591. 590 UInt64 0 - 592. 591 UInt64 1 - 593. 592 UInt64 0 - 594. 593 UInt64 1 - 595. 594 UInt64 0 - 596. 595 UInt64 1 - 597. 596 UInt64 0 - 598. 597 UInt64 1 - 599. 598 UInt64 0 - 600. 599 UInt64 1 - 601. 600 UInt64 0 - 602. 601 UInt64 1 - 603. 602 UInt64 0 - 604. 603 UInt64 1 - 605. 604 UInt64 0 - 606. 605 UInt64 1 - 607. 606 UInt64 0 - 608. 607 UInt64 1 - 609. 608 UInt64 0 - 610. 609 UInt64 1 - 611. 610 UInt64 0 - 612. 611 UInt64 1 - 613. 612 UInt64 0 - 614. 613 UInt64 1 - 615. 614 UInt64 0 - 616. 615 UInt64 1 - 617. 616 UInt64 0 - 618. 617 UInt64 1 - 619. 618 UInt64 0 - 620. 619 UInt64 1 - 621. 620 UInt64 0 - 622. 621 UInt64 1 - 623. 622 UInt64 0 - 624. 623 UInt64 1 - 625. 624 UInt64 0 - 626. 625 UInt64 1 - 627. 626 UInt64 0 - 628. 627 UInt64 1 - 629. 628 UInt64 0 - 630. 629 UInt64 1 - 631. 630 UInt64 0 - 632. 631 UInt64 1 - 633. 632 UInt64 0 - 634. 633 UInt64 1 - 635. 634 UInt64 0 - 636. 635 UInt64 1 - 637. 636 UInt64 0 - 638. 637 UInt64 1 - 639. 638 UInt64 0 - 640. 639 UInt64 1 - 641. 640 UInt64 0 - 642. 641 UInt64 1 - 643. 642 UInt64 0 - 644. 643 UInt64 1 - 645. 644 UInt64 0 - 646. 645 UInt64 1 - 647. 646 UInt64 0 - 648. 647 UInt64 1 - 649. 648 UInt64 0 - 650. 649 UInt64 1 - 651. 650 UInt64 0 - 652. 651 UInt64 1 - 653. 652 UInt64 0 - 654. 653 UInt64 1 - 655. 654 UInt64 0 - 656. 655 UInt64 1 - 657. 656 UInt64 0 - 658. 657 UInt64 1 - 659. 658 UInt64 0 - 660. 659 UInt64 1 - 661. 660 UInt64 0 - 662. 661 UInt64 1 - 663. 662 UInt64 0 - 664. 663 UInt64 1 - 665. 664 UInt64 0 - 666. 665 UInt64 1 - 667. 666 UInt64 0 - 668. 667 UInt64 1 - 669. 668 UInt64 0 - 670. 669 UInt64 1 - 671. 670 UInt64 0 - 672. 671 UInt64 1 - 673. 672 UInt64 0 - 674. 673 UInt64 1 - 675. 674 UInt64 0 - 676. 675 UInt64 1 - 677. 676 UInt64 0 - 678. 677 UInt64 1 - 679. 678 UInt64 0 - 680. 679 UInt64 1 - 681. 680 UInt64 0 - 682. 681 UInt64 1 - 683. 682 UInt64 0 - 684. 683 UInt64 1 - 685. 684 UInt64 0 - 686. 685 UInt64 1 - 687. 686 UInt64 0 - 688. 687 UInt64 1 - 689. 688 UInt64 0 - 690. 689 UInt64 1 - 691. 690 UInt64 0 - 692. 691 UInt64 1 - 693. 692 UInt64 0 - 694. 693 UInt64 1 - 695. 694 UInt64 0 - 696. 695 UInt64 1 - 697. 696 UInt64 0 - 698. 697 UInt64 1 - 699. 698 UInt64 0 - 700. 699 UInt64 1 - 701. 700 UInt64 0 - 702. 701 UInt64 1 - 703. 702 UInt64 0 - 704. 703 UInt64 1 - 705. 704 UInt64 0 - 706. 705 UInt64 1 - 707. 706 UInt64 0 - 708. 707 UInt64 1 - 709. 708 UInt64 0 - 710. 709 UInt64 1 - 711. 710 UInt64 0 - 712. 711 UInt64 1 - 713. 712 UInt64 0 - 714. 713 UInt64 1 - 715. 714 UInt64 0 - 716. 715 UInt64 1 - 717. 716 UInt64 0 - 718. 717 UInt64 1 - 719. 718 UInt64 0 - 720. 719 UInt64 1 - 721. 720 UInt64 0 - 722. 721 UInt64 1 - 723. 722 UInt64 0 - 724. 723 UInt64 1 - 725. 724 UInt64 0 - 726. 725 UInt64 1 - 727. 726 UInt64 0 - 728. 727 UInt64 1 - 729. 728 UInt64 0 - 730. 729 UInt64 1 - 731. 730 UInt64 0 - 732. 731 UInt64 1 - 733. 732 UInt64 0 - 734. 733 UInt64 1 - 735. 734 UInt64 0 - 736. 735 UInt64 1 - 737. 736 UInt64 0 - 738. 737 UInt64 1 - 739. 738 UInt64 0 - 740. 739 UInt64 1 - 741. 740 UInt64 0 - 742. 741 UInt64 1 - 743. 742 UInt64 0 - 744. 743 UInt64 1 - 745. 744 UInt64 0 - 746. 745 UInt64 1 - 747. 746 UInt64 0 - 748. 747 UInt64 1 - 749. 748 UInt64 0 - 750. 749 UInt64 1 - 751. 750 UInt64 0 - 752. 751 UInt64 1 - 753. 752 UInt64 0 - 754. 753 UInt64 1 - 755. 754 UInt64 0 - 756. 755 UInt64 1 - 757. 756 UInt64 0 - 758. 757 UInt64 1 - 759. 758 UInt64 0 - 760. 759 UInt64 1 - 761. 760 UInt64 0 - 762. 761 UInt64 1 - 763. 762 UInt64 0 - 764. 763 UInt64 1 - 765. 764 UInt64 0 - 766. 765 UInt64 1 - 767. 766 UInt64 0 - 768. 767 UInt64 1 - 769. 768 UInt64 0 - 770. 769 UInt64 1 - 771. 770 UInt64 0 - 772. 771 UInt64 1 - 773. 772 UInt64 0 - 774. 773 UInt64 1 - 775. 774 UInt64 0 - 776. 775 UInt64 1 - 777. 776 UInt64 0 - 778. 777 UInt64 1 - 779. 778 UInt64 0 - 780. 779 UInt64 1 - 781. 780 UInt64 0 - 782. 781 UInt64 1 - 783. 782 UInt64 0 - 784. 783 UInt64 1 - 785. 784 UInt64 0 - 786. 785 UInt64 1 - 787. 786 UInt64 0 - 788. 787 UInt64 1 - 789. 788 UInt64 0 - 790. 789 UInt64 1 - 791. 790 UInt64 0 - 792. 791 UInt64 1 - 793. 792 UInt64 0 - 794. 793 UInt64 1 - 795. 794 UInt64 0 - 796. 795 UInt64 1 - 797. 796 UInt64 0 - 798. 797 UInt64 1 - 799. 798 UInt64 0 - 800. 799 UInt64 1 - 801. 800 UInt64 0 - 802. 801 UInt64 1 - 803. 802 UInt64 0 - 804. 803 UInt64 1 - 805. 804 UInt64 0 - 806. 805 UInt64 1 - 807. 806 UInt64 0 - 808. 807 UInt64 1 - 809. 808 UInt64 0 - 810. 809 UInt64 1 - 811. 810 UInt64 0 - 812. 811 UInt64 1 - 813. 812 UInt64 0 - 814. 813 UInt64 1 - 815. 814 UInt64 0 - 816. 815 UInt64 1 - 817. 816 UInt64 0 - 818. 817 UInt64 1 - 819. 818 UInt64 0 - 820. 819 UInt64 1 - 821. 820 UInt64 0 - 822. 821 UInt64 1 - 823. 822 UInt64 0 - 824. 823 UInt64 1 - 825. 824 UInt64 0 - 826. 825 UInt64 1 - 827. 826 UInt64 0 - 828. 827 UInt64 1 - 829. 828 UInt64 0 - 830. 829 UInt64 1 - 831. 830 UInt64 0 - 832. 831 UInt64 1 - 833. 832 UInt64 0 - 834. 833 UInt64 1 - 835. 834 UInt64 0 - 836. 835 UInt64 1 - 837. 836 UInt64 0 - 838. 837 UInt64 1 - 839. 838 UInt64 0 - 840. 839 UInt64 1 - 841. 840 UInt64 0 - 842. 841 UInt64 1 - 843. 842 UInt64 0 - 844. 843 UInt64 1 - 845. 844 UInt64 0 - 846. 845 UInt64 1 - 847. 846 UInt64 0 - 848. 847 UInt64 1 - 849. 848 UInt64 0 - 850. 849 UInt64 1 - 851. 850 UInt64 0 - 852. 851 UInt64 1 - 853. 852 UInt64 0 - 854. 853 UInt64 1 - 855. 854 UInt64 0 - 856. 855 UInt64 1 - 857. 856 UInt64 0 - 858. 857 UInt64 1 - 859. 858 UInt64 0 - 860. 859 UInt64 1 - 861. 860 UInt64 0 - 862. 861 UInt64 1 - 863. 862 UInt64 0 - 864. 863 UInt64 1 - 865. 864 UInt64 0 - 866. 865 UInt64 1 - 867. 866 UInt64 0 - 868. 867 UInt64 1 - 869. 868 UInt64 0 - 870. 869 UInt64 1 - 871. 870 UInt64 0 - 872. 871 UInt64 1 - 873. 872 UInt64 0 - 874. 873 UInt64 1 - 875. 874 UInt64 0 - 876. 875 UInt64 1 - 877. 876 UInt64 0 - 878. 877 UInt64 1 - 879. 878 UInt64 0 - 880. 879 UInt64 1 - 881. 880 UInt64 0 - 882. 881 UInt64 1 - 883. 882 UInt64 0 - 884. 883 UInt64 1 - 885. 884 UInt64 0 - 886. 885 UInt64 1 - 887. 886 UInt64 0 - 888. 887 UInt64 1 - 889. 888 UInt64 0 - 890. 889 UInt64 1 - 891. 890 UInt64 0 - 892. 891 UInt64 1 - 893. 892 UInt64 0 - 894. 893 UInt64 1 - 895. 894 UInt64 0 - 896. 895 UInt64 1 - 897. 896 UInt64 0 - 898. 897 UInt64 1 - 899. 898 UInt64 0 - 900. 899 UInt64 1 - 901. 900 UInt64 0 - 902. 901 UInt64 1 - 903. 902 UInt64 0 - 904. 903 UInt64 1 - 905. 904 UInt64 0 - 906. 905 UInt64 1 - 907. 906 UInt64 0 - 908. 907 UInt64 1 - 909. 908 UInt64 0 - 910. 909 UInt64 1 - 911. 910 UInt64 0 - 912. 911 UInt64 1 - 913. 912 UInt64 0 - 914. 913 UInt64 1 - 915. 914 UInt64 0 - 916. 915 UInt64 1 - 917. 916 UInt64 0 - 918. 917 UInt64 1 - 919. 918 UInt64 0 - 920. 919 UInt64 1 - 921. 920 UInt64 0 - 922. 921 UInt64 1 - 923. 922 UInt64 0 - 924. 923 UInt64 1 - 925. 924 UInt64 0 - 926. 925 UInt64 1 - 927. 926 UInt64 0 - 928. 927 UInt64 1 - 929. 928 UInt64 0 - 930. 929 UInt64 1 - 931. 930 UInt64 0 - 932. 931 UInt64 1 - 933. 932 UInt64 0 - 934. 933 UInt64 1 - 935. 934 UInt64 0 - 936. 935 UInt64 1 - 937. 936 UInt64 0 - 938. 937 UInt64 1 - 939. 938 UInt64 0 - 940. 939 UInt64 1 - 941. 940 UInt64 0 - 942. 941 UInt64 1 - 943. 942 UInt64 0 - 944. 943 UInt64 1 - 945. 944 UInt64 0 - 946. 945 UInt64 1 - 947. 946 UInt64 0 - 948. 947 UInt64 1 - 949. 948 UInt64 0 - 950. 949 UInt64 1 - 951. 950 UInt64 0 - 952. 951 UInt64 1 - 953. 952 UInt64 0 - 954. 953 UInt64 1 - 955. 954 UInt64 0 - 956. 955 UInt64 1 - 957. 956 UInt64 0 - 958. 957 UInt64 1 - 959. 958 UInt64 0 - 960. 959 UInt64 1 - 961. 960 UInt64 0 - 962. 961 UInt64 1 - 963. 962 UInt64 0 - 964. 963 UInt64 1 - 965. 964 UInt64 0 - 966. 965 UInt64 1 - 967. 966 UInt64 0 - 968. 967 UInt64 1 - 969. 968 UInt64 0 - 970. 969 UInt64 1 - 971. 970 UInt64 0 - 972. 971 UInt64 1 - 973. 972 UInt64 0 - 974. 973 UInt64 1 - 975. 974 UInt64 0 - 976. 975 UInt64 1 - 977. 976 UInt64 0 - 978. 977 UInt64 1 - 979. 978 UInt64 0 - 980. 979 UInt64 1 - 981. 980 UInt64 0 - 982. 981 UInt64 1 - 983. 982 UInt64 0 - 984. 983 UInt64 1 - 985. 984 UInt64 0 - 986. 985 UInt64 1 - 987. 986 UInt64 0 - 988. 987 UInt64 1 - 989. 988 UInt64 0 - 990. 989 UInt64 1 - 991. 990 UInt64 0 - 992. 991 UInt64 1 - 993. 992 UInt64 0 - 994. 993 UInt64 1 - 995. 994 UInt64 0 - 996. 995 UInt64 1 - 997. 996 UInt64 0 - 998. 997 UInt64 1 - 999. 998 UInt64 0 -1000. 999 UInt64 1 + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 +100. 99 UInt64 1 + number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 +100. 99 UInt64 1 - number toTypeName(number) mod(number, 2) + number toTypeName(number) mod(number, 2) - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 - 100. 99 UInt64 1 - 101. 100 UInt64 0 - 102. 101 UInt64 1 - 103. 102 UInt64 0 - 104. 103 UInt64 1 - 105. 104 UInt64 0 - 106. 105 UInt64 1 - 107. 106 UInt64 0 - 108. 107 UInt64 1 - 109. 108 UInt64 0 - 110. 109 UInt64 1 - 111. 110 UInt64 0 - 112. 111 UInt64 1 - 113. 112 UInt64 0 - 114. 113 UInt64 1 - 115. 114 UInt64 0 - 116. 115 UInt64 1 - 117. 116 UInt64 0 - 118. 117 UInt64 1 - 119. 118 UInt64 0 - 120. 119 UInt64 1 - 121. 120 UInt64 0 - 122. 121 UInt64 1 - 123. 122 UInt64 0 - 124. 123 UInt64 1 - 125. 124 UInt64 0 - 126. 125 UInt64 1 - 127. 126 UInt64 0 - 128. 127 UInt64 1 - 129. 128 UInt64 0 - 130. 129 UInt64 1 - 131. 130 UInt64 0 - 132. 131 UInt64 1 - 133. 132 UInt64 0 - 134. 133 UInt64 1 - 135. 134 UInt64 0 - 136. 135 UInt64 1 - 137. 136 UInt64 0 - 138. 137 UInt64 1 - 139. 138 UInt64 0 - 140. 139 UInt64 1 - 141. 140 UInt64 0 - 142. 141 UInt64 1 - 143. 142 UInt64 0 - 144. 143 UInt64 1 - 145. 144 UInt64 0 - 146. 145 UInt64 1 - 147. 146 UInt64 0 - 148. 147 UInt64 1 - 149. 148 UInt64 0 - 150. 149 UInt64 1 - 151. 150 UInt64 0 - 152. 151 UInt64 1 - 153. 152 UInt64 0 - 154. 153 UInt64 1 - 155. 154 UInt64 0 - 156. 155 UInt64 1 - 157. 156 UInt64 0 - 158. 157 UInt64 1 - 159. 158 UInt64 0 - 160. 159 UInt64 1 - 161. 160 UInt64 0 - 162. 161 UInt64 1 - 163. 162 UInt64 0 - 164. 163 UInt64 1 - 165. 164 UInt64 0 - 166. 165 UInt64 1 - 167. 166 UInt64 0 - 168. 167 UInt64 1 - 169. 168 UInt64 0 - 170. 169 UInt64 1 - 171. 170 UInt64 0 - 172. 171 UInt64 1 - 173. 172 UInt64 0 - 174. 173 UInt64 1 - 175. 174 UInt64 0 - 176. 175 UInt64 1 - 177. 176 UInt64 0 - 178. 177 UInt64 1 - 179. 178 UInt64 0 - 180. 179 UInt64 1 - 181. 180 UInt64 0 - 182. 181 UInt64 1 - 183. 182 UInt64 0 - 184. 183 UInt64 1 - 185. 184 UInt64 0 - 186. 185 UInt64 1 - 187. 186 UInt64 0 - 188. 187 UInt64 1 - 189. 188 UInt64 0 - 190. 189 UInt64 1 - 191. 190 UInt64 0 - 192. 191 UInt64 1 - 193. 192 UInt64 0 - 194. 193 UInt64 1 - 195. 194 UInt64 0 - 196. 195 UInt64 1 - 197. 196 UInt64 0 - 198. 197 UInt64 1 - 199. 198 UInt64 0 - 200. 199 UInt64 1 - 201. 200 UInt64 0 - 202. 201 UInt64 1 - 203. 202 UInt64 0 - 204. 203 UInt64 1 - 205. 204 UInt64 0 - 206. 205 UInt64 1 - 207. 206 UInt64 0 - 208. 207 UInt64 1 - 209. 208 UInt64 0 - 210. 209 UInt64 1 - 211. 210 UInt64 0 - 212. 211 UInt64 1 - 213. 212 UInt64 0 - 214. 213 UInt64 1 - 215. 214 UInt64 0 - 216. 215 UInt64 1 - 217. 216 UInt64 0 - 218. 217 UInt64 1 - 219. 218 UInt64 0 - 220. 219 UInt64 1 - 221. 220 UInt64 0 - 222. 221 UInt64 1 - 223. 222 UInt64 0 - 224. 223 UInt64 1 - 225. 224 UInt64 0 - 226. 225 UInt64 1 - 227. 226 UInt64 0 - 228. 227 UInt64 1 - 229. 228 UInt64 0 - 230. 229 UInt64 1 - 231. 230 UInt64 0 - 232. 231 UInt64 1 - 233. 232 UInt64 0 - 234. 233 UInt64 1 - 235. 234 UInt64 0 - 236. 235 UInt64 1 - 237. 236 UInt64 0 - 238. 237 UInt64 1 - 239. 238 UInt64 0 - 240. 239 UInt64 1 - 241. 240 UInt64 0 - 242. 241 UInt64 1 - 243. 242 UInt64 0 - 244. 243 UInt64 1 - 245. 244 UInt64 0 - 246. 245 UInt64 1 - 247. 246 UInt64 0 - 248. 247 UInt64 1 - 249. 248 UInt64 0 - 250. 249 UInt64 1 - 251. 250 UInt64 0 - 252. 251 UInt64 1 - 253. 252 UInt64 0 - 254. 253 UInt64 1 - 255. 254 UInt64 0 - 256. 255 UInt64 1 - 257. 256 UInt64 0 - 258. 257 UInt64 1 - 259. 258 UInt64 0 - 260. 259 UInt64 1 - 261. 260 UInt64 0 - 262. 261 UInt64 1 - 263. 262 UInt64 0 - 264. 263 UInt64 1 - 265. 264 UInt64 0 - 266. 265 UInt64 1 - 267. 266 UInt64 0 - 268. 267 UInt64 1 - 269. 268 UInt64 0 - 270. 269 UInt64 1 - 271. 270 UInt64 0 - 272. 271 UInt64 1 - 273. 272 UInt64 0 - 274. 273 UInt64 1 - 275. 274 UInt64 0 - 276. 275 UInt64 1 - 277. 276 UInt64 0 - 278. 277 UInt64 1 - 279. 278 UInt64 0 - 280. 279 UInt64 1 - 281. 280 UInt64 0 - 282. 281 UInt64 1 - 283. 282 UInt64 0 - 284. 283 UInt64 1 - 285. 284 UInt64 0 - 286. 285 UInt64 1 - 287. 286 UInt64 0 - 288. 287 UInt64 1 - 289. 288 UInt64 0 - 290. 289 UInt64 1 - 291. 290 UInt64 0 - 292. 291 UInt64 1 - 293. 292 UInt64 0 - 294. 293 UInt64 1 - 295. 294 UInt64 0 - 296. 295 UInt64 1 - 297. 296 UInt64 0 - 298. 297 UInt64 1 - 299. 298 UInt64 0 - 300. 299 UInt64 1 - 301. 300 UInt64 0 - 302. 301 UInt64 1 - 303. 302 UInt64 0 - 304. 303 UInt64 1 - 305. 304 UInt64 0 - 306. 305 UInt64 1 - 307. 306 UInt64 0 - 308. 307 UInt64 1 - 309. 308 UInt64 0 - 310. 309 UInt64 1 - 311. 310 UInt64 0 - 312. 311 UInt64 1 - 313. 312 UInt64 0 - 314. 313 UInt64 1 - 315. 314 UInt64 0 - 316. 315 UInt64 1 - 317. 316 UInt64 0 - 318. 317 UInt64 1 - 319. 318 UInt64 0 - 320. 319 UInt64 1 - 321. 320 UInt64 0 - 322. 321 UInt64 1 - 323. 322 UInt64 0 - 324. 323 UInt64 1 - 325. 324 UInt64 0 - 326. 325 UInt64 1 - 327. 326 UInt64 0 - 328. 327 UInt64 1 - 329. 328 UInt64 0 - 330. 329 UInt64 1 - 331. 330 UInt64 0 - 332. 331 UInt64 1 - 333. 332 UInt64 0 - 334. 333 UInt64 1 - 335. 334 UInt64 0 - 336. 335 UInt64 1 - 337. 336 UInt64 0 - 338. 337 UInt64 1 - 339. 338 UInt64 0 - 340. 339 UInt64 1 - 341. 340 UInt64 0 - 342. 341 UInt64 1 - 343. 342 UInt64 0 - 344. 343 UInt64 1 - 345. 344 UInt64 0 - 346. 345 UInt64 1 - 347. 346 UInt64 0 - 348. 347 UInt64 1 - 349. 348 UInt64 0 - 350. 349 UInt64 1 - 351. 350 UInt64 0 - 352. 351 UInt64 1 - 353. 352 UInt64 0 - 354. 353 UInt64 1 - 355. 354 UInt64 0 - 356. 355 UInt64 1 - 357. 356 UInt64 0 - 358. 357 UInt64 1 - 359. 358 UInt64 0 - 360. 359 UInt64 1 - 361. 360 UInt64 0 - 362. 361 UInt64 1 - 363. 362 UInt64 0 - 364. 363 UInt64 1 - 365. 364 UInt64 0 - 366. 365 UInt64 1 - 367. 366 UInt64 0 - 368. 367 UInt64 1 - 369. 368 UInt64 0 - 370. 369 UInt64 1 - 371. 370 UInt64 0 - 372. 371 UInt64 1 - 373. 372 UInt64 0 - 374. 373 UInt64 1 - 375. 374 UInt64 0 - 376. 375 UInt64 1 - 377. 376 UInt64 0 - 378. 377 UInt64 1 - 379. 378 UInt64 0 - 380. 379 UInt64 1 - 381. 380 UInt64 0 - 382. 381 UInt64 1 - 383. 382 UInt64 0 - 384. 383 UInt64 1 - 385. 384 UInt64 0 - 386. 385 UInt64 1 - 387. 386 UInt64 0 - 388. 387 UInt64 1 - 389. 388 UInt64 0 - 390. 389 UInt64 1 - 391. 390 UInt64 0 - 392. 391 UInt64 1 - 393. 392 UInt64 0 - 394. 393 UInt64 1 - 395. 394 UInt64 0 - 396. 395 UInt64 1 - 397. 396 UInt64 0 - 398. 397 UInt64 1 - 399. 398 UInt64 0 - 400. 399 UInt64 1 - 401. 400 UInt64 0 - 402. 401 UInt64 1 - 403. 402 UInt64 0 - 404. 403 UInt64 1 - 405. 404 UInt64 0 - 406. 405 UInt64 1 - 407. 406 UInt64 0 - 408. 407 UInt64 1 - 409. 408 UInt64 0 - 410. 409 UInt64 1 - 411. 410 UInt64 0 - 412. 411 UInt64 1 - 413. 412 UInt64 0 - 414. 413 UInt64 1 - 415. 414 UInt64 0 - 416. 415 UInt64 1 - 417. 416 UInt64 0 - 418. 417 UInt64 1 - 419. 418 UInt64 0 - 420. 419 UInt64 1 - 421. 420 UInt64 0 - 422. 421 UInt64 1 - 423. 422 UInt64 0 - 424. 423 UInt64 1 - 425. 424 UInt64 0 - 426. 425 UInt64 1 - 427. 426 UInt64 0 - 428. 427 UInt64 1 - 429. 428 UInt64 0 - 430. 429 UInt64 1 - 431. 430 UInt64 0 - 432. 431 UInt64 1 - 433. 432 UInt64 0 - 434. 433 UInt64 1 - 435. 434 UInt64 0 - 436. 435 UInt64 1 - 437. 436 UInt64 0 - 438. 437 UInt64 1 - 439. 438 UInt64 0 - 440. 439 UInt64 1 - 441. 440 UInt64 0 - 442. 441 UInt64 1 - 443. 442 UInt64 0 - 444. 443 UInt64 1 - 445. 444 UInt64 0 - 446. 445 UInt64 1 - 447. 446 UInt64 0 - 448. 447 UInt64 1 - 449. 448 UInt64 0 - 450. 449 UInt64 1 - 451. 450 UInt64 0 - 452. 451 UInt64 1 - 453. 452 UInt64 0 - 454. 453 UInt64 1 - 455. 454 UInt64 0 - 456. 455 UInt64 1 - 457. 456 UInt64 0 - 458. 457 UInt64 1 - 459. 458 UInt64 0 - 460. 459 UInt64 1 - 461. 460 UInt64 0 - 462. 461 UInt64 1 - 463. 462 UInt64 0 - 464. 463 UInt64 1 - 465. 464 UInt64 0 - 466. 465 UInt64 1 - 467. 466 UInt64 0 - 468. 467 UInt64 1 - 469. 468 UInt64 0 - 470. 469 UInt64 1 - 471. 470 UInt64 0 - 472. 471 UInt64 1 - 473. 472 UInt64 0 - 474. 473 UInt64 1 - 475. 474 UInt64 0 - 476. 475 UInt64 1 - 477. 476 UInt64 0 - 478. 477 UInt64 1 - 479. 478 UInt64 0 - 480. 479 UInt64 1 - 481. 480 UInt64 0 - 482. 481 UInt64 1 - 483. 482 UInt64 0 - 484. 483 UInt64 1 - 485. 484 UInt64 0 - 486. 485 UInt64 1 - 487. 486 UInt64 0 - 488. 487 UInt64 1 - 489. 488 UInt64 0 - 490. 489 UInt64 1 - 491. 490 UInt64 0 - 492. 491 UInt64 1 - 493. 492 UInt64 0 - 494. 493 UInt64 1 - 495. 494 UInt64 0 - 496. 495 UInt64 1 - 497. 496 UInt64 0 - 498. 497 UInt64 1 - 499. 498 UInt64 0 - 500. 499 UInt64 1 - 501. 500 UInt64 0 - 502. 501 UInt64 1 - 503. 502 UInt64 0 - 504. 503 UInt64 1 - 505. 504 UInt64 0 - 506. 505 UInt64 1 - 507. 506 UInt64 0 - 508. 507 UInt64 1 - 509. 508 UInt64 0 - 510. 509 UInt64 1 - 511. 510 UInt64 0 - 512. 511 UInt64 1 - 513. 512 UInt64 0 - 514. 513 UInt64 1 - 515. 514 UInt64 0 - 516. 515 UInt64 1 - 517. 516 UInt64 0 - 518. 517 UInt64 1 - 519. 518 UInt64 0 - 520. 519 UInt64 1 - 521. 520 UInt64 0 - 522. 521 UInt64 1 - 523. 522 UInt64 0 - 524. 523 UInt64 1 - 525. 524 UInt64 0 - 526. 525 UInt64 1 - 527. 526 UInt64 0 - 528. 527 UInt64 1 - 529. 528 UInt64 0 - 530. 529 UInt64 1 - 531. 530 UInt64 0 - 532. 531 UInt64 1 - 533. 532 UInt64 0 - 534. 533 UInt64 1 - 535. 534 UInt64 0 - 536. 535 UInt64 1 - 537. 536 UInt64 0 - 538. 537 UInt64 1 - 539. 538 UInt64 0 - 540. 539 UInt64 1 - 541. 540 UInt64 0 - 542. 541 UInt64 1 - 543. 542 UInt64 0 - 544. 543 UInt64 1 - 545. 544 UInt64 0 - 546. 545 UInt64 1 - 547. 546 UInt64 0 - 548. 547 UInt64 1 - 549. 548 UInt64 0 - 550. 549 UInt64 1 - 551. 550 UInt64 0 - 552. 551 UInt64 1 - 553. 552 UInt64 0 - 554. 553 UInt64 1 - 555. 554 UInt64 0 - 556. 555 UInt64 1 - 557. 556 UInt64 0 - 558. 557 UInt64 1 - 559. 558 UInt64 0 - 560. 559 UInt64 1 - 561. 560 UInt64 0 - 562. 561 UInt64 1 - 563. 562 UInt64 0 - 564. 563 UInt64 1 - 565. 564 UInt64 0 - 566. 565 UInt64 1 - 567. 566 UInt64 0 - 568. 567 UInt64 1 - 569. 568 UInt64 0 - 570. 569 UInt64 1 - 571. 570 UInt64 0 - 572. 571 UInt64 1 - 573. 572 UInt64 0 - 574. 573 UInt64 1 - 575. 574 UInt64 0 - 576. 575 UInt64 1 - 577. 576 UInt64 0 - 578. 577 UInt64 1 - 579. 578 UInt64 0 - 580. 579 UInt64 1 - 581. 580 UInt64 0 - 582. 581 UInt64 1 - 583. 582 UInt64 0 - 584. 583 UInt64 1 - 585. 584 UInt64 0 - 586. 585 UInt64 1 - 587. 586 UInt64 0 - 588. 587 UInt64 1 - 589. 588 UInt64 0 - 590. 589 UInt64 1 - 591. 590 UInt64 0 - 592. 591 UInt64 1 - 593. 592 UInt64 0 - 594. 593 UInt64 1 - 595. 594 UInt64 0 - 596. 595 UInt64 1 - 597. 596 UInt64 0 - 598. 597 UInt64 1 - 599. 598 UInt64 0 - 600. 599 UInt64 1 - 601. 600 UInt64 0 - 602. 601 UInt64 1 - 603. 602 UInt64 0 - 604. 603 UInt64 1 - 605. 604 UInt64 0 - 606. 605 UInt64 1 - 607. 606 UInt64 0 - 608. 607 UInt64 1 - 609. 608 UInt64 0 - 610. 609 UInt64 1 - 611. 610 UInt64 0 - 612. 611 UInt64 1 - 613. 612 UInt64 0 - 614. 613 UInt64 1 - 615. 614 UInt64 0 - 616. 615 UInt64 1 - 617. 616 UInt64 0 - 618. 617 UInt64 1 - 619. 618 UInt64 0 - 620. 619 UInt64 1 - 621. 620 UInt64 0 - 622. 621 UInt64 1 - 623. 622 UInt64 0 - 624. 623 UInt64 1 - 625. 624 UInt64 0 - 626. 625 UInt64 1 - 627. 626 UInt64 0 - 628. 627 UInt64 1 - 629. 628 UInt64 0 - 630. 629 UInt64 1 - 631. 630 UInt64 0 - 632. 631 UInt64 1 - 633. 632 UInt64 0 - 634. 633 UInt64 1 - 635. 634 UInt64 0 - 636. 635 UInt64 1 - 637. 636 UInt64 0 - 638. 637 UInt64 1 - 639. 638 UInt64 0 - 640. 639 UInt64 1 - 641. 640 UInt64 0 - 642. 641 UInt64 1 - 643. 642 UInt64 0 - 644. 643 UInt64 1 - 645. 644 UInt64 0 - 646. 645 UInt64 1 - 647. 646 UInt64 0 - 648. 647 UInt64 1 - 649. 648 UInt64 0 - 650. 649 UInt64 1 - 651. 650 UInt64 0 - 652. 651 UInt64 1 - 653. 652 UInt64 0 - 654. 653 UInt64 1 - 655. 654 UInt64 0 - 656. 655 UInt64 1 - 657. 656 UInt64 0 - 658. 657 UInt64 1 - 659. 658 UInt64 0 - 660. 659 UInt64 1 - 661. 660 UInt64 0 - 662. 661 UInt64 1 - 663. 662 UInt64 0 - 664. 663 UInt64 1 - 665. 664 UInt64 0 - 666. 665 UInt64 1 - 667. 666 UInt64 0 - 668. 667 UInt64 1 - 669. 668 UInt64 0 - 670. 669 UInt64 1 - 671. 670 UInt64 0 - 672. 671 UInt64 1 - 673. 672 UInt64 0 - 674. 673 UInt64 1 - 675. 674 UInt64 0 - 676. 675 UInt64 1 - 677. 676 UInt64 0 - 678. 677 UInt64 1 - 679. 678 UInt64 0 - 680. 679 UInt64 1 - 681. 680 UInt64 0 - 682. 681 UInt64 1 - 683. 682 UInt64 0 - 684. 683 UInt64 1 - 685. 684 UInt64 0 - 686. 685 UInt64 1 - 687. 686 UInt64 0 - 688. 687 UInt64 1 - 689. 688 UInt64 0 - 690. 689 UInt64 1 - 691. 690 UInt64 0 - 692. 691 UInt64 1 - 693. 692 UInt64 0 - 694. 693 UInt64 1 - 695. 694 UInt64 0 - 696. 695 UInt64 1 - 697. 696 UInt64 0 - 698. 697 UInt64 1 - 699. 698 UInt64 0 - 700. 699 UInt64 1 - 701. 700 UInt64 0 - 702. 701 UInt64 1 - 703. 702 UInt64 0 - 704. 703 UInt64 1 - 705. 704 UInt64 0 - 706. 705 UInt64 1 - 707. 706 UInt64 0 - 708. 707 UInt64 1 - 709. 708 UInt64 0 - 710. 709 UInt64 1 - 711. 710 UInt64 0 - 712. 711 UInt64 1 - 713. 712 UInt64 0 - 714. 713 UInt64 1 - 715. 714 UInt64 0 - 716. 715 UInt64 1 - 717. 716 UInt64 0 - 718. 717 UInt64 1 - 719. 718 UInt64 0 - 720. 719 UInt64 1 - 721. 720 UInt64 0 - 722. 721 UInt64 1 - 723. 722 UInt64 0 - 724. 723 UInt64 1 - 725. 724 UInt64 0 - 726. 725 UInt64 1 - 727. 726 UInt64 0 - 728. 727 UInt64 1 - 729. 728 UInt64 0 - 730. 729 UInt64 1 - 731. 730 UInt64 0 - 732. 731 UInt64 1 - 733. 732 UInt64 0 - 734. 733 UInt64 1 - 735. 734 UInt64 0 - 736. 735 UInt64 1 - 737. 736 UInt64 0 - 738. 737 UInt64 1 - 739. 738 UInt64 0 - 740. 739 UInt64 1 - 741. 740 UInt64 0 - 742. 741 UInt64 1 - 743. 742 UInt64 0 - 744. 743 UInt64 1 - 745. 744 UInt64 0 - 746. 745 UInt64 1 - 747. 746 UInt64 0 - 748. 747 UInt64 1 - 749. 748 UInt64 0 - 750. 749 UInt64 1 - 751. 750 UInt64 0 - 752. 751 UInt64 1 - 753. 752 UInt64 0 - 754. 753 UInt64 1 - 755. 754 UInt64 0 - 756. 755 UInt64 1 - 757. 756 UInt64 0 - 758. 757 UInt64 1 - 759. 758 UInt64 0 - 760. 759 UInt64 1 - 761. 760 UInt64 0 - 762. 761 UInt64 1 - 763. 762 UInt64 0 - 764. 763 UInt64 1 - 765. 764 UInt64 0 - 766. 765 UInt64 1 - 767. 766 UInt64 0 - 768. 767 UInt64 1 - 769. 768 UInt64 0 - 770. 769 UInt64 1 - 771. 770 UInt64 0 - 772. 771 UInt64 1 - 773. 772 UInt64 0 - 774. 773 UInt64 1 - 775. 774 UInt64 0 - 776. 775 UInt64 1 - 777. 776 UInt64 0 - 778. 777 UInt64 1 - 779. 778 UInt64 0 - 780. 779 UInt64 1 - 781. 780 UInt64 0 - 782. 781 UInt64 1 - 783. 782 UInt64 0 - 784. 783 UInt64 1 - 785. 784 UInt64 0 - 786. 785 UInt64 1 - 787. 786 UInt64 0 - 788. 787 UInt64 1 - 789. 788 UInt64 0 - 790. 789 UInt64 1 - 791. 790 UInt64 0 - 792. 791 UInt64 1 - 793. 792 UInt64 0 - 794. 793 UInt64 1 - 795. 794 UInt64 0 - 796. 795 UInt64 1 - 797. 796 UInt64 0 - 798. 797 UInt64 1 - 799. 798 UInt64 0 - 800. 799 UInt64 1 - 801. 800 UInt64 0 - 802. 801 UInt64 1 - 803. 802 UInt64 0 - 804. 803 UInt64 1 - 805. 804 UInt64 0 - 806. 805 UInt64 1 - 807. 806 UInt64 0 - 808. 807 UInt64 1 - 809. 808 UInt64 0 - 810. 809 UInt64 1 - 811. 810 UInt64 0 - 812. 811 UInt64 1 - 813. 812 UInt64 0 - 814. 813 UInt64 1 - 815. 814 UInt64 0 - 816. 815 UInt64 1 - 817. 816 UInt64 0 - 818. 817 UInt64 1 - 819. 818 UInt64 0 - 820. 819 UInt64 1 - 821. 820 UInt64 0 - 822. 821 UInt64 1 - 823. 822 UInt64 0 - 824. 823 UInt64 1 - 825. 824 UInt64 0 - 826. 825 UInt64 1 - 827. 826 UInt64 0 - 828. 827 UInt64 1 - 829. 828 UInt64 0 - 830. 829 UInt64 1 - 831. 830 UInt64 0 - 832. 831 UInt64 1 - 833. 832 UInt64 0 - 834. 833 UInt64 1 - 835. 834 UInt64 0 - 836. 835 UInt64 1 - 837. 836 UInt64 0 - 838. 837 UInt64 1 - 839. 838 UInt64 0 - 840. 839 UInt64 1 - 841. 840 UInt64 0 - 842. 841 UInt64 1 - 843. 842 UInt64 0 - 844. 843 UInt64 1 - 845. 844 UInt64 0 - 846. 845 UInt64 1 - 847. 846 UInt64 0 - 848. 847 UInt64 1 - 849. 848 UInt64 0 - 850. 849 UInt64 1 - 851. 850 UInt64 0 - 852. 851 UInt64 1 - 853. 852 UInt64 0 - 854. 853 UInt64 1 - 855. 854 UInt64 0 - 856. 855 UInt64 1 - 857. 856 UInt64 0 - 858. 857 UInt64 1 - 859. 858 UInt64 0 - 860. 859 UInt64 1 - 861. 860 UInt64 0 - 862. 861 UInt64 1 - 863. 862 UInt64 0 - 864. 863 UInt64 1 - 865. 864 UInt64 0 - 866. 865 UInt64 1 - 867. 866 UInt64 0 - 868. 867 UInt64 1 - 869. 868 UInt64 0 - 870. 869 UInt64 1 - 871. 870 UInt64 0 - 872. 871 UInt64 1 - 873. 872 UInt64 0 - 874. 873 UInt64 1 - 875. 874 UInt64 0 - 876. 875 UInt64 1 - 877. 876 UInt64 0 - 878. 877 UInt64 1 - 879. 878 UInt64 0 - 880. 879 UInt64 1 - 881. 880 UInt64 0 - 882. 881 UInt64 1 - 883. 882 UInt64 0 - 884. 883 UInt64 1 - 885. 884 UInt64 0 - 886. 885 UInt64 1 - 887. 886 UInt64 0 - 888. 887 UInt64 1 - 889. 888 UInt64 0 - 890. 889 UInt64 1 - 891. 890 UInt64 0 - 892. 891 UInt64 1 - 893. 892 UInt64 0 - 894. 893 UInt64 1 - 895. 894 UInt64 0 - 896. 895 UInt64 1 - 897. 896 UInt64 0 - 898. 897 UInt64 1 - 899. 898 UInt64 0 - 900. 899 UInt64 1 - 901. 900 UInt64 0 - 902. 901 UInt64 1 - 903. 902 UInt64 0 - 904. 903 UInt64 1 - 905. 904 UInt64 0 - 906. 905 UInt64 1 - 907. 906 UInt64 0 - 908. 907 UInt64 1 - 909. 908 UInt64 0 - 910. 909 UInt64 1 - 911. 910 UInt64 0 - 912. 911 UInt64 1 - 913. 912 UInt64 0 - 914. 913 UInt64 1 - 915. 914 UInt64 0 - 916. 915 UInt64 1 - 917. 916 UInt64 0 - 918. 917 UInt64 1 - 919. 918 UInt64 0 - 920. 919 UInt64 1 - 921. 920 UInt64 0 - 922. 921 UInt64 1 - 923. 922 UInt64 0 - 924. 923 UInt64 1 - 925. 924 UInt64 0 - 926. 925 UInt64 1 - 927. 926 UInt64 0 - 928. 927 UInt64 1 - 929. 928 UInt64 0 - 930. 929 UInt64 1 - 931. 930 UInt64 0 - 932. 931 UInt64 1 - 933. 932 UInt64 0 - 934. 933 UInt64 1 - 935. 934 UInt64 0 - 936. 935 UInt64 1 - 937. 936 UInt64 0 - 938. 937 UInt64 1 - 939. 938 UInt64 0 - 940. 939 UInt64 1 - 941. 940 UInt64 0 - 942. 941 UInt64 1 - 943. 942 UInt64 0 - 944. 943 UInt64 1 - 945. 944 UInt64 0 - 946. 945 UInt64 1 - 947. 946 UInt64 0 - 948. 947 UInt64 1 - 949. 948 UInt64 0 - 950. 949 UInt64 1 - 951. 950 UInt64 0 - 952. 951 UInt64 1 - 953. 952 UInt64 0 - 954. 953 UInt64 1 - 955. 954 UInt64 0 - 956. 955 UInt64 1 - 957. 956 UInt64 0 - 958. 957 UInt64 1 - 959. 958 UInt64 0 - 960. 959 UInt64 1 - 961. 960 UInt64 0 - 962. 961 UInt64 1 - 963. 962 UInt64 0 - 964. 963 UInt64 1 - 965. 964 UInt64 0 - 966. 965 UInt64 1 - 967. 966 UInt64 0 - 968. 967 UInt64 1 - 969. 968 UInt64 0 - 970. 969 UInt64 1 - 971. 970 UInt64 0 - 972. 971 UInt64 1 - 973. 972 UInt64 0 - 974. 973 UInt64 1 - 975. 974 UInt64 0 - 976. 975 UInt64 1 - 977. 976 UInt64 0 - 978. 977 UInt64 1 - 979. 978 UInt64 0 - 980. 979 UInt64 1 - 981. 980 UInt64 0 - 982. 981 UInt64 1 - 983. 982 UInt64 0 - 984. 983 UInt64 1 - 985. 984 UInt64 0 - 986. 985 UInt64 1 - 987. 986 UInt64 0 - 988. 987 UInt64 1 - 989. 988 UInt64 0 - 990. 989 UInt64 1 - 991. 990 UInt64 0 - 992. 991 UInt64 1 - 993. 992 UInt64 0 - 994. 993 UInt64 1 - 995. 994 UInt64 0 - 996. 995 UInt64 1 - 997. 996 UInt64 0 - 998. 997 UInt64 1 - 999. 998 UInt64 0 -1000. 999 UInt64 1 + number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 +100. 99 UInt64 1 - number toTypeName(number) mod(number, 2) + number toTypeName(number) mod(number, 2) - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 - 100. 99 UInt64 1 - 101. 100 UInt64 0 - 102. 101 UInt64 1 - 103. 102 UInt64 0 - 104. 103 UInt64 1 - 105. 104 UInt64 0 - 106. 105 UInt64 1 - 107. 106 UInt64 0 - 108. 107 UInt64 1 - 109. 108 UInt64 0 - 110. 109 UInt64 1 - 111. 110 UInt64 0 - 112. 111 UInt64 1 - 113. 112 UInt64 0 - 114. 113 UInt64 1 - 115. 114 UInt64 0 - 116. 115 UInt64 1 - 117. 116 UInt64 0 - 118. 117 UInt64 1 - 119. 118 UInt64 0 - 120. 119 UInt64 1 - 121. 120 UInt64 0 - 122. 121 UInt64 1 - 123. 122 UInt64 0 - 124. 123 UInt64 1 - 125. 124 UInt64 0 - 126. 125 UInt64 1 - 127. 126 UInt64 0 - 128. 127 UInt64 1 - 129. 128 UInt64 0 - 130. 129 UInt64 1 - 131. 130 UInt64 0 - 132. 131 UInt64 1 - 133. 132 UInt64 0 - 134. 133 UInt64 1 - 135. 134 UInt64 0 - 136. 135 UInt64 1 - 137. 136 UInt64 0 - 138. 137 UInt64 1 - 139. 138 UInt64 0 - 140. 139 UInt64 1 - 141. 140 UInt64 0 - 142. 141 UInt64 1 - 143. 142 UInt64 0 - 144. 143 UInt64 1 - 145. 144 UInt64 0 - 146. 145 UInt64 1 - 147. 146 UInt64 0 - 148. 147 UInt64 1 - 149. 148 UInt64 0 - 150. 149 UInt64 1 - 151. 150 UInt64 0 - 152. 151 UInt64 1 - 153. 152 UInt64 0 - 154. 153 UInt64 1 - 155. 154 UInt64 0 - 156. 155 UInt64 1 - 157. 156 UInt64 0 - 158. 157 UInt64 1 - 159. 158 UInt64 0 - 160. 159 UInt64 1 - 161. 160 UInt64 0 - 162. 161 UInt64 1 - 163. 162 UInt64 0 - 164. 163 UInt64 1 - 165. 164 UInt64 0 - 166. 165 UInt64 1 - 167. 166 UInt64 0 - 168. 167 UInt64 1 - 169. 168 UInt64 0 - 170. 169 UInt64 1 - 171. 170 UInt64 0 - 172. 171 UInt64 1 - 173. 172 UInt64 0 - 174. 173 UInt64 1 - 175. 174 UInt64 0 - 176. 175 UInt64 1 - 177. 176 UInt64 0 - 178. 177 UInt64 1 - 179. 178 UInt64 0 - 180. 179 UInt64 1 - 181. 180 UInt64 0 - 182. 181 UInt64 1 - 183. 182 UInt64 0 - 184. 183 UInt64 1 - 185. 184 UInt64 0 - 186. 185 UInt64 1 - 187. 186 UInt64 0 - 188. 187 UInt64 1 - 189. 188 UInt64 0 - 190. 189 UInt64 1 - 191. 190 UInt64 0 - 192. 191 UInt64 1 - 193. 192 UInt64 0 - 194. 193 UInt64 1 - 195. 194 UInt64 0 - 196. 195 UInt64 1 - 197. 196 UInt64 0 - 198. 197 UInt64 1 - 199. 198 UInt64 0 - 200. 199 UInt64 1 - 201. 200 UInt64 0 - 202. 201 UInt64 1 - 203. 202 UInt64 0 - 204. 203 UInt64 1 - 205. 204 UInt64 0 - 206. 205 UInt64 1 - 207. 206 UInt64 0 - 208. 207 UInt64 1 - 209. 208 UInt64 0 - 210. 209 UInt64 1 - 211. 210 UInt64 0 - 212. 211 UInt64 1 - 213. 212 UInt64 0 - 214. 213 UInt64 1 - 215. 214 UInt64 0 - 216. 215 UInt64 1 - 217. 216 UInt64 0 - 218. 217 UInt64 1 - 219. 218 UInt64 0 - 220. 219 UInt64 1 - 221. 220 UInt64 0 - 222. 221 UInt64 1 - 223. 222 UInt64 0 - 224. 223 UInt64 1 - 225. 224 UInt64 0 - 226. 225 UInt64 1 - 227. 226 UInt64 0 - 228. 227 UInt64 1 - 229. 228 UInt64 0 - 230. 229 UInt64 1 - 231. 230 UInt64 0 - 232. 231 UInt64 1 - 233. 232 UInt64 0 - 234. 233 UInt64 1 - 235. 234 UInt64 0 - 236. 235 UInt64 1 - 237. 236 UInt64 0 - 238. 237 UInt64 1 - 239. 238 UInt64 0 - 240. 239 UInt64 1 - 241. 240 UInt64 0 - 242. 241 UInt64 1 - 243. 242 UInt64 0 - 244. 243 UInt64 1 - 245. 244 UInt64 0 - 246. 245 UInt64 1 - 247. 246 UInt64 0 - 248. 247 UInt64 1 - 249. 248 UInt64 0 - 250. 249 UInt64 1 - 251. 250 UInt64 0 - 252. 251 UInt64 1 - 253. 252 UInt64 0 - 254. 253 UInt64 1 - 255. 254 UInt64 0 - 256. 255 UInt64 1 - 257. 256 UInt64 0 - 258. 257 UInt64 1 - 259. 258 UInt64 0 - 260. 259 UInt64 1 - 261. 260 UInt64 0 - 262. 261 UInt64 1 - 263. 262 UInt64 0 - 264. 263 UInt64 1 - 265. 264 UInt64 0 - 266. 265 UInt64 1 - 267. 266 UInt64 0 - 268. 267 UInt64 1 - 269. 268 UInt64 0 - 270. 269 UInt64 1 - 271. 270 UInt64 0 - 272. 271 UInt64 1 - 273. 272 UInt64 0 - 274. 273 UInt64 1 - 275. 274 UInt64 0 - 276. 275 UInt64 1 - 277. 276 UInt64 0 - 278. 277 UInt64 1 - 279. 278 UInt64 0 - 280. 279 UInt64 1 - 281. 280 UInt64 0 - 282. 281 UInt64 1 - 283. 282 UInt64 0 - 284. 283 UInt64 1 - 285. 284 UInt64 0 - 286. 285 UInt64 1 - 287. 286 UInt64 0 - 288. 287 UInt64 1 - 289. 288 UInt64 0 - 290. 289 UInt64 1 - 291. 290 UInt64 0 - 292. 291 UInt64 1 - 293. 292 UInt64 0 - 294. 293 UInt64 1 - 295. 294 UInt64 0 - 296. 295 UInt64 1 - 297. 296 UInt64 0 - 298. 297 UInt64 1 - 299. 298 UInt64 0 - 300. 299 UInt64 1 - 301. 300 UInt64 0 - 302. 301 UInt64 1 - 303. 302 UInt64 0 - 304. 303 UInt64 1 - 305. 304 UInt64 0 - 306. 305 UInt64 1 - 307. 306 UInt64 0 - 308. 307 UInt64 1 - 309. 308 UInt64 0 - 310. 309 UInt64 1 - 311. 310 UInt64 0 - 312. 311 UInt64 1 - 313. 312 UInt64 0 - 314. 313 UInt64 1 - 315. 314 UInt64 0 - 316. 315 UInt64 1 - 317. 316 UInt64 0 - 318. 317 UInt64 1 - 319. 318 UInt64 0 - 320. 319 UInt64 1 - 321. 320 UInt64 0 - 322. 321 UInt64 1 - 323. 322 UInt64 0 - 324. 323 UInt64 1 - 325. 324 UInt64 0 - 326. 325 UInt64 1 - 327. 326 UInt64 0 - 328. 327 UInt64 1 - 329. 328 UInt64 0 - 330. 329 UInt64 1 - 331. 330 UInt64 0 - 332. 331 UInt64 1 - 333. 332 UInt64 0 - 334. 333 UInt64 1 - 335. 334 UInt64 0 - 336. 335 UInt64 1 - 337. 336 UInt64 0 - 338. 337 UInt64 1 - 339. 338 UInt64 0 - 340. 339 UInt64 1 - 341. 340 UInt64 0 - 342. 341 UInt64 1 - 343. 342 UInt64 0 - 344. 343 UInt64 1 - 345. 344 UInt64 0 - 346. 345 UInt64 1 - 347. 346 UInt64 0 - 348. 347 UInt64 1 - 349. 348 UInt64 0 - 350. 349 UInt64 1 - 351. 350 UInt64 0 - 352. 351 UInt64 1 - 353. 352 UInt64 0 - 354. 353 UInt64 1 - 355. 354 UInt64 0 - 356. 355 UInt64 1 - 357. 356 UInt64 0 - 358. 357 UInt64 1 - 359. 358 UInt64 0 - 360. 359 UInt64 1 - 361. 360 UInt64 0 - 362. 361 UInt64 1 - 363. 362 UInt64 0 - 364. 363 UInt64 1 - 365. 364 UInt64 0 - 366. 365 UInt64 1 - 367. 366 UInt64 0 - 368. 367 UInt64 1 - 369. 368 UInt64 0 - 370. 369 UInt64 1 - 371. 370 UInt64 0 - 372. 371 UInt64 1 - 373. 372 UInt64 0 - 374. 373 UInt64 1 - 375. 374 UInt64 0 - 376. 375 UInt64 1 - 377. 376 UInt64 0 - 378. 377 UInt64 1 - 379. 378 UInt64 0 - 380. 379 UInt64 1 - 381. 380 UInt64 0 - 382. 381 UInt64 1 - 383. 382 UInt64 0 - 384. 383 UInt64 1 - 385. 384 UInt64 0 - 386. 385 UInt64 1 - 387. 386 UInt64 0 - 388. 387 UInt64 1 - 389. 388 UInt64 0 - 390. 389 UInt64 1 - 391. 390 UInt64 0 - 392. 391 UInt64 1 - 393. 392 UInt64 0 - 394. 393 UInt64 1 - 395. 394 UInt64 0 - 396. 395 UInt64 1 - 397. 396 UInt64 0 - 398. 397 UInt64 1 - 399. 398 UInt64 0 - 400. 399 UInt64 1 - 401. 400 UInt64 0 - 402. 401 UInt64 1 - 403. 402 UInt64 0 - 404. 403 UInt64 1 - 405. 404 UInt64 0 - 406. 405 UInt64 1 - 407. 406 UInt64 0 - 408. 407 UInt64 1 - 409. 408 UInt64 0 - 410. 409 UInt64 1 - 411. 410 UInt64 0 - 412. 411 UInt64 1 - 413. 412 UInt64 0 - 414. 413 UInt64 1 - 415. 414 UInt64 0 - 416. 415 UInt64 1 - 417. 416 UInt64 0 - 418. 417 UInt64 1 - 419. 418 UInt64 0 - 420. 419 UInt64 1 - 421. 420 UInt64 0 - 422. 421 UInt64 1 - 423. 422 UInt64 0 - 424. 423 UInt64 1 - 425. 424 UInt64 0 - 426. 425 UInt64 1 - 427. 426 UInt64 0 - 428. 427 UInt64 1 - 429. 428 UInt64 0 - 430. 429 UInt64 1 - 431. 430 UInt64 0 - 432. 431 UInt64 1 - 433. 432 UInt64 0 - 434. 433 UInt64 1 - 435. 434 UInt64 0 - 436. 435 UInt64 1 - 437. 436 UInt64 0 - 438. 437 UInt64 1 - 439. 438 UInt64 0 - 440. 439 UInt64 1 - 441. 440 UInt64 0 - 442. 441 UInt64 1 - 443. 442 UInt64 0 - 444. 443 UInt64 1 - 445. 444 UInt64 0 - 446. 445 UInt64 1 - 447. 446 UInt64 0 - 448. 447 UInt64 1 - 449. 448 UInt64 0 - 450. 449 UInt64 1 - 451. 450 UInt64 0 - 452. 451 UInt64 1 - 453. 452 UInt64 0 - 454. 453 UInt64 1 - 455. 454 UInt64 0 - 456. 455 UInt64 1 - 457. 456 UInt64 0 - 458. 457 UInt64 1 - 459. 458 UInt64 0 - 460. 459 UInt64 1 - 461. 460 UInt64 0 - 462. 461 UInt64 1 - 463. 462 UInt64 0 - 464. 463 UInt64 1 - 465. 464 UInt64 0 - 466. 465 UInt64 1 - 467. 466 UInt64 0 - 468. 467 UInt64 1 - 469. 468 UInt64 0 - 470. 469 UInt64 1 - 471. 470 UInt64 0 - 472. 471 UInt64 1 - 473. 472 UInt64 0 - 474. 473 UInt64 1 - 475. 474 UInt64 0 - 476. 475 UInt64 1 - 477. 476 UInt64 0 - 478. 477 UInt64 1 - 479. 478 UInt64 0 - 480. 479 UInt64 1 - 481. 480 UInt64 0 - 482. 481 UInt64 1 - 483. 482 UInt64 0 - 484. 483 UInt64 1 - 485. 484 UInt64 0 - 486. 485 UInt64 1 - 487. 486 UInt64 0 - 488. 487 UInt64 1 - 489. 488 UInt64 0 - 490. 489 UInt64 1 - 491. 490 UInt64 0 - 492. 491 UInt64 1 - 493. 492 UInt64 0 - 494. 493 UInt64 1 - 495. 494 UInt64 0 - 496. 495 UInt64 1 - 497. 496 UInt64 0 - 498. 497 UInt64 1 - 499. 498 UInt64 0 - 500. 499 UInt64 1 - 501. 500 UInt64 0 - 502. 501 UInt64 1 - 503. 502 UInt64 0 - 504. 503 UInt64 1 - 505. 504 UInt64 0 - 506. 505 UInt64 1 - 507. 506 UInt64 0 - 508. 507 UInt64 1 - 509. 508 UInt64 0 - 510. 509 UInt64 1 - 511. 510 UInt64 0 - 512. 511 UInt64 1 - 513. 512 UInt64 0 - 514. 513 UInt64 1 - 515. 514 UInt64 0 - 516. 515 UInt64 1 - 517. 516 UInt64 0 - 518. 517 UInt64 1 - 519. 518 UInt64 0 - 520. 519 UInt64 1 - 521. 520 UInt64 0 - 522. 521 UInt64 1 - 523. 522 UInt64 0 - 524. 523 UInt64 1 - 525. 524 UInt64 0 - 526. 525 UInt64 1 - 527. 526 UInt64 0 - 528. 527 UInt64 1 - 529. 528 UInt64 0 - 530. 529 UInt64 1 - 531. 530 UInt64 0 - 532. 531 UInt64 1 - 533. 532 UInt64 0 - 534. 533 UInt64 1 - 535. 534 UInt64 0 - 536. 535 UInt64 1 - 537. 536 UInt64 0 - 538. 537 UInt64 1 - 539. 538 UInt64 0 - 540. 539 UInt64 1 - 541. 540 UInt64 0 - 542. 541 UInt64 1 - 543. 542 UInt64 0 - 544. 543 UInt64 1 - 545. 544 UInt64 0 - 546. 545 UInt64 1 - 547. 546 UInt64 0 - 548. 547 UInt64 1 - 549. 548 UInt64 0 - 550. 549 UInt64 1 - 551. 550 UInt64 0 - 552. 551 UInt64 1 - 553. 552 UInt64 0 - 554. 553 UInt64 1 - 555. 554 UInt64 0 - 556. 555 UInt64 1 - 557. 556 UInt64 0 - 558. 557 UInt64 1 - 559. 558 UInt64 0 - 560. 559 UInt64 1 - 561. 560 UInt64 0 - 562. 561 UInt64 1 - 563. 562 UInt64 0 - 564. 563 UInt64 1 - 565. 564 UInt64 0 - 566. 565 UInt64 1 - 567. 566 UInt64 0 - 568. 567 UInt64 1 - 569. 568 UInt64 0 - 570. 569 UInt64 1 - 571. 570 UInt64 0 - 572. 571 UInt64 1 - 573. 572 UInt64 0 - 574. 573 UInt64 1 - 575. 574 UInt64 0 - 576. 575 UInt64 1 - 577. 576 UInt64 0 - 578. 577 UInt64 1 - 579. 578 UInt64 0 - 580. 579 UInt64 1 - 581. 580 UInt64 0 - 582. 581 UInt64 1 - 583. 582 UInt64 0 - 584. 583 UInt64 1 - 585. 584 UInt64 0 - 586. 585 UInt64 1 - 587. 586 UInt64 0 - 588. 587 UInt64 1 - 589. 588 UInt64 0 - 590. 589 UInt64 1 - 591. 590 UInt64 0 - 592. 591 UInt64 1 - 593. 592 UInt64 0 - 594. 593 UInt64 1 - 595. 594 UInt64 0 - 596. 595 UInt64 1 - 597. 596 UInt64 0 - 598. 597 UInt64 1 - 599. 598 UInt64 0 - 600. 599 UInt64 1 - 601. 600 UInt64 0 - 602. 601 UInt64 1 - 603. 602 UInt64 0 - 604. 603 UInt64 1 - 605. 604 UInt64 0 - 606. 605 UInt64 1 - 607. 606 UInt64 0 - 608. 607 UInt64 1 - 609. 608 UInt64 0 - 610. 609 UInt64 1 - 611. 610 UInt64 0 - 612. 611 UInt64 1 - 613. 612 UInt64 0 - 614. 613 UInt64 1 - 615. 614 UInt64 0 - 616. 615 UInt64 1 - 617. 616 UInt64 0 - 618. 617 UInt64 1 - 619. 618 UInt64 0 - 620. 619 UInt64 1 - 621. 620 UInt64 0 - 622. 621 UInt64 1 - 623. 622 UInt64 0 - 624. 623 UInt64 1 - 625. 624 UInt64 0 - 626. 625 UInt64 1 - 627. 626 UInt64 0 - 628. 627 UInt64 1 - 629. 628 UInt64 0 - 630. 629 UInt64 1 - 631. 630 UInt64 0 - 632. 631 UInt64 1 - 633. 632 UInt64 0 - 634. 633 UInt64 1 - 635. 634 UInt64 0 - 636. 635 UInt64 1 - 637. 636 UInt64 0 - 638. 637 UInt64 1 - 639. 638 UInt64 0 - 640. 639 UInt64 1 - 641. 640 UInt64 0 - 642. 641 UInt64 1 - 643. 642 UInt64 0 - 644. 643 UInt64 1 - 645. 644 UInt64 0 - 646. 645 UInt64 1 - 647. 646 UInt64 0 - 648. 647 UInt64 1 - 649. 648 UInt64 0 - 650. 649 UInt64 1 - 651. 650 UInt64 0 - 652. 651 UInt64 1 - 653. 652 UInt64 0 - 654. 653 UInt64 1 - 655. 654 UInt64 0 - 656. 655 UInt64 1 - 657. 656 UInt64 0 - 658. 657 UInt64 1 - 659. 658 UInt64 0 - 660. 659 UInt64 1 - 661. 660 UInt64 0 - 662. 661 UInt64 1 - 663. 662 UInt64 0 - 664. 663 UInt64 1 - 665. 664 UInt64 0 - 666. 665 UInt64 1 - 667. 666 UInt64 0 - 668. 667 UInt64 1 - 669. 668 UInt64 0 - 670. 669 UInt64 1 - 671. 670 UInt64 0 - 672. 671 UInt64 1 - 673. 672 UInt64 0 - 674. 673 UInt64 1 - 675. 674 UInt64 0 - 676. 675 UInt64 1 - 677. 676 UInt64 0 - 678. 677 UInt64 1 - 679. 678 UInt64 0 - 680. 679 UInt64 1 - 681. 680 UInt64 0 - 682. 681 UInt64 1 - 683. 682 UInt64 0 - 684. 683 UInt64 1 - 685. 684 UInt64 0 - 686. 685 UInt64 1 - 687. 686 UInt64 0 - 688. 687 UInt64 1 - 689. 688 UInt64 0 - 690. 689 UInt64 1 - 691. 690 UInt64 0 - 692. 691 UInt64 1 - 693. 692 UInt64 0 - 694. 693 UInt64 1 - 695. 694 UInt64 0 - 696. 695 UInt64 1 - 697. 696 UInt64 0 - 698. 697 UInt64 1 - 699. 698 UInt64 0 - 700. 699 UInt64 1 - 701. 700 UInt64 0 - 702. 701 UInt64 1 - 703. 702 UInt64 0 - 704. 703 UInt64 1 - 705. 704 UInt64 0 - 706. 705 UInt64 1 - 707. 706 UInt64 0 - 708. 707 UInt64 1 - 709. 708 UInt64 0 - 710. 709 UInt64 1 - 711. 710 UInt64 0 - 712. 711 UInt64 1 - 713. 712 UInt64 0 - 714. 713 UInt64 1 - 715. 714 UInt64 0 - 716. 715 UInt64 1 - 717. 716 UInt64 0 - 718. 717 UInt64 1 - 719. 718 UInt64 0 - 720. 719 UInt64 1 - 721. 720 UInt64 0 - 722. 721 UInt64 1 - 723. 722 UInt64 0 - 724. 723 UInt64 1 - 725. 724 UInt64 0 - 726. 725 UInt64 1 - 727. 726 UInt64 0 - 728. 727 UInt64 1 - 729. 728 UInt64 0 - 730. 729 UInt64 1 - 731. 730 UInt64 0 - 732. 731 UInt64 1 - 733. 732 UInt64 0 - 734. 733 UInt64 1 - 735. 734 UInt64 0 - 736. 735 UInt64 1 - 737. 736 UInt64 0 - 738. 737 UInt64 1 - 739. 738 UInt64 0 - 740. 739 UInt64 1 - 741. 740 UInt64 0 - 742. 741 UInt64 1 - 743. 742 UInt64 0 - 744. 743 UInt64 1 - 745. 744 UInt64 0 - 746. 745 UInt64 1 - 747. 746 UInt64 0 - 748. 747 UInt64 1 - 749. 748 UInt64 0 - 750. 749 UInt64 1 - 751. 750 UInt64 0 - 752. 751 UInt64 1 - 753. 752 UInt64 0 - 754. 753 UInt64 1 - 755. 754 UInt64 0 - 756. 755 UInt64 1 - 757. 756 UInt64 0 - 758. 757 UInt64 1 - 759. 758 UInt64 0 - 760. 759 UInt64 1 - 761. 760 UInt64 0 - 762. 761 UInt64 1 - 763. 762 UInt64 0 - 764. 763 UInt64 1 - 765. 764 UInt64 0 - 766. 765 UInt64 1 - 767. 766 UInt64 0 - 768. 767 UInt64 1 - 769. 768 UInt64 0 - 770. 769 UInt64 1 - 771. 770 UInt64 0 - 772. 771 UInt64 1 - 773. 772 UInt64 0 - 774. 773 UInt64 1 - 775. 774 UInt64 0 - 776. 775 UInt64 1 - 777. 776 UInt64 0 - 778. 777 UInt64 1 - 779. 778 UInt64 0 - 780. 779 UInt64 1 - 781. 780 UInt64 0 - 782. 781 UInt64 1 - 783. 782 UInt64 0 - 784. 783 UInt64 1 - 785. 784 UInt64 0 - 786. 785 UInt64 1 - 787. 786 UInt64 0 - 788. 787 UInt64 1 - 789. 788 UInt64 0 - 790. 789 UInt64 1 - 791. 790 UInt64 0 - 792. 791 UInt64 1 - 793. 792 UInt64 0 - 794. 793 UInt64 1 - 795. 794 UInt64 0 - 796. 795 UInt64 1 - 797. 796 UInt64 0 - 798. 797 UInt64 1 - 799. 798 UInt64 0 - 800. 799 UInt64 1 - 801. 800 UInt64 0 - 802. 801 UInt64 1 - 803. 802 UInt64 0 - 804. 803 UInt64 1 - 805. 804 UInt64 0 - 806. 805 UInt64 1 - 807. 806 UInt64 0 - 808. 807 UInt64 1 - 809. 808 UInt64 0 - 810. 809 UInt64 1 - 811. 810 UInt64 0 - 812. 811 UInt64 1 - 813. 812 UInt64 0 - 814. 813 UInt64 1 - 815. 814 UInt64 0 - 816. 815 UInt64 1 - 817. 816 UInt64 0 - 818. 817 UInt64 1 - 819. 818 UInt64 0 - 820. 819 UInt64 1 - 821. 820 UInt64 0 - 822. 821 UInt64 1 - 823. 822 UInt64 0 - 824. 823 UInt64 1 - 825. 824 UInt64 0 - 826. 825 UInt64 1 - 827. 826 UInt64 0 - 828. 827 UInt64 1 - 829. 828 UInt64 0 - 830. 829 UInt64 1 - 831. 830 UInt64 0 - 832. 831 UInt64 1 - 833. 832 UInt64 0 - 834. 833 UInt64 1 - 835. 834 UInt64 0 - 836. 835 UInt64 1 - 837. 836 UInt64 0 - 838. 837 UInt64 1 - 839. 838 UInt64 0 - 840. 839 UInt64 1 - 841. 840 UInt64 0 - 842. 841 UInt64 1 - 843. 842 UInt64 0 - 844. 843 UInt64 1 - 845. 844 UInt64 0 - 846. 845 UInt64 1 - 847. 846 UInt64 0 - 848. 847 UInt64 1 - 849. 848 UInt64 0 - 850. 849 UInt64 1 - 851. 850 UInt64 0 - 852. 851 UInt64 1 - 853. 852 UInt64 0 - 854. 853 UInt64 1 - 855. 854 UInt64 0 - 856. 855 UInt64 1 - 857. 856 UInt64 0 - 858. 857 UInt64 1 - 859. 858 UInt64 0 - 860. 859 UInt64 1 - 861. 860 UInt64 0 - 862. 861 UInt64 1 - 863. 862 UInt64 0 - 864. 863 UInt64 1 - 865. 864 UInt64 0 - 866. 865 UInt64 1 - 867. 866 UInt64 0 - 868. 867 UInt64 1 - 869. 868 UInt64 0 - 870. 869 UInt64 1 - 871. 870 UInt64 0 - 872. 871 UInt64 1 - 873. 872 UInt64 0 - 874. 873 UInt64 1 - 875. 874 UInt64 0 - 876. 875 UInt64 1 - 877. 876 UInt64 0 - 878. 877 UInt64 1 - 879. 878 UInt64 0 - 880. 879 UInt64 1 - 881. 880 UInt64 0 - 882. 881 UInt64 1 - 883. 882 UInt64 0 - 884. 883 UInt64 1 - 885. 884 UInt64 0 - 886. 885 UInt64 1 - 887. 886 UInt64 0 - 888. 887 UInt64 1 - 889. 888 UInt64 0 - 890. 889 UInt64 1 - 891. 890 UInt64 0 - 892. 891 UInt64 1 - 893. 892 UInt64 0 - 894. 893 UInt64 1 - 895. 894 UInt64 0 - 896. 895 UInt64 1 - 897. 896 UInt64 0 - 898. 897 UInt64 1 - 899. 898 UInt64 0 - 900. 899 UInt64 1 - 901. 900 UInt64 0 - 902. 901 UInt64 1 - 903. 902 UInt64 0 - 904. 903 UInt64 1 - 905. 904 UInt64 0 - 906. 905 UInt64 1 - 907. 906 UInt64 0 - 908. 907 UInt64 1 - 909. 908 UInt64 0 - 910. 909 UInt64 1 - 911. 910 UInt64 0 - 912. 911 UInt64 1 - 913. 912 UInt64 0 - 914. 913 UInt64 1 - 915. 914 UInt64 0 - 916. 915 UInt64 1 - 917. 916 UInt64 0 - 918. 917 UInt64 1 - 919. 918 UInt64 0 - 920. 919 UInt64 1 - 921. 920 UInt64 0 - 922. 921 UInt64 1 - 923. 922 UInt64 0 - 924. 923 UInt64 1 - 925. 924 UInt64 0 - 926. 925 UInt64 1 - 927. 926 UInt64 0 - 928. 927 UInt64 1 - 929. 928 UInt64 0 - 930. 929 UInt64 1 - 931. 930 UInt64 0 - 932. 931 UInt64 1 - 933. 932 UInt64 0 - 934. 933 UInt64 1 - 935. 934 UInt64 0 - 936. 935 UInt64 1 - 937. 936 UInt64 0 - 938. 937 UInt64 1 - 939. 938 UInt64 0 - 940. 939 UInt64 1 - 941. 940 UInt64 0 - 942. 941 UInt64 1 - 943. 942 UInt64 0 - 944. 943 UInt64 1 - 945. 944 UInt64 0 - 946. 945 UInt64 1 - 947. 946 UInt64 0 - 948. 947 UInt64 1 - 949. 948 UInt64 0 - 950. 949 UInt64 1 - 951. 950 UInt64 0 - 952. 951 UInt64 1 - 953. 952 UInt64 0 - 954. 953 UInt64 1 - 955. 954 UInt64 0 - 956. 955 UInt64 1 - 957. 956 UInt64 0 - 958. 957 UInt64 1 - 959. 958 UInt64 0 - 960. 959 UInt64 1 - 961. 960 UInt64 0 - 962. 961 UInt64 1 - 963. 962 UInt64 0 - 964. 963 UInt64 1 - 965. 964 UInt64 0 - 966. 965 UInt64 1 - 967. 966 UInt64 0 - 968. 967 UInt64 1 - 969. 968 UInt64 0 - 970. 969 UInt64 1 - 971. 970 UInt64 0 - 972. 971 UInt64 1 - 973. 972 UInt64 0 - 974. 973 UInt64 1 - 975. 974 UInt64 0 - 976. 975 UInt64 1 - 977. 976 UInt64 0 - 978. 977 UInt64 1 - 979. 978 UInt64 0 - 980. 979 UInt64 1 - 981. 980 UInt64 0 - 982. 981 UInt64 1 - 983. 982 UInt64 0 - 984. 983 UInt64 1 - 985. 984 UInt64 0 - 986. 985 UInt64 1 - 987. 986 UInt64 0 - 988. 987 UInt64 1 - 989. 988 UInt64 0 - 990. 989 UInt64 1 - 991. 990 UInt64 0 - 992. 991 UInt64 1 - 993. 992 UInt64 0 - 994. 993 UInt64 1 - 995. 994 UInt64 0 - 996. 995 UInt64 1 - 997. 996 UInt64 0 - 998. 997 UInt64 1 - 999. 998 UInt64 0 -1000. 999 UInt64 1 + number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 +100. 99 UInt64 1 - number toTypeName(number) mod(number, 2) + number toTypeName(number) mod(number, 2) - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 - 100. 99 UInt64 1 - 101. 100 UInt64 0 - 102. 101 UInt64 1 - 103. 102 UInt64 0 - 104. 103 UInt64 1 - 105. 104 UInt64 0 - 106. 105 UInt64 1 - 107. 106 UInt64 0 - 108. 107 UInt64 1 - 109. 108 UInt64 0 - 110. 109 UInt64 1 - 111. 110 UInt64 0 - 112. 111 UInt64 1 - 113. 112 UInt64 0 - 114. 113 UInt64 1 - 115. 114 UInt64 0 - 116. 115 UInt64 1 - 117. 116 UInt64 0 - 118. 117 UInt64 1 - 119. 118 UInt64 0 - 120. 119 UInt64 1 - 121. 120 UInt64 0 - 122. 121 UInt64 1 - 123. 122 UInt64 0 - 124. 123 UInt64 1 - 125. 124 UInt64 0 - 126. 125 UInt64 1 - 127. 126 UInt64 0 - 128. 127 UInt64 1 - 129. 128 UInt64 0 - 130. 129 UInt64 1 - 131. 130 UInt64 0 - 132. 131 UInt64 1 - 133. 132 UInt64 0 - 134. 133 UInt64 1 - 135. 134 UInt64 0 - 136. 135 UInt64 1 - 137. 136 UInt64 0 - 138. 137 UInt64 1 - 139. 138 UInt64 0 - 140. 139 UInt64 1 - 141. 140 UInt64 0 - 142. 141 UInt64 1 - 143. 142 UInt64 0 - 144. 143 UInt64 1 - 145. 144 UInt64 0 - 146. 145 UInt64 1 - 147. 146 UInt64 0 - 148. 147 UInt64 1 - 149. 148 UInt64 0 - 150. 149 UInt64 1 - 151. 150 UInt64 0 - 152. 151 UInt64 1 - 153. 152 UInt64 0 - 154. 153 UInt64 1 - 155. 154 UInt64 0 - 156. 155 UInt64 1 - 157. 156 UInt64 0 - 158. 157 UInt64 1 - 159. 158 UInt64 0 - 160. 159 UInt64 1 - 161. 160 UInt64 0 - 162. 161 UInt64 1 - 163. 162 UInt64 0 - 164. 163 UInt64 1 - 165. 164 UInt64 0 - 166. 165 UInt64 1 - 167. 166 UInt64 0 - 168. 167 UInt64 1 - 169. 168 UInt64 0 - 170. 169 UInt64 1 - 171. 170 UInt64 0 - 172. 171 UInt64 1 - 173. 172 UInt64 0 - 174. 173 UInt64 1 - 175. 174 UInt64 0 - 176. 175 UInt64 1 - 177. 176 UInt64 0 - 178. 177 UInt64 1 - 179. 178 UInt64 0 - 180. 179 UInt64 1 - 181. 180 UInt64 0 - 182. 181 UInt64 1 - 183. 182 UInt64 0 - 184. 183 UInt64 1 - 185. 184 UInt64 0 - 186. 185 UInt64 1 - 187. 186 UInt64 0 - 188. 187 UInt64 1 - 189. 188 UInt64 0 - 190. 189 UInt64 1 - 191. 190 UInt64 0 - 192. 191 UInt64 1 - 193. 192 UInt64 0 - 194. 193 UInt64 1 - 195. 194 UInt64 0 - 196. 195 UInt64 1 - 197. 196 UInt64 0 - 198. 197 UInt64 1 - 199. 198 UInt64 0 - 200. 199 UInt64 1 - 201. 200 UInt64 0 - 202. 201 UInt64 1 - 203. 202 UInt64 0 - 204. 203 UInt64 1 - 205. 204 UInt64 0 - 206. 205 UInt64 1 - 207. 206 UInt64 0 - 208. 207 UInt64 1 - 209. 208 UInt64 0 - 210. 209 UInt64 1 - 211. 210 UInt64 0 - 212. 211 UInt64 1 - 213. 212 UInt64 0 - 214. 213 UInt64 1 - 215. 214 UInt64 0 - 216. 215 UInt64 1 - 217. 216 UInt64 0 - 218. 217 UInt64 1 - 219. 218 UInt64 0 - 220. 219 UInt64 1 - 221. 220 UInt64 0 - 222. 221 UInt64 1 - 223. 222 UInt64 0 - 224. 223 UInt64 1 - 225. 224 UInt64 0 - 226. 225 UInt64 1 - 227. 226 UInt64 0 - 228. 227 UInt64 1 - 229. 228 UInt64 0 - 230. 229 UInt64 1 - 231. 230 UInt64 0 - 232. 231 UInt64 1 - 233. 232 UInt64 0 - 234. 233 UInt64 1 - 235. 234 UInt64 0 - 236. 235 UInt64 1 - 237. 236 UInt64 0 - 238. 237 UInt64 1 - 239. 238 UInt64 0 - 240. 239 UInt64 1 - 241. 240 UInt64 0 - 242. 241 UInt64 1 - 243. 242 UInt64 0 - 244. 243 UInt64 1 - 245. 244 UInt64 0 - 246. 245 UInt64 1 - 247. 246 UInt64 0 - 248. 247 UInt64 1 - 249. 248 UInt64 0 - 250. 249 UInt64 1 - 251. 250 UInt64 0 - 252. 251 UInt64 1 - 253. 252 UInt64 0 - 254. 253 UInt64 1 - 255. 254 UInt64 0 - 256. 255 UInt64 1 - 257. 256 UInt64 0 - 258. 257 UInt64 1 - 259. 258 UInt64 0 - 260. 259 UInt64 1 - 261. 260 UInt64 0 - 262. 261 UInt64 1 - 263. 262 UInt64 0 - 264. 263 UInt64 1 - 265. 264 UInt64 0 - 266. 265 UInt64 1 - 267. 266 UInt64 0 - 268. 267 UInt64 1 - 269. 268 UInt64 0 - 270. 269 UInt64 1 - 271. 270 UInt64 0 - 272. 271 UInt64 1 - 273. 272 UInt64 0 - 274. 273 UInt64 1 - 275. 274 UInt64 0 - 276. 275 UInt64 1 - 277. 276 UInt64 0 - 278. 277 UInt64 1 - 279. 278 UInt64 0 - 280. 279 UInt64 1 - 281. 280 UInt64 0 - 282. 281 UInt64 1 - 283. 282 UInt64 0 - 284. 283 UInt64 1 - 285. 284 UInt64 0 - 286. 285 UInt64 1 - 287. 286 UInt64 0 - 288. 287 UInt64 1 - 289. 288 UInt64 0 - 290. 289 UInt64 1 - 291. 290 UInt64 0 - 292. 291 UInt64 1 - 293. 292 UInt64 0 - 294. 293 UInt64 1 - 295. 294 UInt64 0 - 296. 295 UInt64 1 - 297. 296 UInt64 0 - 298. 297 UInt64 1 - 299. 298 UInt64 0 - 300. 299 UInt64 1 - 301. 300 UInt64 0 - 302. 301 UInt64 1 - 303. 302 UInt64 0 - 304. 303 UInt64 1 - 305. 304 UInt64 0 - 306. 305 UInt64 1 - 307. 306 UInt64 0 - 308. 307 UInt64 1 - 309. 308 UInt64 0 - 310. 309 UInt64 1 - 311. 310 UInt64 0 - 312. 311 UInt64 1 - 313. 312 UInt64 0 - 314. 313 UInt64 1 - 315. 314 UInt64 0 - 316. 315 UInt64 1 - 317. 316 UInt64 0 - 318. 317 UInt64 1 - 319. 318 UInt64 0 - 320. 319 UInt64 1 - 321. 320 UInt64 0 - 322. 321 UInt64 1 - 323. 322 UInt64 0 - 324. 323 UInt64 1 - 325. 324 UInt64 0 - 326. 325 UInt64 1 - 327. 326 UInt64 0 - 328. 327 UInt64 1 - 329. 328 UInt64 0 - 330. 329 UInt64 1 - 331. 330 UInt64 0 - 332. 331 UInt64 1 - 333. 332 UInt64 0 - 334. 333 UInt64 1 - 335. 334 UInt64 0 - 336. 335 UInt64 1 - 337. 336 UInt64 0 - 338. 337 UInt64 1 - 339. 338 UInt64 0 - 340. 339 UInt64 1 - 341. 340 UInt64 0 - 342. 341 UInt64 1 - 343. 342 UInt64 0 - 344. 343 UInt64 1 - 345. 344 UInt64 0 - 346. 345 UInt64 1 - 347. 346 UInt64 0 - 348. 347 UInt64 1 - 349. 348 UInt64 0 - 350. 349 UInt64 1 - 351. 350 UInt64 0 - 352. 351 UInt64 1 - 353. 352 UInt64 0 - 354. 353 UInt64 1 - 355. 354 UInt64 0 - 356. 355 UInt64 1 - 357. 356 UInt64 0 - 358. 357 UInt64 1 - 359. 358 UInt64 0 - 360. 359 UInt64 1 - 361. 360 UInt64 0 - 362. 361 UInt64 1 - 363. 362 UInt64 0 - 364. 363 UInt64 1 - 365. 364 UInt64 0 - 366. 365 UInt64 1 - 367. 366 UInt64 0 - 368. 367 UInt64 1 - 369. 368 UInt64 0 - 370. 369 UInt64 1 - 371. 370 UInt64 0 - 372. 371 UInt64 1 - 373. 372 UInt64 0 - 374. 373 UInt64 1 - 375. 374 UInt64 0 - 376. 375 UInt64 1 - 377. 376 UInt64 0 - 378. 377 UInt64 1 - 379. 378 UInt64 0 - 380. 379 UInt64 1 - 381. 380 UInt64 0 - 382. 381 UInt64 1 - 383. 382 UInt64 0 - 384. 383 UInt64 1 - 385. 384 UInt64 0 - 386. 385 UInt64 1 - 387. 386 UInt64 0 - 388. 387 UInt64 1 - 389. 388 UInt64 0 - 390. 389 UInt64 1 - 391. 390 UInt64 0 - 392. 391 UInt64 1 - 393. 392 UInt64 0 - 394. 393 UInt64 1 - 395. 394 UInt64 0 - 396. 395 UInt64 1 - 397. 396 UInt64 0 - 398. 397 UInt64 1 - 399. 398 UInt64 0 - 400. 399 UInt64 1 - 401. 400 UInt64 0 - 402. 401 UInt64 1 - 403. 402 UInt64 0 - 404. 403 UInt64 1 - 405. 404 UInt64 0 - 406. 405 UInt64 1 - 407. 406 UInt64 0 - 408. 407 UInt64 1 - 409. 408 UInt64 0 - 410. 409 UInt64 1 - 411. 410 UInt64 0 - 412. 411 UInt64 1 - 413. 412 UInt64 0 - 414. 413 UInt64 1 - 415. 414 UInt64 0 - 416. 415 UInt64 1 - 417. 416 UInt64 0 - 418. 417 UInt64 1 - 419. 418 UInt64 0 - 420. 419 UInt64 1 - 421. 420 UInt64 0 - 422. 421 UInt64 1 - 423. 422 UInt64 0 - 424. 423 UInt64 1 - 425. 424 UInt64 0 - 426. 425 UInt64 1 - 427. 426 UInt64 0 - 428. 427 UInt64 1 - 429. 428 UInt64 0 - 430. 429 UInt64 1 - 431. 430 UInt64 0 - 432. 431 UInt64 1 - 433. 432 UInt64 0 - 434. 433 UInt64 1 - 435. 434 UInt64 0 - 436. 435 UInt64 1 - 437. 436 UInt64 0 - 438. 437 UInt64 1 - 439. 438 UInt64 0 - 440. 439 UInt64 1 - 441. 440 UInt64 0 - 442. 441 UInt64 1 - 443. 442 UInt64 0 - 444. 443 UInt64 1 - 445. 444 UInt64 0 - 446. 445 UInt64 1 - 447. 446 UInt64 0 - 448. 447 UInt64 1 - 449. 448 UInt64 0 - 450. 449 UInt64 1 - 451. 450 UInt64 0 - 452. 451 UInt64 1 - 453. 452 UInt64 0 - 454. 453 UInt64 1 - 455. 454 UInt64 0 - 456. 455 UInt64 1 - 457. 456 UInt64 0 - 458. 457 UInt64 1 - 459. 458 UInt64 0 - 460. 459 UInt64 1 - 461. 460 UInt64 0 - 462. 461 UInt64 1 - 463. 462 UInt64 0 - 464. 463 UInt64 1 - 465. 464 UInt64 0 - 466. 465 UInt64 1 - 467. 466 UInt64 0 - 468. 467 UInt64 1 - 469. 468 UInt64 0 - 470. 469 UInt64 1 - 471. 470 UInt64 0 - 472. 471 UInt64 1 - 473. 472 UInt64 0 - 474. 473 UInt64 1 - 475. 474 UInt64 0 - 476. 475 UInt64 1 - 477. 476 UInt64 0 - 478. 477 UInt64 1 - 479. 478 UInt64 0 - 480. 479 UInt64 1 - 481. 480 UInt64 0 - 482. 481 UInt64 1 - 483. 482 UInt64 0 - 484. 483 UInt64 1 - 485. 484 UInt64 0 - 486. 485 UInt64 1 - 487. 486 UInt64 0 - 488. 487 UInt64 1 - 489. 488 UInt64 0 - 490. 489 UInt64 1 - 491. 490 UInt64 0 - 492. 491 UInt64 1 - 493. 492 UInt64 0 - 494. 493 UInt64 1 - 495. 494 UInt64 0 - 496. 495 UInt64 1 - 497. 496 UInt64 0 - 498. 497 UInt64 1 - 499. 498 UInt64 0 - 500. 499 UInt64 1 - 501. 500 UInt64 0 - 502. 501 UInt64 1 - 503. 502 UInt64 0 - 504. 503 UInt64 1 - 505. 504 UInt64 0 - 506. 505 UInt64 1 - 507. 506 UInt64 0 - 508. 507 UInt64 1 - 509. 508 UInt64 0 - 510. 509 UInt64 1 - 511. 510 UInt64 0 - 512. 511 UInt64 1 - 513. 512 UInt64 0 - 514. 513 UInt64 1 - 515. 514 UInt64 0 - 516. 515 UInt64 1 - 517. 516 UInt64 0 - 518. 517 UInt64 1 - 519. 518 UInt64 0 - 520. 519 UInt64 1 - 521. 520 UInt64 0 - 522. 521 UInt64 1 - 523. 522 UInt64 0 - 524. 523 UInt64 1 - 525. 524 UInt64 0 - 526. 525 UInt64 1 - 527. 526 UInt64 0 - 528. 527 UInt64 1 - 529. 528 UInt64 0 - 530. 529 UInt64 1 - 531. 530 UInt64 0 - 532. 531 UInt64 1 - 533. 532 UInt64 0 - 534. 533 UInt64 1 - 535. 534 UInt64 0 - 536. 535 UInt64 1 - 537. 536 UInt64 0 - 538. 537 UInt64 1 - 539. 538 UInt64 0 - 540. 539 UInt64 1 - 541. 540 UInt64 0 - 542. 541 UInt64 1 - 543. 542 UInt64 0 - 544. 543 UInt64 1 - 545. 544 UInt64 0 - 546. 545 UInt64 1 - 547. 546 UInt64 0 - 548. 547 UInt64 1 - 549. 548 UInt64 0 - 550. 549 UInt64 1 - 551. 550 UInt64 0 - 552. 551 UInt64 1 - 553. 552 UInt64 0 - 554. 553 UInt64 1 - 555. 554 UInt64 0 - 556. 555 UInt64 1 - 557. 556 UInt64 0 - 558. 557 UInt64 1 - 559. 558 UInt64 0 - 560. 559 UInt64 1 - 561. 560 UInt64 0 - 562. 561 UInt64 1 - 563. 562 UInt64 0 - 564. 563 UInt64 1 - 565. 564 UInt64 0 - 566. 565 UInt64 1 - 567. 566 UInt64 0 - 568. 567 UInt64 1 - 569. 568 UInt64 0 - 570. 569 UInt64 1 - 571. 570 UInt64 0 - 572. 571 UInt64 1 - 573. 572 UInt64 0 - 574. 573 UInt64 1 - 575. 574 UInt64 0 - 576. 575 UInt64 1 - 577. 576 UInt64 0 - 578. 577 UInt64 1 - 579. 578 UInt64 0 - 580. 579 UInt64 1 - 581. 580 UInt64 0 - 582. 581 UInt64 1 - 583. 582 UInt64 0 - 584. 583 UInt64 1 - 585. 584 UInt64 0 - 586. 585 UInt64 1 - 587. 586 UInt64 0 - 588. 587 UInt64 1 - 589. 588 UInt64 0 - 590. 589 UInt64 1 - 591. 590 UInt64 0 - 592. 591 UInt64 1 - 593. 592 UInt64 0 - 594. 593 UInt64 1 - 595. 594 UInt64 0 - 596. 595 UInt64 1 - 597. 596 UInt64 0 - 598. 597 UInt64 1 - 599. 598 UInt64 0 - 600. 599 UInt64 1 - 601. 600 UInt64 0 - 602. 601 UInt64 1 - 603. 602 UInt64 0 - 604. 603 UInt64 1 - 605. 604 UInt64 0 - 606. 605 UInt64 1 - 607. 606 UInt64 0 - 608. 607 UInt64 1 - 609. 608 UInt64 0 - 610. 609 UInt64 1 - 611. 610 UInt64 0 - 612. 611 UInt64 1 - 613. 612 UInt64 0 - 614. 613 UInt64 1 - 615. 614 UInt64 0 - 616. 615 UInt64 1 - 617. 616 UInt64 0 - 618. 617 UInt64 1 - 619. 618 UInt64 0 - 620. 619 UInt64 1 - 621. 620 UInt64 0 - 622. 621 UInt64 1 - 623. 622 UInt64 0 - 624. 623 UInt64 1 - 625. 624 UInt64 0 - 626. 625 UInt64 1 - 627. 626 UInt64 0 - 628. 627 UInt64 1 - 629. 628 UInt64 0 - 630. 629 UInt64 1 - 631. 630 UInt64 0 - 632. 631 UInt64 1 - 633. 632 UInt64 0 - 634. 633 UInt64 1 - 635. 634 UInt64 0 - 636. 635 UInt64 1 - 637. 636 UInt64 0 - 638. 637 UInt64 1 - 639. 638 UInt64 0 - 640. 639 UInt64 1 - 641. 640 UInt64 0 - 642. 641 UInt64 1 - 643. 642 UInt64 0 - 644. 643 UInt64 1 - 645. 644 UInt64 0 - 646. 645 UInt64 1 - 647. 646 UInt64 0 - 648. 647 UInt64 1 - 649. 648 UInt64 0 - 650. 649 UInt64 1 - 651. 650 UInt64 0 - 652. 651 UInt64 1 - 653. 652 UInt64 0 - 654. 653 UInt64 1 - 655. 654 UInt64 0 - 656. 655 UInt64 1 - 657. 656 UInt64 0 - 658. 657 UInt64 1 - 659. 658 UInt64 0 - 660. 659 UInt64 1 - 661. 660 UInt64 0 - 662. 661 UInt64 1 - 663. 662 UInt64 0 - 664. 663 UInt64 1 - 665. 664 UInt64 0 - 666. 665 UInt64 1 - 667. 666 UInt64 0 - 668. 667 UInt64 1 - 669. 668 UInt64 0 - 670. 669 UInt64 1 - 671. 670 UInt64 0 - 672. 671 UInt64 1 - 673. 672 UInt64 0 - 674. 673 UInt64 1 - 675. 674 UInt64 0 - 676. 675 UInt64 1 - 677. 676 UInt64 0 - 678. 677 UInt64 1 - 679. 678 UInt64 0 - 680. 679 UInt64 1 - 681. 680 UInt64 0 - 682. 681 UInt64 1 - 683. 682 UInt64 0 - 684. 683 UInt64 1 - 685. 684 UInt64 0 - 686. 685 UInt64 1 - 687. 686 UInt64 0 - 688. 687 UInt64 1 - 689. 688 UInt64 0 - 690. 689 UInt64 1 - 691. 690 UInt64 0 - 692. 691 UInt64 1 - 693. 692 UInt64 0 - 694. 693 UInt64 1 - 695. 694 UInt64 0 - 696. 695 UInt64 1 - 697. 696 UInt64 0 - 698. 697 UInt64 1 - 699. 698 UInt64 0 - 700. 699 UInt64 1 - 701. 700 UInt64 0 - 702. 701 UInt64 1 - 703. 702 UInt64 0 - 704. 703 UInt64 1 - 705. 704 UInt64 0 - 706. 705 UInt64 1 - 707. 706 UInt64 0 - 708. 707 UInt64 1 - 709. 708 UInt64 0 - 710. 709 UInt64 1 - 711. 710 UInt64 0 - 712. 711 UInt64 1 - 713. 712 UInt64 0 - 714. 713 UInt64 1 - 715. 714 UInt64 0 - 716. 715 UInt64 1 - 717. 716 UInt64 0 - 718. 717 UInt64 1 - 719. 718 UInt64 0 - 720. 719 UInt64 1 - 721. 720 UInt64 0 - 722. 721 UInt64 1 - 723. 722 UInt64 0 - 724. 723 UInt64 1 - 725. 724 UInt64 0 - 726. 725 UInt64 1 - 727. 726 UInt64 0 - 728. 727 UInt64 1 - 729. 728 UInt64 0 - 730. 729 UInt64 1 - 731. 730 UInt64 0 - 732. 731 UInt64 1 - 733. 732 UInt64 0 - 734. 733 UInt64 1 - 735. 734 UInt64 0 - 736. 735 UInt64 1 - 737. 736 UInt64 0 - 738. 737 UInt64 1 - 739. 738 UInt64 0 - 740. 739 UInt64 1 - 741. 740 UInt64 0 - 742. 741 UInt64 1 - 743. 742 UInt64 0 - 744. 743 UInt64 1 - 745. 744 UInt64 0 - 746. 745 UInt64 1 - 747. 746 UInt64 0 - 748. 747 UInt64 1 - 749. 748 UInt64 0 - 750. 749 UInt64 1 - 751. 750 UInt64 0 - 752. 751 UInt64 1 - 753. 752 UInt64 0 - 754. 753 UInt64 1 - 755. 754 UInt64 0 - 756. 755 UInt64 1 - 757. 756 UInt64 0 - 758. 757 UInt64 1 - 759. 758 UInt64 0 - 760. 759 UInt64 1 - 761. 760 UInt64 0 - 762. 761 UInt64 1 - 763. 762 UInt64 0 - 764. 763 UInt64 1 - 765. 764 UInt64 0 - 766. 765 UInt64 1 - 767. 766 UInt64 0 - 768. 767 UInt64 1 - 769. 768 UInt64 0 - 770. 769 UInt64 1 - 771. 770 UInt64 0 - 772. 771 UInt64 1 - 773. 772 UInt64 0 - 774. 773 UInt64 1 - 775. 774 UInt64 0 - 776. 775 UInt64 1 - 777. 776 UInt64 0 - 778. 777 UInt64 1 - 779. 778 UInt64 0 - 780. 779 UInt64 1 - 781. 780 UInt64 0 - 782. 781 UInt64 1 - 783. 782 UInt64 0 - 784. 783 UInt64 1 - 785. 784 UInt64 0 - 786. 785 UInt64 1 - 787. 786 UInt64 0 - 788. 787 UInt64 1 - 789. 788 UInt64 0 - 790. 789 UInt64 1 - 791. 790 UInt64 0 - 792. 791 UInt64 1 - 793. 792 UInt64 0 - 794. 793 UInt64 1 - 795. 794 UInt64 0 - 796. 795 UInt64 1 - 797. 796 UInt64 0 - 798. 797 UInt64 1 - 799. 798 UInt64 0 - 800. 799 UInt64 1 - 801. 800 UInt64 0 - 802. 801 UInt64 1 - 803. 802 UInt64 0 - 804. 803 UInt64 1 - 805. 804 UInt64 0 - 806. 805 UInt64 1 - 807. 806 UInt64 0 - 808. 807 UInt64 1 - 809. 808 UInt64 0 - 810. 809 UInt64 1 - 811. 810 UInt64 0 - 812. 811 UInt64 1 - 813. 812 UInt64 0 - 814. 813 UInt64 1 - 815. 814 UInt64 0 - 816. 815 UInt64 1 - 817. 816 UInt64 0 - 818. 817 UInt64 1 - 819. 818 UInt64 0 - 820. 819 UInt64 1 - 821. 820 UInt64 0 - 822. 821 UInt64 1 - 823. 822 UInt64 0 - 824. 823 UInt64 1 - 825. 824 UInt64 0 - 826. 825 UInt64 1 - 827. 826 UInt64 0 - 828. 827 UInt64 1 - 829. 828 UInt64 0 - 830. 829 UInt64 1 - 831. 830 UInt64 0 - 832. 831 UInt64 1 - 833. 832 UInt64 0 - 834. 833 UInt64 1 - 835. 834 UInt64 0 - 836. 835 UInt64 1 - 837. 836 UInt64 0 - 838. 837 UInt64 1 - 839. 838 UInt64 0 - 840. 839 UInt64 1 - 841. 840 UInt64 0 - 842. 841 UInt64 1 - 843. 842 UInt64 0 - 844. 843 UInt64 1 - 845. 844 UInt64 0 - 846. 845 UInt64 1 - 847. 846 UInt64 0 - 848. 847 UInt64 1 - 849. 848 UInt64 0 - 850. 849 UInt64 1 - 851. 850 UInt64 0 - 852. 851 UInt64 1 - 853. 852 UInt64 0 - 854. 853 UInt64 1 - 855. 854 UInt64 0 - 856. 855 UInt64 1 - 857. 856 UInt64 0 - 858. 857 UInt64 1 - 859. 858 UInt64 0 - 860. 859 UInt64 1 - 861. 860 UInt64 0 - 862. 861 UInt64 1 - 863. 862 UInt64 0 - 864. 863 UInt64 1 - 865. 864 UInt64 0 - 866. 865 UInt64 1 - 867. 866 UInt64 0 - 868. 867 UInt64 1 - 869. 868 UInt64 0 - 870. 869 UInt64 1 - 871. 870 UInt64 0 - 872. 871 UInt64 1 - 873. 872 UInt64 0 - 874. 873 UInt64 1 - 875. 874 UInt64 0 - 876. 875 UInt64 1 - 877. 876 UInt64 0 - 878. 877 UInt64 1 - 879. 878 UInt64 0 - 880. 879 UInt64 1 - 881. 880 UInt64 0 - 882. 881 UInt64 1 - 883. 882 UInt64 0 - 884. 883 UInt64 1 - 885. 884 UInt64 0 - 886. 885 UInt64 1 - 887. 886 UInt64 0 - 888. 887 UInt64 1 - 889. 888 UInt64 0 - 890. 889 UInt64 1 - 891. 890 UInt64 0 - 892. 891 UInt64 1 - 893. 892 UInt64 0 - 894. 893 UInt64 1 - 895. 894 UInt64 0 - 896. 895 UInt64 1 - 897. 896 UInt64 0 - 898. 897 UInt64 1 - 899. 898 UInt64 0 - 900. 899 UInt64 1 - 901. 900 UInt64 0 - 902. 901 UInt64 1 - 903. 902 UInt64 0 - 904. 903 UInt64 1 - 905. 904 UInt64 0 - 906. 905 UInt64 1 - 907. 906 UInt64 0 - 908. 907 UInt64 1 - 909. 908 UInt64 0 - 910. 909 UInt64 1 - 911. 910 UInt64 0 - 912. 911 UInt64 1 - 913. 912 UInt64 0 - 914. 913 UInt64 1 - 915. 914 UInt64 0 - 916. 915 UInt64 1 - 917. 916 UInt64 0 - 918. 917 UInt64 1 - 919. 918 UInt64 0 - 920. 919 UInt64 1 - 921. 920 UInt64 0 - 922. 921 UInt64 1 - 923. 922 UInt64 0 - 924. 923 UInt64 1 - 925. 924 UInt64 0 - 926. 925 UInt64 1 - 927. 926 UInt64 0 - 928. 927 UInt64 1 - 929. 928 UInt64 0 - 930. 929 UInt64 1 - 931. 930 UInt64 0 - 932. 931 UInt64 1 - 933. 932 UInt64 0 - 934. 933 UInt64 1 - 935. 934 UInt64 0 - 936. 935 UInt64 1 - 937. 936 UInt64 0 - 938. 937 UInt64 1 - 939. 938 UInt64 0 - 940. 939 UInt64 1 - 941. 940 UInt64 0 - 942. 941 UInt64 1 - 943. 942 UInt64 0 - 944. 943 UInt64 1 - 945. 944 UInt64 0 - 946. 945 UInt64 1 - 947. 946 UInt64 0 - 948. 947 UInt64 1 - 949. 948 UInt64 0 - 950. 949 UInt64 1 - 951. 950 UInt64 0 - 952. 951 UInt64 1 - 953. 952 UInt64 0 - 954. 953 UInt64 1 - 955. 954 UInt64 0 - 956. 955 UInt64 1 - 957. 956 UInt64 0 - 958. 957 UInt64 1 - 959. 958 UInt64 0 - 960. 959 UInt64 1 - 961. 960 UInt64 0 - 962. 961 UInt64 1 - 963. 962 UInt64 0 - 964. 963 UInt64 1 - 965. 964 UInt64 0 - 966. 965 UInt64 1 - 967. 966 UInt64 0 - 968. 967 UInt64 1 - 969. 968 UInt64 0 - 970. 969 UInt64 1 - 971. 970 UInt64 0 - 972. 971 UInt64 1 - 973. 972 UInt64 0 - 974. 973 UInt64 1 - 975. 974 UInt64 0 - 976. 975 UInt64 1 - 977. 976 UInt64 0 - 978. 977 UInt64 1 - 979. 978 UInt64 0 - 980. 979 UInt64 1 - 981. 980 UInt64 0 - 982. 981 UInt64 1 - 983. 982 UInt64 0 - 984. 983 UInt64 1 - 985. 984 UInt64 0 - 986. 985 UInt64 1 - 987. 986 UInt64 0 - 988. 987 UInt64 1 - 989. 988 UInt64 0 - 990. 989 UInt64 1 - 991. 990 UInt64 0 - 992. 991 UInt64 1 - 993. 992 UInt64 0 - 994. 993 UInt64 1 - 995. 994 UInt64 0 - 996. 995 UInt64 1 - 997. 996 UInt64 0 - 998. 997 UInt64 1 - 999. 998 UInt64 0 -1000. 999 UInt64 1 + number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) + 1. 0 UInt64 0 + 2. 1 UInt64 1 + 3. 2 UInt64 0 + 4. 3 UInt64 1 + 5. 4 UInt64 0 + 6. 5 UInt64 1 + 7. 6 UInt64 0 + 8. 7 UInt64 1 + 9. 8 UInt64 0 + 10. 9 UInt64 1 + 11. 10 UInt64 0 + 12. 11 UInt64 1 + 13. 12 UInt64 0 + 14. 13 UInt64 1 + 15. 14 UInt64 0 + 16. 15 UInt64 1 + 17. 16 UInt64 0 + 18. 17 UInt64 1 + 19. 18 UInt64 0 + 20. 19 UInt64 1 + 21. 20 UInt64 0 + 22. 21 UInt64 1 + 23. 22 UInt64 0 + 24. 23 UInt64 1 + 25. 24 UInt64 0 + 26. 25 UInt64 1 + 27. 26 UInt64 0 + 28. 27 UInt64 1 + 29. 28 UInt64 0 + 30. 29 UInt64 1 + 31. 30 UInt64 0 + 32. 31 UInt64 1 + 33. 32 UInt64 0 + 34. 33 UInt64 1 + 35. 34 UInt64 0 + 36. 35 UInt64 1 + 37. 36 UInt64 0 + 38. 37 UInt64 1 + 39. 38 UInt64 0 + 40. 39 UInt64 1 + 41. 40 UInt64 0 + 42. 41 UInt64 1 + 43. 42 UInt64 0 + 44. 43 UInt64 1 + 45. 44 UInt64 0 + 46. 45 UInt64 1 + 47. 46 UInt64 0 + 48. 47 UInt64 1 + 49. 48 UInt64 0 + 50. 49 UInt64 1 + 51. 50 UInt64 0 + 52. 51 UInt64 1 + 53. 52 UInt64 0 + 54. 53 UInt64 1 + 55. 54 UInt64 0 + 56. 55 UInt64 1 + 57. 56 UInt64 0 + 58. 57 UInt64 1 + 59. 58 UInt64 0 + 60. 59 UInt64 1 + 61. 60 UInt64 0 + 62. 61 UInt64 1 + 63. 62 UInt64 0 + 64. 63 UInt64 1 + 65. 64 UInt64 0 + 66. 65 UInt64 1 + 67. 66 UInt64 0 + 68. 67 UInt64 1 + 69. 68 UInt64 0 + 70. 69 UInt64 1 + 71. 70 UInt64 0 + 72. 71 UInt64 1 + 73. 72 UInt64 0 + 74. 73 UInt64 1 + 75. 74 UInt64 0 + 76. 75 UInt64 1 + 77. 76 UInt64 0 + 78. 77 UInt64 1 + 79. 78 UInt64 0 + 80. 79 UInt64 1 + 81. 80 UInt64 0 + 82. 81 UInt64 1 + 83. 82 UInt64 0 + 84. 83 UInt64 1 + 85. 84 UInt64 0 + 86. 85 UInt64 1 + 87. 86 UInt64 0 + 88. 87 UInt64 1 + 89. 88 UInt64 0 + 90. 89 UInt64 1 + 91. 90 UInt64 0 + 92. 91 UInt64 1 + 93. 92 UInt64 0 + 94. 93 UInt64 1 + 95. 94 UInt64 0 + 96. 95 UInt64 1 + 97. 96 UInt64 0 + 98. 97 UInt64 1 + 99. 98 UInt64 0 +100. 99 UInt64 1 + + number toTypeName(number) mod(number, 2) diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.sql b/tests/queries/0_stateless/03169_display_column_names_in_footer.sql index 8d2e36da74a..46779045dca 100644 --- a/tests/queries/0_stateless/03169_display_column_names_in_footer.sql +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.sql @@ -1,14 +1,19 @@ -- https://github.com/ClickHouse/ClickHouse/issues/65035 -SET output_format_pretty_display_footer_column_names=1; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT Pretty; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyNoEscapes; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyNoEscapesMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyCompact; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyCompactNoEscapes; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyCompactMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettyCompactNoEscapesMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettySpace; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettySpaceNoEscapes; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettySpaceMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 1000) FORMAT PrettySpaceNoEscapesMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 49) FORMAT Pretty; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 10) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names_min_rows=9; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapes; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapes; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapesMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapes; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceMonoBlock; +SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapesMonoBlock; From 7ca8f51bf24c5c149abefb9f147ccebd82e2a820 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 12 Jun 2024 09:47:57 +0200 Subject: [PATCH 563/856] Correct typo in docs --- docs/en/operations/settings/settings-formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0a6f26e8e87..670c9c6cbf1 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1708,12 +1708,12 @@ Result: ## output_format_pretty_display_footer_column_names -Display column names in the footer if there are many table rows. The minimum number of rows for which +Display column names in the footer if there are many table rows. Possible values: - 0 — No column names are displayed in the footer. -- 1 — Column names are displayed in the footer if row count is greater than or equal to the threshold value set by [output_format_pretty_display_footer_column_names_min_rows](#output_format_pretty_display_footer_column_names_min_rows). +- 1 — Column names are displayed in the footer if row count is greater than or equal to the threshold value set by [output_format_pretty_display_footer_column_names_min_rows](#output_format_pretty_display_footer_column_names_min_rows) (50 by default). Default value: `1`. From 7683f06188d8dc901bd912c4ace935a4b3f498e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 12 Jun 2024 11:26:21 +0200 Subject: [PATCH 564/856] Revert "S3: reduce retires time for queries, increase retries count for backups" --- src/Backups/BackupIO_S3.cpp | 6 +++--- src/Core/Settings.h | 1 - src/Core/SettingsChangesHistory.h | 1 - src/IO/S3/Client.h | 2 +- .../integration/test_mask_sensitive_info/configs/users.xml | 1 - 5 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 1ea59c1d38b..92f086295a0 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -54,9 +54,9 @@ namespace S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings.region, context->getRemoteHostFilter(), - static_cast(local_settings.s3_max_redirects), - static_cast(local_settings.backup_restore_s3_retry_attempts), - local_settings.enable_s3_requests_logging, + static_cast(global_settings.s3_max_redirects), + static_cast(global_settings.s3_retry_attempts), + global_settings.enable_s3_requests_logging, /* for_disk_s3 = */ false, request_settings.get_request_throttler, request_settings.put_request_throttler, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f99243e285..b3e83092a77 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -517,7 +517,6 @@ class IColumn; M(UInt64, backup_restore_keeper_value_max_size, 1048576, "Maximum size of data of a [Zoo]Keeper's node during backup", 0) \ M(UInt64, backup_restore_batch_size_for_keeper_multiread, 10000, "Maximum size of batch for multiread request to [Zoo]Keeper during backup or restore", 0) \ M(UInt64, backup_restore_batch_size_for_keeper_multi, 1000, "Maximum size of batch for multi request to [Zoo]Keeper during backup or restore", 0) \ - M(UInt64, backup_restore_s3_retry_attempts, 1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore.", 0) \ M(UInt64, max_backup_bandwidth, 0, "The maximum read speed in bytes per second for particular backup on server. Zero means unlimited.", 0) \ \ M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b447421671e..69bc8c5d207 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -115,7 +115,6 @@ static const std::map& error, long attemptedRetries) const override; diff --git a/tests/integration/test_mask_sensitive_info/configs/users.xml b/tests/integration/test_mask_sensitive_info/configs/users.xml index f767216e907..f129a5bb3e3 100644 --- a/tests/integration/test_mask_sensitive_info/configs/users.xml +++ b/tests/integration/test_mask_sensitive_info/configs/users.xml @@ -2,7 +2,6 @@ 5 - 5 From 71d76aa4ac308e52e2663b409cf2e78c7d7b672f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 12 Jun 2024 11:27:56 +0200 Subject: [PATCH 565/856] 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 e7d95d8db72..b100f96befa 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 4b252dea727816380c377ddc2e425d394c1b4d47 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 12 Jun 2024 13:04:42 +0200 Subject: [PATCH 566/856] rework tests with sleep: use sleep_until instead sleep_for --- src/Common/tests/gtest_resolve_pool.cpp | 85 ++++++++++++++++++------- 1 file changed, 63 insertions(+), 22 deletions(-) diff --git a/src/Common/tests/gtest_resolve_pool.cpp b/src/Common/tests/gtest_resolve_pool.cpp index 2391fc8bacf..7cfe158d90f 100644 --- a/src/Common/tests/gtest_resolve_pool.cpp +++ b/src/Common/tests/gtest_resolve_pool.cpp @@ -1,12 +1,39 @@ #include -#include #include #include -#include "base/defines.h" +#include + +#include #include +#include #include -#include + + +using namespace std::literals::chrono_literals; + + +auto now() +{ + return std::chrono::steady_clock::now(); +} + +void sleep_until(auto time_point) +{ + std::this_thread::sleep_until(time_point); +} + +void sleep_for(auto duration) +{ + std::this_thread::sleep_for(duration); +} + +size_t toMilliseconds(auto duration) +{ + return std::chrono::duration_cast(duration).count(); +} + +const auto epsilon = 500us; class ResolvePoolMock : public DB::HostResolver { @@ -267,13 +294,14 @@ TEST_F(ResolvePoolTest, CanFailAndHeal) TEST_F(ResolvePoolTest, CanExpire) { - auto resolver = make_resolver(); + auto history = 5ms; + auto resolver = make_resolver(toMilliseconds(history)); auto expired_addr = resolver->resolve(); ASSERT_TRUE(addresses.contains(*expired_addr)); - addresses.erase(*expired_addr); - sleepForSeconds(1); + + sleep_for(history + epsilon); for (size_t i = 0; i < 1000; ++i) { @@ -310,12 +338,16 @@ TEST_F(ResolvePoolTest, DuplicatesInAddresses) ASSERT_EQ(3, DB::CurrentThread::getProfileEvents()[metrics.discovered]); } -void check_no_failed_address(size_t iteration, auto & resolver, auto & addresses, auto & failed_addr, auto & metrics) +void check_no_failed_address(size_t iteration, auto & resolver, auto & addresses, auto & failed_addr, auto & metrics, auto deadline) { ASSERT_EQ(iteration, DB::CurrentThread::getProfileEvents()[metrics.failed]); for (size_t i = 0; i < 100; ++i) { auto next_addr = resolver->resolve(); + + if (now() > deadline) + break; + ASSERT_TRUE(addresses.contains(*next_addr)); ASSERT_NE(*next_addr, *failed_addr); } @@ -323,52 +355,60 @@ void check_no_failed_address(size_t iteration, auto & resolver, auto & addresses TEST_F(ResolvePoolTest, BannedForConsiquenceFail) { - size_t history_ms = 5; - auto resolver = make_resolver(history_ms); - + auto history = 5ms; + auto resolver = make_resolver(toMilliseconds(history)); auto failed_addr = resolver->resolve(); ASSERT_TRUE(addresses.contains(*failed_addr)); + auto start_at = now(); + failed_addr.setFail(); ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count)); - check_no_failed_address(1, resolver, addresses, failed_addr, metrics); + check_no_failed_address(1, resolver, addresses, failed_addr, metrics, start_at + history - epsilon); + + sleep_until(start_at + history + epsilon); + start_at = now(); - sleepForMilliseconds(history_ms + 1); resolver->update(); ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(0, CurrentMetrics::get(metrics.banned_count)); failed_addr.setFail(); - check_no_failed_address(2, resolver, addresses, failed_addr, metrics); + check_no_failed_address(2, resolver, addresses, failed_addr, metrics, start_at + history - epsilon); + + sleep_until(start_at + history + epsilon); + start_at = now(); - sleepForMilliseconds(history_ms + 1); resolver->update(); ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count)); // ip still banned adter history_ms + update, because it was his second consiquent fail - check_no_failed_address(2, resolver, addresses, failed_addr, metrics); + check_no_failed_address(2, resolver, addresses, failed_addr, metrics, start_at + history - epsilon); } TEST_F(ResolvePoolTest, NoAditionalBannForConcurrentFail) { - size_t history_ms = 5; - auto resolver = make_resolver(history_ms); + auto history = 5ms; + auto resolver = make_resolver(toMilliseconds(history)); auto failed_addr = resolver->resolve(); ASSERT_TRUE(addresses.contains(*failed_addr)); + auto start_at = now(); + failed_addr.setFail(); failed_addr.setFail(); failed_addr.setFail(); ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count)); - check_no_failed_address(3, resolver, addresses, failed_addr, metrics); + check_no_failed_address(3, resolver, addresses, failed_addr, metrics, start_at + history - epsilon); + + sleep_until(start_at + history + epsilon); - sleepForMilliseconds(history_ms + 1); resolver->update(); // ip is cleared after just 1 history_ms interval. ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); @@ -377,8 +417,8 @@ TEST_F(ResolvePoolTest, NoAditionalBannForConcurrentFail) TEST_F(ResolvePoolTest, StillBannedAfterSuccess) { - size_t history_ms = 5; - auto resolver = make_resolver(history_ms); + auto history = 5ms; + auto resolver = make_resolver(toMilliseconds(history)); auto failed_addr = resolver->resolve(); ASSERT_TRUE(addresses.contains(*failed_addr)); @@ -395,11 +435,12 @@ TEST_F(ResolvePoolTest, StillBannedAfterSuccess) } chassert(again_addr); + auto start_at = now(); failed_addr.setFail(); ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count)); - check_no_failed_address(1, resolver, addresses, failed_addr, metrics); + check_no_failed_address(1, resolver, addresses, failed_addr, metrics, start_at + history - epsilon); again_addr = std::nullopt; // success; From 6e9285791d7c8b3812f038114618bf1abf694997 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 12 Jun 2024 13:34:03 +0200 Subject: [PATCH 567/856] 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 a7f3c9fde8f7d483904b5befc078c120d097b467 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Wed, 12 Jun 2024 13:52:19 +0200 Subject: [PATCH 568/856] Update src/Common/tests/gtest_resolve_pool.cpp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Common/tests/gtest_resolve_pool.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/tests/gtest_resolve_pool.cpp b/src/Common/tests/gtest_resolve_pool.cpp index 7cfe158d90f..b760b9b1524 100644 --- a/src/Common/tests/gtest_resolve_pool.cpp +++ b/src/Common/tests/gtest_resolve_pool.cpp @@ -346,7 +346,10 @@ void check_no_failed_address(size_t iteration, auto & resolver, auto & addresses auto next_addr = resolver->resolve(); if (now() > deadline) + { + ASSERT_NE(i, 0); break; + } ASSERT_TRUE(addresses.contains(*next_addr)); ASSERT_NE(*next_addr, *failed_addr); From 89d2de28729fc4593c80a429783019d70f4f8169 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Jun 2024 14:08:05 +0200 Subject: [PATCH 569/856] Fix AWS ECS --- src/Common/ProxyConfiguration.h | 2 +- .../ProxyConfigurationResolverProvider.cpp | 35 +++++++------------ .../ProxyConfigurationResolverProvider.h | 5 ++- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 +- src/IO/S3/Client.cpp | 6 ++-- src/IO/S3/Credentials.h | 3 -- src/IO/S3/PocoHTTPClient.cpp | 19 ++++++++-- src/IO/S3/PocoHTTPClient.h | 14 ++++---- src/IO/S3/PocoHTTPClientFactory.cpp | 5 ++- 9 files changed, 47 insertions(+), 44 deletions(-) diff --git a/src/Common/ProxyConfiguration.h b/src/Common/ProxyConfiguration.h index 97577735bce..0cb53bd4a2e 100644 --- a/src/Common/ProxyConfiguration.h +++ b/src/Common/ProxyConfiguration.h @@ -44,7 +44,7 @@ struct ProxyConfiguration } } - std::string host = std::string{}; + std::string host{}; Protocol protocol = Protocol::HTTP; uint16_t port = 0; bool tunneling = false; diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index 4008ac2d8a5..71e7c9f78dd 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -101,9 +101,8 @@ namespace return configuration.has(config_prefix + ".uri"); } - /* - * New syntax requires protocol prefix " or " - * */ + /* New syntax requires protocol prefix " or " + */ std::optional getProtocolPrefix( ProxyConfiguration::Protocol request_protocol, const String & config_prefix, @@ -119,22 +118,18 @@ namespace return protocol_prefix; } - template std::optional calculatePrefixBasedOnSettingsSyntax( + bool new_syntax, ProxyConfiguration::Protocol request_protocol, const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration ) { if (!configuration.has(config_prefix)) - { return std::nullopt; - } - if constexpr (new_syntax) - { + if (new_syntax) return getProtocolPrefix(request_protocol, config_prefix, configuration); - } return config_prefix; } @@ -144,24 +139,21 @@ std::shared_ptr ProxyConfigurationResolverProvider:: Protocol request_protocol, const Poco::Util::AbstractConfiguration & configuration) { - if (auto resolver = getFromSettings(request_protocol, "proxy", configuration)) - { + if (auto resolver = getFromSettings(true, request_protocol, "proxy", configuration)) return resolver; - } return std::make_shared( request_protocol, isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); } -template std::shared_ptr ProxyConfigurationResolverProvider::getFromSettings( + bool new_syntax, Protocol request_protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration -) + const Poco::Util::AbstractConfiguration & configuration) { - auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(request_protocol, config_prefix, configuration); + auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(new_syntax, request_protocol, config_prefix, configuration); if (!prefix_opt) { @@ -184,20 +176,17 @@ std::shared_ptr ProxyConfigurationResolverProvider:: std::shared_ptr ProxyConfigurationResolverProvider::getFromOldSettingsFormat( Protocol request_protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration -) + const Poco::Util::AbstractConfiguration & configuration) { - /* - * First try to get it from settings only using the combination of config_prefix and configuration. + /* First try to get it from settings only using the combination of config_prefix and configuration. * This logic exists for backward compatibility with old S3 storage specific proxy configuration. * */ - if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(request_protocol, config_prefix + ".proxy", configuration)) + if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(false, request_protocol, config_prefix + ".proxy", configuration)) { return resolver; } - /* - * In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings. + /* In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings. * Falls back to Environment resolver if no configuration is found. * */ return ProxyConfigurationResolverProvider::get(request_protocol, configuration); diff --git a/src/Common/ProxyConfigurationResolverProvider.h b/src/Common/ProxyConfigurationResolverProvider.h index ebf22f7e92a..357b218e499 100644 --- a/src/Common/ProxyConfigurationResolverProvider.h +++ b/src/Common/ProxyConfigurationResolverProvider.h @@ -33,12 +33,11 @@ public: ); private: - template static std::shared_ptr getFromSettings( + bool is_new_syntax, Protocol protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration - ); + const Poco::Util::AbstractConfiguration & configuration); }; } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 139472a8b01..e37c7c11afd 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -115,7 +115,7 @@ std::unique_ptr getClient( /* * Override proxy configuration for backwards compatibility with old configuration format. * */ - if (auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + if (auto proxy_config = ProxyConfigurationResolverProvider::getFromOldSettingsFormat( ProxyConfiguration::protocolFromString(url.uri.getScheme()), config_prefix, config)) { client_configuration.per_request_configuration diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 9229342b8c1..cbb61deea9f 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -972,10 +972,10 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT { auto context = Context::getGlobalContextInstance(); chassert(context); - auto proxy_configuration_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::protocolFromString(protocol), context->getConfigRef()); + auto proxy_configuration_resolver = ProxyConfigurationResolverProvider::get(ProxyConfiguration::protocolFromString(protocol), context->getConfigRef()); - auto per_request_configuration = [=] () { return proxy_configuration_resolver->resolve(); }; - auto error_report = [=] (const DB::ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); }; + auto per_request_configuration = [=]{ return proxy_configuration_resolver->resolve(); }; + auto error_report = [=](const ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); }; auto config = PocoHTTPClientConfiguration( per_request_configuration, diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 8d586223035..89648f07e7d 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -19,9 +19,6 @@ namespace DB::S3 { inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; -inline static constexpr uint64_t DEFAULT_CONNECT_TIMEOUT_MS = 1000; -inline static constexpr uint64_t DEFAULT_REQUEST_TIMEOUT_MS = 30000; -inline static constexpr uint64_t DEFAULT_MAX_CONNECTIONS = 100; inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_TIMEOUT = 5; inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_MAX_REQUESTS = 100; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 1cef43530e0..dcd644c2d81 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -29,6 +30,7 @@ #include + static const int SUCCESS_RESPONSE_MIN = 200; static const int SUCCESS_RESPONSE_MAX = 299; @@ -84,7 +86,7 @@ namespace DB::S3 { PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( - std::function per_request_configuration_, + std::function per_request_configuration_, const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, @@ -94,7 +96,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, - std::function error_report_) + std::function error_report_) : per_request_configuration(per_request_configuration_) , force_region(force_region_) , remote_host_filter(remote_host_filter_) @@ -107,6 +109,8 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_) , error_report(error_report_) { + /// This is used to identify configurations created by us. + userAgent = "ClickHouse"; } void PocoHTTPClientConfiguration::updateSchemeAndRegion() @@ -166,6 +170,17 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config { } +PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration) + : timeouts(ConnectionTimeouts() + .withConnectionTimeout(Poco::Timespan(client_configuration.connectTimeoutMs * 1000)) + .withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) + .withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) + .withTCPKeepAliveTimeout(Poco::Timespan( + client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0))), + remote_host_filter(Context::getGlobalContextInstance()->getRemoteHostFilter()) +{ +} + std::shared_ptr PocoHTTPClient::MakeRequest( const std::shared_ptr & request, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 88251b964e2..18a21649167 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -38,7 +38,7 @@ class PocoHTTPClient; struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration { - std::function per_request_configuration; + std::function per_request_configuration; String force_region; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; @@ -54,13 +54,13 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration size_t http_keep_alive_timeout = DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT; size_t http_keep_alive_max_requests = DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST; - std::function error_report; + std::function error_report; void updateSchemeAndRegion(); private: PocoHTTPClientConfiguration( - std::function per_request_configuration_, + std::function per_request_configuration_, const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, @@ -70,8 +70,7 @@ private: bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, - std::function error_report_ - ); + std::function error_report_); /// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization. friend ClientFactory; @@ -120,6 +119,7 @@ class PocoHTTPClient : public Aws::Http::HttpClient { public: explicit PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration); + explicit PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration); ~PocoHTTPClient() override = default; std::shared_ptr MakeRequest( @@ -166,8 +166,8 @@ protected: static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; - std::function per_request_configuration; - std::function error_report; + std::function per_request_configuration; + std::function error_report; ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; diff --git a/src/IO/S3/PocoHTTPClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp index ef7af2d01ba..b2f84c5e827 100644 --- a/src/IO/S3/PocoHTTPClientFactory.cpp +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -15,7 +15,10 @@ namespace DB::S3 std::shared_ptr PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & client_configuration) const { - return std::make_shared(static_cast(client_configuration)); + if (client_configuration.userAgent == "ClickHouse") + return std::make_shared(static_cast(client_configuration)); + else /// This client is created inside the AWS SDK with default settings to obtain ECS credentials from localhost. + return std::make_shared(client_configuration); } std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest( From d851fa871fe1c732fa3f976654fcab74b6d788fd Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 12 Jun 2024 14:09:37 +0200 Subject: [PATCH 570/856] Fix broken links in docs --- .../aggregate-functions/combinators.md | 8 +-- .../parametric-functions.md | 4 +- .../reference/stochasticlinearregression.md | 4 +- .../aggregate-functions/reference/varpop.md | 8 +-- .../aggregate-functions/reference/varsamp.md | 14 +++--- docs/en/sql-reference/data-types/geo.md | 6 +-- docs/en/sql-reference/dictionaries/index.md | 50 +++++++++---------- .../functions/array-functions.md | 17 +++++-- .../functions/bitmap-functions.md | 4 +- .../functions/date-time-functions.md | 35 +++++++------ .../functions/ext-dict-functions.md | 2 +- .../en/sql-reference/functions/geo/geohash.md | 8 +-- docs/en/sql-reference/functions/geo/h3.md | 2 + docs/en/sql-reference/functions/geo/s2.md | 10 ++-- .../sql-reference/functions/hash-functions.md | 4 +- .../functions/ip-address-functions.md | 2 +- .../sql-reference/functions/json-functions.md | 4 +- .../functions/other-functions.md | 24 ++++----- .../functions/rounding-functions.md | 2 +- .../functions/string-functions.md | 2 +- .../functions/string-search-functions.md | 26 +++++----- .../functions/type-conversion-functions.md | 30 +++++------ .../sql-reference/functions/uuid-functions.md | 4 +- docs/en/sql-reference/operators/in.md | 2 +- .../sql-reference/statements/alter/column.md | 4 +- .../statements/alter/partition.md | 6 +-- .../sql-reference/statements/create/view.md | 2 +- docs/en/sql-reference/statements/grant.md | 6 +-- .../sql-reference/statements/select/sample.md | 16 +++--- docs/en/sql-reference/statements/system.md | 2 +- docs/en/sql-reference/syntax.md | 4 +- docs/en/sql-reference/table-functions/file.md | 4 +- .../table-functions/fileCluster.md | 2 +- 33 files changed, 167 insertions(+), 151 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index 8ccc5e292b5..5351531afdb 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -106,14 +106,14 @@ To work with these states, use: - [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) table engine. - [finalizeAggregation](../../sql-reference/functions/other-functions.md#function-finalizeaggregation) function. - [runningAccumulate](../../sql-reference/functions/other-functions.md#runningaccumulate) function. -- [-Merge](#aggregate_functions_combinators-merge) combinator. -- [-MergeState](#aggregate_functions_combinators-mergestate) combinator. +- [-Merge](#aggregate_functions_combinators_merge) combinator. +- [-MergeState](#aggregate_functions_combinators_mergestate) combinator. -## -Merge +## -Merge {#aggregate_functions_combinators_merge} If you apply this combinator, the aggregate function takes the intermediate aggregation state as an argument, combines the states to finish aggregation, and returns the resulting value. -## -MergeState +## -MergeState {#aggregate_functions_combinators_mergestate} Merges the intermediate aggregation states in the same way as the -Merge combinator. However, it does not return the resulting value, but an intermediate aggregation state, similar to the -State combinator. diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 1dc89b8dcf9..43ded9df60a 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -82,7 +82,7 @@ FROM In this case, you should remember that you do not know the histogram bin borders. -## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) +## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) {#function-sequencematch} Checks whether the sequence contains an event chain that matches the pattern. @@ -172,7 +172,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM - [sequenceCount](#function-sequencecount) -## sequenceCount(pattern)(time, cond1, cond2, ...) +## sequenceCount(pattern)(time, cond1, cond2, ...) {#function-sequencecount} Counts the number of event chains that matched the pattern. The function searches event chains that do not overlap. It starts to search for the next chain after the current chain is matched. diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md index ddac82a0977..15533ba9fd7 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md @@ -3,7 +3,7 @@ slug: /en/sql-reference/aggregate-functions/reference/stochasticlinearregression sidebar_position: 221 --- -# stochasticLinearRegression +# stochasticLinearRegression {#agg_functions_stochasticlinearregression_parameters} This function implements stochastic linear regression. It supports custom parameters for learning rate, L2 regularization coefficient, mini-batch size, and has a few methods for updating weights ([Adam](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Adam) (used by default), [simple SGD](https://en.wikipedia.org/wiki/Stochastic_gradient_descent), [Momentum](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Momentum), and [Nesterov](https://mipt.ru/upload/medialibrary/d7e/41-91.pdf)). @@ -72,5 +72,5 @@ The query will return a column of predicted values. Note that first argument of **See Also** -- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#agg_functions-stochasticlogisticregression) +- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#agg_functions_stochasticlinearregression_parameters) - [Difference between linear and logistic regressions](https://stackoverflow.com/questions/12146914/what-is-the-difference-between-linear-regression-and-logistic-regression) diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpop.md b/docs/en/sql-reference/aggregate-functions/reference/varpop.md index fcabeb4c6a8..d2b19fe2a3e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varpop.md @@ -6,7 +6,7 @@ sidebar_position: 32 This page covers the `varPop` and `varPopStable` functions available in ClickHouse. -## varPop +## varPop {#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`. @@ -27,7 +27,7 @@ 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). +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable`](#varPopStable) function. **Example** @@ -55,7 +55,7 @@ Result: 3 ``` -## varPopStable +## varPopStable {#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. @@ -76,7 +76,7 @@ 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. +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** diff --git a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md index be669a16ae8..e9ec9ba2bc1 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md @@ -40,7 +40,7 @@ Where: 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). +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varSampStable`](#varSampStable) function. **Example** @@ -66,7 +66,7 @@ Response: 0.8650000000000091 ``` -## varSampStable +## varSampStable {#varSampStable} Calculate the sample variance of a data set using a numerically stable algorithm. @@ -82,11 +82,11 @@ varSampStable(expr) **Returned value** -The `varSampStable()` function returns a Float64 value representing the sample variance of the input data set. +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): +The `varSampStable` function calculates the sample variance using the same formula as the [`varSamp`](#varSamp) function: ```plaintext ∑(x - mean(x))^2 / (n - 1) @@ -97,9 +97,9 @@ Where: - `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. +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. +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`](./varpop#varpopstable) function instead. **Example** @@ -125,4 +125,4 @@ 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. +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. diff --git a/docs/en/sql-reference/data-types/geo.md b/docs/en/sql-reference/data-types/geo.md index 7e3c32b3451..7ffc7447d96 100644 --- a/docs/en/sql-reference/data-types/geo.md +++ b/docs/en/sql-reference/data-types/geo.md @@ -33,7 +33,7 @@ Result: ## Ring -`Ring` is a simple polygon without holes stored as an array of points: [Array](array.md)([Point](#point-data-type)). +`Ring` is a simple polygon without holes stored as an array of points: [Array](array.md)([Point](#point)). **Example** @@ -54,7 +54,7 @@ Result: ## Polygon -`Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring-data-type)). First element of outer array is the outer shape of polygon and all the following elements are holes. +`Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring)). First element of outer array is the outer shape of polygon and all the following elements are holes. **Example** @@ -76,7 +76,7 @@ Result: ## MultiPolygon -`MultiPolygon` consists of multiple polygons and is stored as an array of polygons: [Array](array.md)([Polygon](#polygon-data-type)). +`MultiPolygon` consists of multiple polygons and is stored as an array of polygons: [Array](array.md)([Polygon](#polygon)). **Example** diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 080de94f8b7..437b836ec0e 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -16,14 +16,14 @@ ClickHouse supports special functions for working with dictionaries that can be ClickHouse supports: - Dictionaries with a [set of functions](../../sql-reference/functions/ext-dict-functions.md). -- [Embedded dictionaries](#embedded_dictionaries) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md). +- [Embedded dictionaries](#embedded-dictionaries) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md). :::tip Tutorial If you are getting started with Dictionaries in ClickHouse we have a tutorial that covers that topic. Take a look [here](/docs/en/tutorial.md). ::: -You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](#dictionary-sources)”. +You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](#dictionary_sources)”. ClickHouse: @@ -75,14 +75,14 @@ The dictionary configuration file has the following format: ``` -You can [configure](#configuring-a-dictionary) any number of dictionaries in the same file. +You can [configure](#configuring_a_dictionary) any number of dictionaries in the same file. :::note You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../../sql-reference/functions/other-functions.md) function). This functionality is not related to dictionaries. ::: -## Configuring a Dictionary {#configuring-a-dictionary} +## Configuring a Dictionary {#configuring_a_dictionary} @@ -679,7 +679,7 @@ When searching for a dictionary, the cache is searched first. For each block of If keys are not found in dictionary, then update cache task is created and added into update queue. Update queue properties can be controlled with settings `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates`. -For cache dictionaries, the expiration [lifetime](#dictionary-updates) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. +For cache dictionaries, the expiration [lifetime](#lifetime) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the [system.dictionaries](../../operations/system-tables/dictionaries.md) table. @@ -771,7 +771,7 @@ The dictionary is not stored in memory and directly goes to the source during th The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. -All types of [sources](#dictionary-sources), except local files, are supported. +All types of [sources](#dictionary_sources), except local files, are supported. Configuration example: @@ -952,7 +952,7 @@ LIFETIME(MIN 300 MAX 360) If `0` and `0`, ClickHouse does not reload the dictionary by timeout. In this case, ClickHouse can reload the dictionary earlier if the dictionary configuration file was changed or the `SYSTEM RELOAD DICTIONARY` command was executed. -When updating the dictionaries, the ClickHouse server applies different logic depending on the type of [source](#dictionary-sources): +When updating the dictionaries, the ClickHouse server applies different logic depending on the type of [source](#dictionary_sources): - For a text file, it checks the time of modification. If the time differs from the previously recorded time, the dictionary is updated. - For MySQL source, the time of modification is checked using a `SHOW TABLE STATUS` query (in case of MySQL 8 you need to disable meta-information caching in MySQL by `set global information_schema_stats_expiry=0`). @@ -961,7 +961,7 @@ When updating the dictionaries, the ClickHouse server applies different logic de For other sources (ODBC, PostgreSQL, ClickHouse, etc), you can set up a query that will update the dictionaries only if they really changed, rather than each time. To do this, follow these steps: - The dictionary table must have a field that always changes when the source data is updated. -- The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. Specify the query in the `` field in the settings for the [source](#dictionary-sources). +- The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. Specify the query in the `` field in the settings for the [source](#dictionary_sources). Example of settings: @@ -1031,7 +1031,7 @@ SOURCE(CLICKHOUSE(... update_field 'added_time' update_lag 15)) ... ``` -## Dictionary Sources {#dictionary-sources} +## Dictionary Sources {#dictionary_sources} @@ -1092,7 +1092,7 @@ Types of sources (`source_type`): - [Local file](#local_file) - [Executable File](#executable) - [Executable Pool](#executable_pool) -- [HTTP(S)](#http) +- [HTTP(S)](#https) - DBMS - [ODBC](#odbc) - [MySQL](#mysql) @@ -1134,7 +1134,7 @@ When a dictionary with source `FILE` is created via DDL command (`CREATE DICTION ### Executable File {#executable} -Working with executable files depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. +Working with executable files depends on [how the dictionary is stored in memory](#storing-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. Example of settings: @@ -1285,7 +1285,7 @@ Setting fields: - `db` – Name of the database. Omit it if the database name is set in the `` parameters. - `table` – Name of the table and schema if exists. - `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#lifetime). - `query` – The custom query. Optional parameter. :::note @@ -1575,7 +1575,7 @@ Setting fields: - `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in MySQL, for example, `id > 10 AND id < 20`. Optional parameter. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#lifetime). - `fail_on_connection_loss` – The configuration parameter that controls behavior of the server on connection loss. If `true`, an exception is thrown immediately if the connection between client and server was lost. If `false`, the ClickHouse server retries to execute the query three times before throwing an exception. Note that retrying leads to increased response times. Default value: `false`. @@ -1672,7 +1672,7 @@ Setting fields: - `db` – Name of the database. - `table` – Name of the table. - `where` – The selection criteria. May be omitted. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#lifetime). - `secure` - Use ssl for connection. - `query` – The custom query. Optional parameter. @@ -1849,7 +1849,7 @@ Setting fields: - `db` – Name of the database. - `table` – Name of the table. - `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in PostgreSQL. For example, `id > 10 AND id < 20`. Optional parameter. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#lifetime). - `query` – The custom query. Optional parameter. :::note @@ -2030,17 +2030,17 @@ CREATE DICTIONARY somename ( Configuration fields: -| Tag | Description | Required | -|------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------| -| `name` | Column name. | Yes | -| `type` | ClickHouse data type: [UInt8](../../sql-reference/data-types/int-uint.md), [UInt16](../../sql-reference/data-types/int-uint.md), [UInt32](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md), [Int8](../../sql-reference/data-types/int-uint.md), [Int16](../../sql-reference/data-types/int-uint.md), [Int32](../../sql-reference/data-types/int-uint.md), [Int64](../../sql-reference/data-types/int-uint.md), [Float32](../../sql-reference/data-types/float.md), [Float64](../../sql-reference/data-types/float.md), [UUID](../../sql-reference/data-types/uuid.md), [Decimal32](../../sql-reference/data-types/decimal.md), [Decimal64](../../sql-reference/data-types/decimal.md), [Decimal128](../../sql-reference/data-types/decimal.md), [Decimal256](../../sql-reference/data-types/decimal.md),[Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), [String](../../sql-reference/data-types/string.md), [Array](../../sql-reference/data-types/array.md).
ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.
[Nullable](../../sql-reference/data-types/nullable.md) is currently supported for [Flat](#flat), [Hashed](#hashed), [ComplexKeyHashed](#complex_key_hashed), [Direct](#direct), [ComplexKeyDirect](#complex_key_direct), [RangeHashed](#range_hashed), Polygon, [Cache](#cache), [ComplexKeyCache](#complex_key_cache), [SSDCache](#ssd_cache), [SSDComplexKeyCache](#complex_key_ssd_cache) dictionaries. In [IPTrie](#ip_trie) dictionaries `Nullable` types are not supported. | Yes | -| `null_value` | Default value for a non-existing element.
In the example, it is an empty string. [NULL](../syntax.md#null) value can be used only for the `Nullable` types (see the previous line with types description). | Yes | -| `expression` | [Expression](../../sql-reference/syntax.md#expressions) that ClickHouse executes on the value.
The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.

Default value: no expression. | No | -| `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](#hierarchical-dictionaries).

Default value: `false`. | No | -| `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.

Default value: `false`. | No | -| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. +| Tag | Description | Required | +|------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------| +| `name` | Column name. | Yes | +| `type` | ClickHouse data type: [UInt8](../../sql-reference/data-types/int-uint.md), [UInt16](../../sql-reference/data-types/int-uint.md), [UInt32](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md), [Int8](../../sql-reference/data-types/int-uint.md), [Int16](../../sql-reference/data-types/int-uint.md), [Int32](../../sql-reference/data-types/int-uint.md), [Int64](../../sql-reference/data-types/int-uint.md), [Float32](../../sql-reference/data-types/float.md), [Float64](../../sql-reference/data-types/float.md), [UUID](../../sql-reference/data-types/uuid.md), [Decimal32](../../sql-reference/data-types/decimal.md), [Decimal64](../../sql-reference/data-types/decimal.md), [Decimal128](../../sql-reference/data-types/decimal.md), [Decimal256](../../sql-reference/data-types/decimal.md),[Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), [String](../../sql-reference/data-types/string.md), [Array](../../sql-reference/data-types/array.md).
ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.
[Nullable](../../sql-reference/data-types/nullable.md) is currently supported for [Flat](#flat), [Hashed](#hashed), [ComplexKeyHashed](#complex_key_hashed), [Direct](#direct), [ComplexKeyDirect](#complex_key_direct), [RangeHashed](#range_hashed), Polygon, [Cache](#cache), [ComplexKeyCache](#complex_key_cache), [SSDCache](#ssd_cache), [SSDComplexKeyCache](#complex_key_ssd_cache) dictionaries. In [IPTrie](#ip_trie) dictionaries `Nullable` types are not supported. | Yes | +| `null_value` | Default value for a non-existing element.
In the example, it is an empty string. [NULL](../syntax.md#null) value can be used only for the `Nullable` types (see the previous line with types description). | Yes | +| `expression` | [Expression](../../sql-reference/syntax.md#expressions) that ClickHouse executes on the value.
The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.

Default value: no expression. | No | +| `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](#hierarchical_dictionaries).

Default value: `false`. | No | +| `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.

Default value: `false`. | No | +| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. -## Hierarchical Dictionaries {#hierarchical-dictionaries} +## Hierarchical Dictionaries {#hierarchical_dictionaries} ClickHouse supports hierarchical dictionaries with a [numeric key](#numeric-key). diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 7b52fbff714..d87ca4a0fe7 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1261,7 +1261,7 @@ SELECT arraySort((x) -> -x, [1, 2, 3]) as res; └─────────┘ ``` -For each element of the source array, the lambda function returns the sorting key, that is, \[1 –\> -1, 2 –\> -2, 3 –\> -3\]. Since the `arraySort` function sorts the keys in ascending order, the result is \[3, 2, 1\]. Thus, the `(x) –> -x` lambda function sets the [descending order](#reverse-sort) in a sorting. +For each element of the source array, the lambda function returns the sorting key, that is, \[1 –\> -1, 2 –\> -2, 3 –\> -3\]. Since the `arraySort` function sorts the keys in ascending order, the result is \[3, 2, 1\]. Thus, the `(x) –> -x` lambda function sets the [descending order](#arrayreversesort) in a sorting. The lambda function can accept multiple arguments. In this case, you need to pass the `arraySort` function several arrays of identical length that the arguments of lambda function will correspond to. The resulting array will consist of elements from the first input array; elements from the next input array(s) specify the sorting keys. For example: @@ -1307,10 +1307,15 @@ To improve sorting efficiency, the [Schwartzian transform](https://en.wikipedia. Same as `arraySort` with additional `limit` argument allowing partial sorting. Returns an array of the same size as the original array where elements in range `[1..limit]` are sorted in ascending order. Remaining elements `(limit..N]` shall contain elements in unspecified order. -## arrayReverseSort(\[func,\] arr, ...) {#reverse-sort} +## arrayReverseSort Sorts the elements of the `arr` array in descending order. If the `func` function is specified, `arr` is sorted according to the result of the `func` function applied to the elements of the array, and then the sorted array is reversed. If `func` accepts multiple arguments, the `arrayReverseSort` function is passed several arrays that the arguments of `func` will correspond to. Detailed examples are shown at the end of `arrayReverseSort` description. +**Syntax** + +```sql +arrayReverseSort([func,] arr, ...) +``` Example of integer values sorting: ``` sql @@ -1907,10 +1912,16 @@ FROM numbers(1,10); - [arrayReduce](#arrayreduce) -## arrayReverse(arr) +## arrayReverse Returns an array of the same size as the original array containing the elements in reverse order. +**Syntax** + +```sql +arrayReverse(arr) +``` + Example: ``` sql diff --git a/docs/en/sql-reference/functions/bitmap-functions.md b/docs/en/sql-reference/functions/bitmap-functions.md index a5c8a663b71..d98d7d19d7c 100644 --- a/docs/en/sql-reference/functions/bitmap-functions.md +++ b/docs/en/sql-reference/functions/bitmap-functions.md @@ -74,7 +74,7 @@ bitmapSubsetInRange(bitmap, range_start, range_end) **Arguments** -- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild). +- `bitmap` – [Bitmap object](#bitmapbuild). - `range_start` – Start of the range (inclusive). [UInt32](../data-types/int-uint.md). - `range_end` – End of the range (exclusive). [UInt32](../data-types/int-uint.md). @@ -188,7 +188,7 @@ Result: Checks whether two bitmaps intersect. -If `bitmap2` contains exactly one element, consider using [bitmapContains](#bitmap_functions-bitmapcontains) instead as it works more efficiently. +If `bitmap2` contains exactly one element, consider using [bitmapContains](#bitmapcontains) instead as it works more efficiently. **Syntax** diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4092c83954a..e8661b5f5c3 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -83,7 +83,7 @@ Result: ``` ## makeDate32 -Like [makeDate](#makeDate) but produces a [Date32](../data-types/date32.md). +Like [makeDate](#makedate) but produces a [Date32](../data-types/date32.md). ## makeDateTime @@ -214,7 +214,7 @@ Result: **See also** -- [serverTimeZone](#serverTimeZone) +- [serverTimeZone](#servertimezone) ## serverTimeZone @@ -249,7 +249,7 @@ Result: **See also** -- [timeZone](#timeZone) +- [timeZone](#timezone) ## toTimeZone @@ -305,7 +305,7 @@ int32samoa: 1546300800 **See Also** -- [formatDateTime](#formatDateTime) - supports non-constant timezone. +- [formatDateTime](#formatdatetime) - supports non-constant timezone. - [toString](type-conversion-functions.md#tostring) - supports non-constant timezone. ## timeZoneOf @@ -1006,7 +1006,7 @@ toStartOfWeek(t[, mode[, timezone]]) **Arguments** - `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) -- `mode` - determines the first day of the week as described in the [toWeek()](date-time-functions#toweek) function +- `mode` - determines the first day of the week as described in the [toWeek()](#toweek) function - `timezone` - Optional parameter, it behaves like any other conversion function **Returned value** @@ -1719,7 +1719,7 @@ Result: **See Also** -- [fromDaysSinceYearZero](#fromDaysSinceYearZero) +- [fromDaysSinceYearZero](#fromdayssinceyearzero) ## fromDaysSinceYearZero @@ -1759,7 +1759,7 @@ Result: **See Also** -- [toDaysSinceYearZero](#toDaysSinceYearZero) +- [toDaysSinceYearZero](#todayssinceyearzero) ## fromDaysSinceYearZero32 @@ -1982,7 +1982,7 @@ Result: **See Also** -- [toStartOfInterval](#tostartofintervaldate_or_date_with_time-interval-x-unit--time_zone) +- [toStartOfInterval](#tostartofinterval) ## date\_add @@ -2055,7 +2055,7 @@ Result: **See Also** -- [addDate](#addDate) +- [addDate](#adddate) ## date\_sub @@ -2129,7 +2129,7 @@ Result: **See Also** -- [subDate](#subDate) +- [subDate](#subdate) ## timestamp\_add @@ -2310,7 +2310,7 @@ Alias: `SUBDATE` - [date_sub](#date_sub) -## now {#now} +## now Returns the current date and time at the moment of query analysis. The function is a constant expression. @@ -3609,7 +3609,7 @@ SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64 └───────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` -## formatDateTime {#formatDateTime} +## formatDateTime Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. @@ -3734,10 +3734,9 @@ LIMIT 10 **See Also** -- [formatDateTimeInJodaSyntax](##formatDateTimeInJodaSyntax) +- [formatDateTimeInJodaSyntax](#formatdatetimeinjodasyntax) - -## formatDateTimeInJodaSyntax {#formatDateTimeInJodaSyntax} +## formatDateTimeInJodaSyntax Similar to formatDateTime, except that it formats datetime in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. @@ -3902,11 +3901,11 @@ Result: **See Also** -- [fromUnixTimestampInJodaSyntax](##fromUnixTimestampInJodaSyntax) +- [fromUnixTimestampInJodaSyntax](#fromunixtimestampinjodasyntax) ## fromUnixTimestampInJodaSyntax -Same as [fromUnixTimestamp](#fromUnixTimestamp) but when called in the second way (two or three arguments), the formatting is performed using [Joda style](https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) instead of MySQL style. +Same as [fromUnixTimestamp](#fromunixtimestamp) but when called in the second way (two or three arguments), the formatting is performed using [Joda style](https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) instead of MySQL style. **Example:** @@ -4121,7 +4120,7 @@ Result: Returns the current date and time at the moment of query analysis. The function is a constant expression. :::note -This function gives the same result that `now('UTC')` would. It was added only for MySQL support and [`now`](#now-now) is the preferred usage. +This function gives the same result that `now('UTC')` would. It was added only for MySQL support and [`now`](#now) is the preferred usage. ::: **Syntax** diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 82c21ce40c8..093ee690d47 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -12,7 +12,7 @@ For dictionaries created with [DDL queries](../../sql-reference/statements/creat For information on connecting and configuring dictionaries, see [Dictionaries](../../sql-reference/dictionaries/index.md). -## dictGet, dictGetOrDefault, dictGetOrNull {#dictGet} +## dictGet, dictGetOrDefault, dictGetOrNull Retrieves values from a dictionary. diff --git a/docs/en/sql-reference/functions/geo/geohash.md b/docs/en/sql-reference/functions/geo/geohash.md index 8abc8006e5d..9a3d52824f6 100644 --- a/docs/en/sql-reference/functions/geo/geohash.md +++ b/docs/en/sql-reference/functions/geo/geohash.md @@ -4,13 +4,15 @@ sidebar_label: Geohash title: "Functions for Working with Geohash" --- +## Geohash {#geohash_description} + [Geohash](https://en.wikipedia.org/wiki/Geohash) is the geocode system, which subdivides Earth’s surface into buckets of grid shape and encodes each cell into a short string of letters and digits. It is a hierarchical data structure, so the longer is the geohash string, the more precise is the geographic location. If you need to manually convert geographic coordinates to geohash strings, you can use [geohash.org](http://geohash.org/). ## geohashEncode -Encodes latitude and longitude as a [geohash](#geohash)-string. +Encodes latitude and longitude as a [geohash](#geohash_description)-string. ``` sql geohashEncode(longitude, latitude, [precision]) @@ -40,7 +42,7 @@ SELECT geohashEncode(-5.60302734375, 42.593994140625, 0) AS res; ## geohashDecode -Decodes any [geohash](#geohash)-encoded string into longitude and latitude. +Decodes any [geohash](#geohash_description)-encoded string into longitude and latitude. **Input values** @@ -64,7 +66,7 @@ SELECT geohashDecode('ezs42') AS res; ## geohashesInBox -Returns an array of [geohash](#geohash)-encoded strings of given precision that fall inside and intersect boundaries of given box, basically a 2D grid flattened into array. +Returns an array of [geohash](#geohash_description)-encoded strings of given precision that fall inside and intersect boundaries of given box, basically a 2D grid flattened into array. **Syntax** diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index bcdd457964a..6fce91f4d8e 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -4,6 +4,8 @@ sidebar_label: H3 Indexes title: "Functions for Working with H3 Indexes" --- +## H3 Index {#h3index} + [H3](https://eng.uber.com/h3/) is a geographical indexing system where Earth’s surface divided into a grid of even hexagonal cells. This system is hierarchical, i. e. each hexagon on the top level ("parent") can be split into seven even but smaller ones ("children"), and so on. The level of the hierarchy is called `resolution` and can receive a value from `0` till `15`, where `0` is the `base` level with the largest and coarsest cells. diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index 3165b21318b..bcb6b2833c9 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -5,13 +5,15 @@ sidebar_label: S2 Geometry # Functions for Working with S2 Index +## S2Index {#s2_index} + [S2](https://s2geometry.io/) is a geographical indexing system where all geographical data is represented on a three-dimensional sphere (similar to a globe). In the S2 library points are represented as the S2 Index - a specific number which encodes internally a point on the surface of a unit sphere, unlike traditional (latitude, longitude) pairs. To get the S2 point index for a given point specified in the format (latitude, longitude) use the [geoToS2](#geotos2) function. Also, you can use the [s2ToGeo](#s2togeo) function for getting geographical coordinates corresponding to the specified S2 point index. ## geoToS2 -Returns [S2](#s2index) point index corresponding to the provided coordinates `(longitude, latitude)`. +Returns [S2](#s2_index) point index corresponding to the provided coordinates `(longitude, latitude)`. **Syntax** @@ -46,7 +48,7 @@ Result: ## s2ToGeo -Returns geo coordinates `(longitude, latitude)` corresponding to the provided [S2](#s2index) point index. +Returns geo coordinates `(longitude, latitude)` corresponding to the provided [S2](#s2_index) point index. **Syntax** @@ -82,7 +84,7 @@ Result: ## s2GetNeighbors -Returns S2 neighbor indexes corresponding to the provided [S2](#s2index). Each cell in the S2 system is a quadrilateral bounded by four geodesics. So, each cell has 4 neighbors. +Returns S2 neighbor indexes corresponding to the provided [S2](#s2_index). Each cell in the S2 system is a quadrilateral bounded by four geodesics. So, each cell has 4 neighbors. **Syntax** @@ -116,7 +118,7 @@ Result: ## s2CellsIntersect -Determines if the two provided [S2](#s2index) cells intersect or not. +Determines if the two provided [S2](#s2_index) cells intersect or not. **Syntax** diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 506114038f7..e431ed75465 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -45,13 +45,13 @@ SELECT halfMD5(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00') Calculates the MD4 from a string and returns the resulting set of bytes as FixedString(16). -## MD5 {#md5} +## MD5 Calculates the MD5 from a string and returns the resulting set of bytes as FixedString(16). If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 {#siphash64} +## sipHash64 Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index 5b6a3aef2c8..11a7749b33d 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -295,7 +295,7 @@ Same as `toIPv6`, but if the IPv6 address has an invalid format, it returns null ## toIPv6 Converts a string form of IPv6 address to [IPv6](../data-types/ipv6.md) type. If the IPv6 address has an invalid format, returns an empty value. -Similar to [IPv6StringToNum](#ipv6stringtonums) function, which converts IPv6 address to binary format. +Similar to [IPv6StringToNum](#ipv6stringtonum) function, which converts IPv6 address to binary format. If the input string contains a valid IPv4 address, then the IPv6 equivalent of the IPv4 address is returned. diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 5d73c9a83b3..7bff6a6cba5 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -5,10 +5,10 @@ sidebar_label: JSON --- There are two sets of functions to parse JSON: - - [`simpleJSON*` (`visitParam*`)](#simplejson--visitparam-functions) which is made for parsing a limited subset of JSON extremely fast. + - [`simpleJSON*` (`visitParam*`)](#simplejson-visitparam-functions) which is made for parsing a limited subset of JSON extremely fast. - [`JSONExtract*`](#jsonextract-functions) which is made for parsing ordinary JSON. -## simpleJSON / visitParam functions +## simpleJSON (visitParam) functions ClickHouse has special functions for working with simplified JSON. All these JSON functions are based on strong assumptions about what the JSON can be. They try to do as little as possible to get the job done as quickly as possible. diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5e63d9824b4..5eae8b7905e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -762,7 +762,7 @@ LIMIT 10 Given a size (number of bytes), this function returns a readable, rounded size with suffix (KB, MB, etc.) as string. -The opposite operations of this function are [parseReadableSize](#parseReadableSize), [parseReadableSizeOrZero](#parseReadableSizeOrZero), and [parseReadableSizeOrNull](#parseReadableSizeOrNull). +The opposite operations of this function are [parseReadableSize](#parseReadableSize), [parseReadableSizeOrZero](#parsereadablesizeorzero), and [parseReadableSizeOrNull](#parsereadablesizeornull). **Syntax** @@ -795,7 +795,7 @@ Result: Given a size (number of bytes), this function returns a readable, rounded size with suffix (KiB, MiB, etc.) as string. -The opposite operations of this function are [parseReadableSize](#parseReadableSize), [parseReadableSizeOrZero](#parseReadableSizeOrZero), and [parseReadableSizeOrNull](#parseReadableSizeOrNull). +The opposite operations of this function are [parseReadableSize](#parseReadableSize), [parseReadableSizeOrZero](#parsereadablesizeorzero), and [parseReadableSizeOrNull](#parsereadablesizeornull). **Syntax** @@ -921,12 +921,12 @@ SELECT └────────────────────┴────────────────────────────────────────────────┘ ``` -## parseReadableSize +## parseReadableSize {#parseReadableSize} Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. If the function is unable to parse the input value, it throws an exception. -The inverse operations of this function are [formatReadableSize](#formatReadableSize) and [formatReadableDecimalSize](#formatReadableDecimalSize). +The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatreadabledecimalsize). **Syntax** @@ -964,7 +964,7 @@ SELECT Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. If the function is unable to parse the input value, it returns `NULL`. -The inverse operations of this function are [formatReadableSize](#formatReadableSize) and [formatReadableDecimalSize](#formatReadableDecimalSize). +The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatReadableDecimalSize). **Syntax** @@ -1002,7 +1002,7 @@ SELECT Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. If the function is unable to parse the input value, it returns `0`. -The inverse operations of this function are [formatReadableSize](#formatReadableSize) and [formatReadableDecimalSize](#formatReadableDecimalSize). +The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatReadableDecimalSize). **Syntax** @@ -2711,7 +2711,7 @@ countDigits(x) - Number of digits. [UInt8](../data-types/int-uint.md#uint-ranges). :::note -For `Decimal` values takes into account their scales: calculates result over underlying integer type which is `(value * scale)`. For example: `countDigits(42) = 2`, `countDigits(42.000) = 5`, `countDigits(0.04200) = 4`. I.e. you may check decimal overflow for `Decimal64` with `countDecimal(x) > 18`. It's a slow variant of [isDecimalOverflow](#is-decimal-overflow). +For `Decimal` values takes into account their scales: calculates result over underlying integer type which is `(value * scale)`. For example: `countDigits(42) = 2`, `countDigits(42.000) = 5`, `countDigits(0.04200) = 4`. I.e. you may check decimal overflow for `Decimal64` with `countDecimal(x) > 18`. It's a slow variant of [isDecimalOverflow](#isdecimaloverflow). ::: **Example** @@ -2803,7 +2803,7 @@ currentProfiles() ## enabledProfiles -Returns settings profiles, assigned to the current user both explicitly and implicitly. Explicitly assigned profiles are the same as returned by the [currentProfiles](#current-profiles) function. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the `default_profile` section in the main server configuration file). +Returns settings profiles, assigned to the current user both explicitly and implicitly. Explicitly assigned profiles are the same as returned by the [currentProfiles](#currentprofiles) function. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the `default_profile` section in the main server configuration file). **Syntax** @@ -2916,11 +2916,11 @@ Result: └───────────────────────────┘ ``` -## queryID {#queryID} +## queryID Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. -In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see the example). +In contrast to [initialQueryID](#initialqueryid) function, `queryID` can return different results on different shards (see the example). **Syntax** @@ -2954,7 +2954,7 @@ Result: Returns the ID of the initial current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `initial_query_id`. -In contrast to [queryID](#query-id) function, `initialQueryID` returns the same results on different shards (see example). +In contrast to [queryID](#queryid) function, `initialQueryID` returns the same results on different shards (see example). **Syntax** @@ -3041,7 +3041,7 @@ shardCount() **See Also** -- [shardNum()](#shard-num) function example also contains `shardCount()` function call. +- [shardNum()](#shardnum) function example also contains `shardCount()` function call. ## getOSKernelVersion diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 6495a43fc85..c3a915ca195 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -200,7 +200,7 @@ Banker's rounding is a method of rounding fractional numbers When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. For example: 3.5 rounds up to 4, 2.5 rounds down to 2. It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). -The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. +The [round](#round) function performs the same rounding for floating point numbers. The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`. In other cases, the function rounds numbers to the nearest integer. diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 342ca2b9f03..c2d19f58422 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1994,7 +1994,7 @@ Result: ## stringJaccardIndexUTF8 -Like [stringJaccardIndex](#stringJaccardIndex) but for UTF8-encoded strings. +Like [stringJaccardIndex](#stringjaccardindex) but for UTF8-encoded strings. ## editDistance diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index d261cff3580..5353bbf9b27 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -262,7 +262,7 @@ Result: ## multiSearchAllPositionsUTF8 -Like [multiSearchAllPositions](#multiSearchAllPositions) but assumes `haystack` and the `needle` substrings are UTF-8 encoded strings. +Like [multiSearchAllPositions](#multisearchallpositions) but assumes `haystack` and the `needle` substrings are UTF-8 encoded strings. **Syntax** @@ -336,7 +336,7 @@ Result: Like [`position`](#position) but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings. -Functions [`multiSearchFirstPositionCaseInsensitive`](#multiSearchFirstPositionCaseInsensitive), [`multiSearchFirstPositionUTF8`](#multiSearchFirstPositionUTF8) and [`multiSearchFirstPositionCaseInsensitiveUTF8`](#multiSearchFirstPositionCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchFirstPositionCaseInsensitive`](#multisearchfirstpositioncaseinsensitive), [`multiSearchFirstPositionUTF8`](#multisearchfirstpositionutf8) and [`multiSearchFirstPositionCaseInsensitiveUTF8`](#multisearchfirstpositioncaseinsensitiveutf8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** @@ -370,7 +370,7 @@ Result: ## multiSearchFirstPositionCaseInsensitive -Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but ignores case. +Like [`multiSearchFirstPosition`](#multisearchfirstposition) but ignores case. **Syntax** @@ -404,7 +404,7 @@ Result: ## multiSearchFirstPositionUTF8 -Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings. +Like [`multiSearchFirstPosition`](#multisearchfirstposition) but assumes `haystack` and `needle` to be UTF-8 strings. **Syntax** @@ -440,7 +440,7 @@ Result: ## multiSearchFirstPositionCaseInsensitiveUTF8 -Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings and ignores case. +Like [`multiSearchFirstPosition`](#multisearchfirstposition) but assumes `haystack` and `needle` to be UTF-8 strings and ignores case. **Syntax** @@ -478,7 +478,7 @@ Result: Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. -Functions [`multiSearchFirstIndexCaseInsensitive`](#multiSearchFirstIndexCaseInsensitive), [`multiSearchFirstIndexUTF8`](#multiSearchFirstIndexUTF8) and [`multiSearchFirstIndexCaseInsensitiveUTF8`](#multiSearchFirstIndexCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchFirstIndexCaseInsensitive`](#multisearchfirstindexcaseinsensitive), [`multiSearchFirstIndexUTF8`](#multisearchfirstindexutf8) and [`multiSearchFirstIndexCaseInsensitiveUTF8`](#multisearchfirstindexcaseinsensitiveutf8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** @@ -615,7 +615,7 @@ Result: Returns 1, if at least one string needlei matches the string `haystack` and 0 otherwise. -Functions [`multiSearchAnyCaseInsensitive`](#multiSearchAnyCaseInsensitive), [`multiSearchAnyUTF8`](#multiSearchAnyUTF8) and []`multiSearchAnyCaseInsensitiveUTF8`](#multiSearchAnyCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchAnyCaseInsensitive`](#multisearchanycaseinsensitive), [`multiSearchAnyUTF8`](#multisearchanyutf8) and [`multiSearchAnyCaseInsensitiveUTF8`](#multisearchanycaseinsensitiveutf8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** @@ -719,7 +719,7 @@ Result: ## multiSearchAnyCaseInsensitiveUTF8 -Like [multiSearchAnyUTF8](#multiSearchAnyUTF8) but ignores case. +Like [multiSearchAnyUTF8](#multisearchanyutf8) but ignores case. *Syntax** @@ -880,7 +880,7 @@ extractAll(haystack, pattern) Matches all groups of the `haystack` string using the `pattern` regular expression. Returns an array of arrays, where the first array includes all fragments matching the first group, the second array - matching the second group, etc. -This function is slower than [extractAllGroupsVertical](#extractallgroups-vertical). +This function is slower than [extractAllGroupsVertical](#extractallgroupsvertical). **Syntax** @@ -952,7 +952,7 @@ Result: └────────────────────────────────────────────────────────────────────────────────────────┘ ``` -## like {#like} +## like Returns whether string `haystack` matches the LIKE expression `pattern`. @@ -1215,7 +1215,7 @@ Result: ## ngramSearchCaseInsensitive -Provides a case-insensitive variant of [ngramSearch](#ngramSearch). +Provides a case-insensitive variant of [ngramSearch](#ngramsearch). **Syntax** @@ -1630,7 +1630,7 @@ Result: ## hasSubsequenceCaseInsensitive -Like [hasSubsequence](#hasSubsequence) but searches case-insensitively. +Like [hasSubsequence](#hassubsequence) but searches case-insensitively. **Syntax** @@ -1700,7 +1700,7 @@ Result: ## hasSubsequenceCaseInsensitiveUTF8 -Like [hasSubsequenceUTF8](#hasSubsequenceUTF8) but searches case-insensitively. +Like [hasSubsequenceUTF8](#hassubsequenceutf8) but searches case-insensitively. **Syntax** diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 2ec51d43c59..86739ac0b8d 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -70,7 +70,7 @@ Integer value in the `Int8`, `Int16`, `Int32`, `Int64`, `Int128` or `Int256` dat Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. -The behavior of functions for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions. +The behavior of functions for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. **Example** @@ -169,7 +169,7 @@ Converts an input value to the [UInt](../data-types/int-uint.md) data type. This Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. -The behavior of functions for negative arguments and for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions. +The behavior of functions for negative arguments and for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. **Example** @@ -1730,7 +1730,7 @@ Result: └─────────────────────┘ ``` -## reinterpret(x, T) +## reinterpret Uses the same source in-memory bytes sequence for `x` value and reinterprets it to destination type. @@ -1766,9 +1766,9 @@ Result: └─────────────┴──────────────┴───────────────┘ ``` -## CAST(x, T) +## CAST -Converts an input value to the specified data type. Unlike the [reinterpret](#type_conversion_function-reinterpret) function, `CAST` tries to present the same value using the new data type. If the conversion can not be done then an exception is raised. +Converts an input value to the specified data type. Unlike the [reinterpret](#reinterpret) function, `CAST` tries to present the same value using the new data type. If the conversion can not be done then an exception is raised. Several syntax variants are supported. **Syntax** @@ -1875,7 +1875,7 @@ Result: Converts `x` to the `T` data type. -The difference from [cast(x, T)](#type_conversion_function-cast) is that `accurateCast` does not allow overflow of numeric types during cast if type value `x` does not fit the bounds of type `T`. For example, `accurateCast(-1, 'UInt8')` throws an exception. +The difference from [cast](#cast) is that `accurateCast` does not allow overflow of numeric types during cast if type value `x` does not fit the bounds of type `T`. For example, `accurateCast(-1, 'UInt8')` throws an exception. **Example** @@ -2061,7 +2061,7 @@ Result: └───────────────────────────┴──────────────────────────────┘ ``` -## parseDateTime {#type_conversion_functions-parseDateTime} +## parseDateTime Converts a [String](../data-types/string.md) to [DateTime](../data-types/datetime.md) according to a [MySQL format string](https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format). @@ -2102,15 +2102,15 @@ Alias: `TO_TIMESTAMP`. ## parseDateTimeOrZero -Same as for [parseDateTime](#type_conversion_functions-parseDateTime) except that it returns zero date when it encounters a date format that cannot be processed. +Same as for [parseDateTime](#parsedatetime) except that it returns zero date when it encounters a date format that cannot be processed. ## parseDateTimeOrNull -Same as for [parseDateTime](#type_conversion_functions-parseDateTime) except that it returns `NULL` when it encounters a date format that cannot be processed. +Same as for [parseDateTime](#parsedatetime) except that it returns `NULL` when it encounters a date format that cannot be processed. Alias: `str_to_date`. -## parseDateTimeInJodaSyntax {#type_conversion_functions-parseDateTimeInJodaSyntax} +## parseDateTimeInJodaSyntax Similar to [parseDateTime](#parsedatetime), except that the format string is in [Joda](https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) instead of MySQL syntax. @@ -2151,11 +2151,11 @@ SELECT parseDateTimeInJodaSyntax('2023-02-24 14:53:31', 'yyyy-MM-dd HH:mm:ss', ' ## parseDateTimeInJodaSyntaxOrZero -Same as for [parseDateTimeInJodaSyntax](#type_conversion_functions-parseDateTimeInJodaSyntax) except that it returns zero date when it encounters a date format that cannot be processed. +Same as for [parseDateTimeInJodaSyntax](#parsedatetimeinjodasyntax) except that it returns zero date when it encounters a date format that cannot be processed. ## parseDateTimeInJodaSyntaxOrNull -Same as for [parseDateTimeInJodaSyntax](#type_conversion_functions-parseDateTimeInJodaSyntax) except that it returns `NULL` when it encounters a date format that cannot be processed. +Same as for [parseDateTimeInJodaSyntax](#parsedatetimeinjodasyntax) except that it returns `NULL` when it encounters a date format that cannot be processed. ## parseDateTimeBestEffort ## parseDateTime32BestEffort @@ -2313,11 +2313,11 @@ Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it r ## parseDateTimeBestEffortUSOrNull -Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except that it returns `NULL` when it encounters a date format that cannot be processed. +Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortus) function except that it returns `NULL` when it encounters a date format that cannot be processed. ## parseDateTimeBestEffortUSOrZero -Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except that it returns zero date (`1970-01-01`) or zero date with time (`1970-01-01 00:00:00`) when it encounters a date format that cannot be processed. +Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortus) function except that it returns zero date (`1970-01-01`) or zero date with time (`1970-01-01 00:00:00`) when it encounters a date format that cannot be processed. ## parseDateTime64BestEffort @@ -2389,7 +2389,7 @@ Same as for [parseDateTime64BestEffort](#parsedatetime64besteffort), except that Converts input parameter to the [LowCardinality](../data-types/lowcardinality.md) version of same data type. -To convert data from the `LowCardinality` data type use the [CAST](#type_conversion_function-cast) function. For example, `CAST(x as String)`. +To convert data from the `LowCardinality` data type use the [CAST](#cast) function. For example, `CAST(x as String)`. **Syntax** diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index 0323ae728a9..5f15907d029 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -150,7 +150,7 @@ The function also works for [Arrays](array-functions.md#function-empty) and [Str **Example** -To generate the UUID value, ClickHouse provides the [generateUUIDv4](#uuid-function-generate) function. +To generate the UUID value, ClickHouse provides the [generateUUIDv4](#generateuuidv4) function. Query: @@ -190,7 +190,7 @@ The function also works for [Arrays](array-functions.md#function-notempty) or [S **Example** -To generate the UUID value, ClickHouse provides the [generateUUIDv4](#uuid-function-generate) function. +To generate the UUID value, ClickHouse provides the [generateUUIDv4](#generateuuidv4) function. Query: diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 0257d21b30f..5c83b2363e0 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -255,7 +255,7 @@ where `M` is between `1` and `3` depending on which replica the local query is e These settings affect every MergeTree-family table in the query and have the same effect as applying `SAMPLE 1/3 OFFSET (M-1)/3` on each table. -Therefore adding the [max_parallel_replicas](#settings-max_parallel_replicas) setting will only produce correct results if both tables have the same replication scheme and are sampled by UserID or a subkey of it. In particular, if `local_table_2` does not have a sampling key, incorrect results will be produced. The same rule applies to `JOIN`. +Therefore adding the [max_parallel_replicas](#distributed-subqueries-and-max_parallel_replicas) setting will only produce correct results if both tables have the same replication scheme and are sampled by UserID or a subkey of it. In particular, if `local_table_2` does not have a sampling key, incorrect results will be produced. The same rule applies to `JOIN`. One workaround if `local_table_2` does not meet the requirements, is to use `GLOBAL IN` or `GLOBAL JOIN`. diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index a23710b12bd..aa6f132e08e 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -108,7 +108,7 @@ ALTER TABLE visits RENAME COLUMN webBrowser TO browser CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name ``` -Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](partition.md/#how-to-set-partition-expression). +Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](../alter/partition.md/#how-to-set-partition-expression). If the `IF EXISTS` clause is specified, the query won’t return an error if the column does not exist. @@ -173,7 +173,7 @@ ALTER TABLE visits MODIFY COLUMN browser Array(String) Changing the column type is the only complex action – it changes the contents of files with data. For large tables, this may take a long time. -The query also can change the order of the columns using `FIRST | AFTER` clause, see [ADD COLUMN](#alter_add-column) description, but column type is mandatory in this case. +The query also can change the order of the columns using `FIRST | AFTER` clause, see [ADD COLUMN](#add-column) description, but column type is mandatory in this case. Example: diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 0ed1e523669..778816f8934 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -31,7 +31,7 @@ The following operations with [partitions](/docs/en/engines/table-engines/merget ALTER TABLE table_name [ON CLUSTER cluster] DETACH PARTITION|PART partition_expr ``` -Moves all data for the specified partition to the `detached` directory. The server forgets about the detached data partition as if it does not exist. The server will not know about this data until you make the [ATTACH](#alter_attach-partition) query. +Moves all data for the specified partition to the `detached` directory. The server forgets about the detached data partition as if it does not exist. The server will not know about this data until you make the [ATTACH](#attach-partitionpart) query. Example: @@ -252,7 +252,7 @@ Downloads a partition from another server. This query only works for the replica The query does the following: 1. Downloads the partition|part from the specified shard. In ‘path-in-zookeeper’ you must specify a path to the shard in ZooKeeper. -2. Then the query puts the downloaded data to the `detached` directory of the `table_name` table. Use the [ATTACH PARTITION\|PART](#alter_attach-partition) query to add the data to the table. +2. Then the query puts the downloaded data to the `detached` directory of the `table_name` table. Use the [ATTACH PARTITION\|PART](#attach-partitionpart) query to add the data to the table. For example: @@ -353,7 +353,7 @@ You can specify the partition expression in `ALTER ... PARTITION` queries in dif - Using the keyword `ALL`. It can be used only with DROP/DETACH/ATTACH. For example, `ALTER TABLE visits ATTACH PARTITION ALL`. - As a tuple of expressions or constants that matches (in types) the table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- In the [ALTER ATTACH PART](#attach-partitionpart) and [DROP DETACHED PART](#drop-detached-partitionpart) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. Usage of quotes when specifying the partition depends on the type of partition expression. For example, for the `String` type, you have to specify its name in quotes (`'`). For the `Date` and `Int*` types no quotes are needed. diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 1bdf22b35b0..1fabb6d8cc7 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -6,7 +6,7 @@ sidebar_label: VIEW # CREATE VIEW -Creates a new view. Views can be [normal](#normal-view), [materialized](#materialized-view), [live](#live-view-experimental), and [window](#window-view-experimental) (live view and window view are experimental features). +Creates a new view. Views can be [normal](#normal-view), [materialized](#materialized-view), [live](#live-view-deprecated), and [window](#window-view-experimental) (live view and window view are experimental features). ## Normal View diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 2850ce71781..43fa344a16d 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -33,7 +33,7 @@ GRANT [ON CLUSTER cluster_name] role [,...] TO {user | another_role | CURRENT_US - `role` — ClickHouse user role. - `user` — ClickHouse user account. -The `WITH ADMIN OPTION` clause grants [ADMIN OPTION](#admin-option-privilege) privilege to `user` or `role`. +The `WITH ADMIN OPTION` clause grants [ADMIN OPTION](#admin-option) privilege to `user` or `role`. The `WITH REPLACE OPTION` clause replace old roles by new role for the `user` or `role`, if is not specified it appends roles. ## Grant Current Grants Syntax @@ -201,7 +201,7 @@ Hierarchy of privileges: - `HDFS` - `S3` - [dictGet](#dictget) -- [displaySecretsInShowAndSelect](#display-secrets) +- [displaySecretsInShowAndSelect](#displaysecretsinshowandselect) - [NAMED COLLECTION ADMIN](#named-collection-admin) - `CREATE NAMED COLLECTION` - `DROP NAMED COLLECTION` @@ -498,7 +498,7 @@ Privilege level: `DICTIONARY`. - `GRANT dictGet ON mydictionary TO john` -### displaySecretsInShowAndSelect {#display-secrets} +### displaySecretsInShowAndSelect Allows a user to view secrets in `SHOW` and `SELECT` queries if both [`display_secrets_in_show_and_select` server setting](../../operations/server-configuration-parameters/settings#display_secrets_in_show_and_select) diff --git a/docs/en/sql-reference/statements/select/sample.md b/docs/en/sql-reference/statements/select/sample.md index 137f86cc8b9..78e05b19bd1 100644 --- a/docs/en/sql-reference/statements/select/sample.md +++ b/docs/en/sql-reference/statements/select/sample.md @@ -27,14 +27,14 @@ The features of data sampling are listed below: For the `SAMPLE` clause the following syntax is supported: -| SAMPLE Clause Syntax | Description | -|----------------------|------------------------------| -| `SAMPLE k` | Here `k` is the number from 0 to 1. The query is executed on `k` fraction of data. For example, `SAMPLE 0.1` runs the query on 10% of data. [Read more](#select-sample-k) | -| `SAMPLE n` | Here `n` is a sufficiently large integer. The query is executed on a sample of at least `n` rows (but not significantly more than this). For example, `SAMPLE 10000000` runs the query on a minimum of 10,000,000 rows. [Read more](#select-sample-n) | -| `SAMPLE k OFFSET m` | Here `k` and `m` are the numbers from 0 to 1. The query is executed on a sample of `k` fraction of the data. The data used for the sample is offset by `m` fraction. [Read more](#select-sample-offset) | +| SAMPLE Clause Syntax | Description | +|----------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `SAMPLE k` | Here `k` is the number from 0 to 1. The query is executed on `k` fraction of data. For example, `SAMPLE 0.1` runs the query on 10% of data. [Read more](#sample-k) | +| `SAMPLE n` | Here `n` is a sufficiently large integer. The query is executed on a sample of at least `n` rows (but not significantly more than this). For example, `SAMPLE 10000000` runs the query on a minimum of 10,000,000 rows. [Read more](#sample-n) | +| `SAMPLE k OFFSET m` | Here `k` and `m` are the numbers from 0 to 1. The query is executed on a sample of `k` fraction of the data. The data used for the sample is offset by `m` fraction. [Read more](#sample-k-offset-m) | -## SAMPLE K {#select-sample-k} +## SAMPLE K Here `k` is the number from 0 to 1 (both fractional and decimal notations are supported). For example, `SAMPLE 1/2` or `SAMPLE 0.5`. @@ -54,7 +54,7 @@ ORDER BY PageViews DESC LIMIT 1000 In this example, the query is executed on a sample from 0.1 (10%) of data. Values of aggregate functions are not corrected automatically, so to get an approximate result, the value `count()` is manually multiplied by 10. -## SAMPLE N {#select-sample-n} +## SAMPLE N Here `n` is a sufficiently large integer. For example, `SAMPLE 10000000`. @@ -90,7 +90,7 @@ FROM visits SAMPLE 10000000 ``` -## SAMPLE K OFFSET M {#select-sample-offset} +## SAMPLE K OFFSET M Here `k` and `m` are numbers from 0 to 1. Examples are shown below. diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 7efbff1b42b..e6d3439d2b9 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -174,7 +174,7 @@ Aborts ClickHouse process (like `kill -9 {$ pid_clickhouse-server}`) ## Managing Distributed Tables -ClickHouse can manage [distributed](../../engines/table-engines/special/distributed.md) tables. When a user inserts data into these tables, ClickHouse first creates a queue of the data that should be sent to cluster nodes, then asynchronously sends it. You can manage queue processing with the [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), and [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) queries. You can also synchronously insert distributed data with the [distributed_foreground_insert](../../operations/settings/settings.md#distributed_foreground_insert) setting. +ClickHouse can manage [distributed](../../engines/table-engines/special/distributed.md) tables. When a user inserts data into these tables, ClickHouse first creates a queue of the data that should be sent to cluster nodes, then asynchronously sends it. You can manage queue processing with the [STOP DISTRIBUTED SENDS](#stop-distributed-sends), [FLUSH DISTRIBUTED](#flush-distributed), and [START DISTRIBUTED SENDS](#start-distributed-sends) queries. You can also synchronously insert distributed data with the [distributed_foreground_insert](../../operations/settings/settings.md#distributed_foreground_insert) setting. ### STOP DISTRIBUTED SENDS diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md index fc0286e76ad..6a4afb63db8 100644 --- a/docs/en/sql-reference/syntax.md +++ b/docs/en/sql-reference/syntax.md @@ -54,11 +54,11 @@ Identifiers are: - Cluster, database, table, partition, and column names. - Functions. - Data types. -- [Expression aliases](#expression_aliases). +- [Expression aliases](#expression-aliases). Identifiers can be quoted or non-quoted. The latter is preferred. -Non-quoted identifiers must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$` and can not be equal to [keywords](#syntax-keywords). Examples: `x`, `_1`, `X_y__Z123_`. +Non-quoted identifiers must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$` and can not be equal to [keywords](#keywords). Examples: `x`, `_1`, `X_y__Z123_`. If you want to use identifiers the same as keywords or you want to use other symbols in identifiers, quote it using double quotes or backticks, for example, `"id"`, `` `id` ``. diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 4fec772c373..3a3162dad9a 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -18,7 +18,7 @@ file([path_to_archive ::] path [,format] [,structure] [,compression]) **Parameters** -- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports in read-only mode the following [globs](#globs_in_path): `*`, `?`, `{abc,def}` (with `'abc'` and `'def'` being strings) and `{N..M}` (with `N` and `M` being numbers). +- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports in read-only mode the following [globs](#globs-in-path): `*`, `?`, `{abc,def}` (with `'abc'` and `'def'` being strings) and `{N..M}` (with `N` and `M` being numbers). - `path_to_archive` - The relative path to a zip/tar/7z archive. Supports the same globs as `path`. - `format` — The [format](/docs/en/interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format: `'column1_name column1_type, column2_name column2_type, ...'`. @@ -128,7 +128,7 @@ Reading data from `table.csv`, located in `archive1.zip` or/and `archive2.zip`: SELECT * FROM file('user_files/archives/archive{1..2}.zip :: table.csv'); ``` -## Globs in path {#globs_in_path} +## Globs in path Paths may use globbing. Files must match the whole path pattern, not only the suffix or prefix. diff --git a/docs/en/sql-reference/table-functions/fileCluster.md b/docs/en/sql-reference/table-functions/fileCluster.md index 4677d2883a7..3060e6c151d 100644 --- a/docs/en/sql-reference/table-functions/fileCluster.md +++ b/docs/en/sql-reference/table-functions/fileCluster.md @@ -74,7 +74,7 @@ SELECT * FROM fileCluster('my_cluster', 'file{1,2}.csv', 'CSV', 'i UInt32, s Str ``` -## Globs in Path {#globs_in_path} +## Globs in Path All patterns supported by [File](../../sql-reference/table-functions/file.md#globs-in-path) table function are supported by FileCluster. From a32df984564ca7b4dfc45115b876632de70526ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Jun 2024 14:18:51 +0200 Subject: [PATCH 571/856] Add a test --- tests/queries/0_stateless/03170_esc_crash.reference | 4 ++++ tests/queries/0_stateless/03170_esc_crash.sh | 9 +++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03170_esc_crash.reference create mode 100755 tests/queries/0_stateless/03170_esc_crash.sh diff --git a/tests/queries/0_stateless/03170_esc_crash.reference b/tests/queries/0_stateless/03170_esc_crash.reference new file mode 100644 index 00000000000..acd7c60768b --- /dev/null +++ b/tests/queries/0_stateless/03170_esc_crash.reference @@ -0,0 +1,4 @@ +1 2 3 +4 5 6 +7 8 9 +0 0 0 diff --git a/tests/queries/0_stateless/03170_esc_crash.sh b/tests/queries/0_stateless/03170_esc_crash.sh new file mode 100755 index 00000000000..fa6870c4cf2 --- /dev/null +++ b/tests/queries/0_stateless/03170_esc_crash.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Previous versions crashed in attempt to use this authentication method (regardless of whether it was able to authenticate): +AWS_CONTAINER_CREDENTIALS_FULL_URI=http://localhost:1338/latest/meta-data/container/security-credentials $CLICKHOUSE_LOCAL -q "select * from s3('http://localhost:11111/test/a.tsv')" From af4541755a43fcacf1a1a58edcf6b07bd5fcedec Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 12 Jun 2024 14:27:13 +0200 Subject: [PATCH 572/856] fix black --- .../test_manipulate_statistics/test.py | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 0ce90731e8d..9485b611c01 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -156,20 +156,29 @@ def test_replicated_table_ddl(started_cluster): == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" ) - node2.query("insert into test_stat values(1,2,3), (2,3,4)"); + node2.query("insert into test_stat values(1,2,3), (2,3,4)") # check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True) - node1.query("ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"}) - assert (node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n") + node1.query( + "ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"} + ) + assert node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n" check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "c", False) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "d", True) - node1.query("ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"}) - node1.query("ALTER TABLE test_stat ADD STATISTICS b type tdigest", settings={"alter_sync": "2"}) + node1.query( + "ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"} + ) + node1.query( + "ALTER TABLE test_stat ADD STATISTICS b type tdigest", + settings={"alter_sync": "2"}, + ) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "b", False) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "d", False) - node1.query("ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"}) + node1.query( + "ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"} + ) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True) From dcd3e9d1511572b71af3b9149e3ecf713fb35a8a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jun 2024 14:06:26 +0200 Subject: [PATCH 573/856] Move changelog script to tests/ci --- tests/ci/changelog.py | 427 ++++++++++++++++++++++++++++++ utils/changelog/changelog.py | 428 +------------------------------ utils/changelog/git_helper.py | 1 - utils/changelog/github_helper.py | 1 - 4 files changed, 435 insertions(+), 422 deletions(-) create mode 100755 tests/ci/changelog.py delete mode 120000 utils/changelog/git_helper.py delete mode 120000 utils/changelog/github_helper.py diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py new file mode 100755 index 00000000000..b1a43b1520f --- /dev/null +++ b/tests/ci/changelog.py @@ -0,0 +1,427 @@ +#!/usr/bin/env python3 +# In our CI this script runs in style-test containers + +import argparse +import logging +import os +import os.path as p +import re +from datetime import date, timedelta +from subprocess import DEVNULL +from typing import Dict, List, Optional, TextIO + +from github.GithubException import RateLimitExceededException, UnknownObjectException +from github.NamedUser import NamedUser +from thefuzz.fuzz import ratio # type: ignore + +from git_helper import git_runner as runner +from git_helper import is_shallow +from github_helper import GitHub, PullRequest, PullRequests, Repository + +# This array gives the preferred category order, and is also used to +# normalize category names. +# Categories are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there +# updated accordingly +categories_preferred_order = ( + "Backward Incompatible Change", + "New Feature", + "Performance Improvement", + "Improvement", + "Critical Bug Fix", + "Bug Fix", + "Build/Testing/Packaging Improvement", + "Other", +) + +FROM_REF = "" +TO_REF = "" +SHA_IN_CHANGELOG = [] # type: List[str] +gh = GitHub(create_cache_dir=False) +CACHE_PATH = p.join(p.dirname(p.realpath(__file__)), "gh_cache") + + +class Description: + def __init__( + self, number: int, user: NamedUser, html_url: str, entry: str, category: str + ): + self.number = number + self.html_url = html_url + self.user = gh.get_user_cached(user._rawData["login"]) # type: ignore + self.entry = entry + self.category = category + + @property + def formatted_entry(self) -> str: + # Substitute issue links. + # 1) issue number w/o markdown link + entry = re.sub( + r"([^[])#([0-9]{4,})", + r"\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)", + self.entry, + ) + # 2) issue URL w/o markdown link + # including #issuecomment-1 or #event-12 + entry = re.sub( + r"([^(])(https://github.com/ClickHouse/ClickHouse/issues/([0-9]{4,})[-#a-z0-9]*)", + r"\1[#\3](\2)", + entry, + ) + # It's possible that we face a secondary rate limit. + # In this case we should sleep until we get it + while True: + try: + user_name = self.user.name if self.user.name else self.user.login + break + except UnknownObjectException: + user_name = self.user.login + break + except RateLimitExceededException: + gh.sleep_on_rate_limit() + return ( + f"* {entry} [#{self.number}]({self.html_url}) " + f"([{user_name}]({self.user.html_url}))." + ) + + # Sort PR descriptions by numbers + def __eq__(self, other) -> bool: + if not isinstance(self, type(other)): + return NotImplemented + return self.number == other.number + + def __lt__(self, other: "Description") -> bool: + return self.number < other.number + + +def get_descriptions(prs: PullRequests) -> Dict[str, List[Description]]: + descriptions = {} # type: Dict[str, List[Description]] + repos = {} # type: Dict[str, Repository] + for pr in prs: + # See https://github.com/PyGithub/PyGithub/issues/2202, + # obj._rawData doesn't spend additional API requests + # We'll save some requests + # pylint: disable=protected-access + repo_name = pr._rawData["base"]["repo"]["full_name"] + # pylint: enable=protected-access + if repo_name not in repos: + repos[repo_name] = pr.base.repo + in_changelog = False + merge_commit = pr.merge_commit_sha + if merge_commit is None: + logging.warning("PR %s does not have merge-commit, skipping", pr.number) + continue + + in_changelog = merge_commit in SHA_IN_CHANGELOG + if in_changelog: + desc = generate_description(pr, repos[repo_name]) + if desc: + if desc.category not in descriptions: + descriptions[desc.category] = [] + descriptions[desc.category].append(desc) + + for descs in descriptions.values(): + descs.sort() + + return descriptions + + +def parse_args() -> argparse.Namespace: + parser = argparse.ArgumentParser( + formatter_class=argparse.ArgumentDefaultsHelpFormatter, + description="Generate a changelog in Markdown format between given tags. " + "It fetches all tags and unshallow the git repository automatically", + ) + parser.add_argument( + "-v", + "--verbose", + action="count", + default=0, + help="set the script verbosity, could be used multiple", + ) + parser.add_argument( + "--debug-helpers", + action="store_true", + help="add debug logging for git_helper and github_helper", + ) + parser.add_argument( + "--output", + type=argparse.FileType("w"), + default="-", + help="output file for changelog", + ) + parser.add_argument( + "--repo", + default="ClickHouse/ClickHouse", + help="a repository to query for pull-requests from GitHub", + ) + parser.add_argument( + "--jobs", + type=int, + default=10, + help="number of jobs to get pull-requests info from GitHub API", + ) + parser.add_argument( + "--gh-user-or-token", + help="user name or GH token to authenticate", + ) + parser.add_argument( + "--gh-password", + help="a password that should be used when user is given", + ) + parser.add_argument( + "--with-testing-tags", + action="store_true", + help="by default '*-testing' tags are ignored, this argument enables them too", + ) + parser.add_argument( + "--from", + dest="from_ref", + help="git ref for a starting point of changelog, by default is calculated " + "automatically to match a previous tag in history", + ) + parser.add_argument( + "to_ref", + metavar="TO_REF", + help="git ref for the changelog end", + ) + args = parser.parse_args() + return args + + +# This function mirrors the PR description checks in ClickhousePullRequestTrigger. +# Returns None if the PR should not be mentioned in changelog. +def generate_description(item: PullRequest, repo: Repository) -> Optional[Description]: + backport_number = item.number + if item.head.ref.startswith("backport/"): + branch_parts = item.head.ref.split("/") + if len(branch_parts) == 3: + 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) + else: + logging.warning( + "The branch %s doesn't match backport template, using PR %s as is", + item.head.ref, + item.number, + ) + description = item.body + # Don't skip empty lines because they delimit parts of description + lines = [x.strip() for x in (description.split("\n") if description else [])] + lines = [re.sub(r"\s+", " ", ln) for ln in lines] + + category = "" + entry = "" + + if lines: + i = 0 + while i < len(lines): + if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): + i += 1 + if i >= len(lines): + break + # Can have one empty line between header and the category itself. + # Filter it out. + if not lines[i]: + i += 1 + if i >= len(lines): + break + category = re.sub(r"^[-*\s]*", "", lines[i]) + i += 1 + elif re.match( + r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] + ): + i += 1 + # Can have one empty line between header and the entry itself. + # Filter it out. + if i < len(lines) and not lines[i]: + i += 1 + # All following lines until empty one are the changelog entry. + entry_lines = [] + while i < len(lines) and lines[i]: + entry_lines.append(lines[i]) + i += 1 + entry = " ".join(entry_lines) + else: + i += 1 + + # Remove excessive bullets from the entry. + if re.match(r"^[\-\*] ", entry): + entry = entry[2:] + + # Better style. + if re.match(r"^[a-z]", entry): + entry = entry.capitalize() + + if not category: + # Shouldn't happen, because description check in CI should catch such PRs. + # Fall through, so that it shows up in output and the user can fix it. + category = "NO CL CATEGORY" + + # Filter out the PR categories that are not for changelog. + if re.match( + r"(?i)((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", + category, + ): + category = "NOT FOR CHANGELOG / INSIGNIFICANT" + return Description(item.number, item.user, item.html_url, item.title, category) + + # Normalize bug fixes + if re.match( + r"(?i)bug\Wfix", + category, + ): + category = "Bug Fix (user-visible misbehavior in an official stable release)" + + # Filter out documentations changelog + if re.match( + r"(?i)doc", + category, + ): + return None + + if backport_number != item.number: + entry = f"Backported in #{backport_number}: {entry}" + + if not entry: + # Shouldn't happen, because description check in CI should catch such PRs. + category = "NO CL ENTRY" + entry = "NO CL ENTRY: '" + item.title + "'" + + entry = entry.strip() + if entry[-1] != ".": + entry += "." + + for c in categories_preferred_order: + if ratio(category.lower(), c.lower()) >= 90: + category = c + break + + return Description(item.number, item.user, item.html_url, entry, category) + + +def write_changelog( + fd: TextIO, descriptions: Dict[str, List[Description]], year: int +) -> None: + to_commit = runner(f"git rev-parse {TO_REF}^{{}}")[:11] + from_commit = runner(f"git rev-parse {FROM_REF}^{{}}")[:11] + fd.write( + f"---\nsidebar_position: 1\nsidebar_label: {year}\n---\n\n" + f"# {year} Changelog\n\n" + f"### ClickHouse release {TO_REF} ({to_commit}) FIXME " + f"as compared to {FROM_REF} ({from_commit})\n\n" + ) + + seen_categories = [] # type: List[str] + for category in categories_preferred_order: + if category in descriptions: + seen_categories.append(category) + fd.write(f"#### {category}\n") + for desc in descriptions[category]: + fd.write(f"{desc.formatted_entry}\n") + + fd.write("\n") + + for category in sorted(descriptions): + if category not in seen_categories: + fd.write(f"#### {category}\n\n") + for desc in descriptions[category]: + fd.write(f"{desc.formatted_entry}\n") + + fd.write("\n") + + +def check_refs(from_ref: Optional[str], to_ref: str, with_testing_tags: bool): + global FROM_REF, TO_REF + TO_REF = to_ref + + # Check TO_REF + runner.run(f"git rev-parse {TO_REF}") + + # Check from_ref + if from_ref is None: + # Get all tags pointing to TO_REF + tags = runner.run(f"git tag --points-at '{TO_REF}^{{}}'").split("\n") + logging.info("All tags pointing to %s:\n%s", TO_REF, tags) + if not with_testing_tags: + tags.append("*-testing") + exclude = " ".join([f"--exclude='{tag}'" for tag in tags]) + cmd = f"git describe --abbrev=0 --tags {exclude} '{TO_REF}'" + FROM_REF = runner.run(cmd) + else: + runner.run(f"git rev-parse {FROM_REF}") + FROM_REF = from_ref + + +def set_sha_in_changelog(): + global SHA_IN_CHANGELOG + SHA_IN_CHANGELOG = runner.run( + f"git log --format=format:%H {FROM_REF}..{TO_REF}" + ).split("\n") + + +def get_year(prs: PullRequests) -> int: + if not prs: + return date.today().year + return max(pr.created_at.year for pr in prs) + + +def main(): + log_levels = [logging.WARN, logging.INFO, logging.DEBUG] + args = parse_args() + logging.basicConfig( + format="%(asctime)s %(levelname)-8s [%(filename)s:%(lineno)d]:\n%(message)s", + level=log_levels[min(args.verbose, 2)], + ) + if args.debug_helpers: + logging.getLogger("github_helper").setLevel(logging.DEBUG) + logging.getLogger("git_helper").setLevel(logging.DEBUG) + # Create a cache directory + if not p.isdir(CACHE_PATH): + os.mkdir(CACHE_PATH, 0o700) + + # Get the full repo + if is_shallow(): + logging.info("Unshallow repository") + runner.run("git fetch --unshallow", stderr=DEVNULL) + logging.info("Fetching all tags") + runner.run("git fetch --tags", stderr=DEVNULL) + + check_refs(args.from_ref, args.to_ref, args.with_testing_tags) + set_sha_in_changelog() + + logging.info("Using %s..%s as changelog interval", FROM_REF, TO_REF) + + # use merge-base commit as a starting point, if used ref in another branch + base_commit = runner.run(f"git merge-base '{FROM_REF}^{{}}' '{TO_REF}^{{}}'") + # Get starting and ending dates for gathering PRs + # Add one day after and before to mitigate TZ possible issues + # `tag^{}` format gives commit ref when we have annotated tags + # format %cs gives a committer date, works better for cherry-picked commits + from_date = runner.run(f"git log -1 --format=format:%cs '{base_commit}'") + to_date = runner.run(f"git log -1 --format=format:%cs '{TO_REF}^{{}}'") + merged = ( + date.fromisoformat(from_date) - timedelta(1), + date.fromisoformat(to_date) + timedelta(1), + ) + + # Get all PRs for the given time frame + global gh + gh = GitHub( + args.gh_user_or_token, + args.gh_password, + create_cache_dir=False, + per_page=100, + pool_size=args.jobs, + ) + gh.cache_path = CACHE_PATH + query = f"type:pr repo:{args.repo} is:merged" + prs = gh.get_pulls_from_search(query=query, merged=merged, sort="created") + + descriptions = get_descriptions(prs) + changelog_year = get_year(prs) + + write_changelog(args.output, descriptions, changelog_year) + + +if __name__ == "__main__": + main() diff --git a/utils/changelog/changelog.py b/utils/changelog/changelog.py index 314461a6b3a..b79e4139bcc 100755 --- a/utils/changelog/changelog.py +++ b/utils/changelog/changelog.py @@ -1,427 +1,15 @@ #!/usr/bin/env python3 # In our CI this script runs in style-test containers -import argparse -import logging -import os -import os.path as p -import re -from datetime import date, timedelta -from subprocess import DEVNULL, CalledProcessError -from typing import Dict, List, Optional, TextIO +# The main script is moved to tests/ci/changelog.py +# It depends on the ci scripts too hard to keep it here +# Here's only a wrapper around it for the people who used to it -from github.GithubException import RateLimitExceededException, UnknownObjectException -from github.NamedUser import NamedUser -from thefuzz.fuzz import ratio # type: ignore - -from git_helper import git_runner as runner -from git_helper import is_shallow -from github_helper import GitHub, PullRequest, PullRequests, Repository - -# This array gives the preferred category order, and is also used to -# normalize category names. -# Categories are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there -# updated accordingly -categories_preferred_order = ( - "Backward Incompatible Change", - "New Feature", - "Performance Improvement", - "Improvement", - "Critical Bug Fix", - "Bug Fix", - "Build/Testing/Packaging Improvement", - "Other", -) - -FROM_REF = "" -TO_REF = "" -SHA_IN_CHANGELOG = [] # type: List[str] -gh = GitHub(create_cache_dir=False) -CACHE_PATH = p.join(p.dirname(p.realpath(__file__)), "gh_cache") - - -class Description: - def __init__( - self, number: int, user: NamedUser, html_url: str, entry: str, category: str - ): - self.number = number - self.html_url = html_url - self.user = gh.get_user_cached(user._rawData["login"]) # type: ignore - self.entry = entry - self.category = category - - @property - def formatted_entry(self) -> str: - # Substitute issue links. - # 1) issue number w/o markdown link - entry = re.sub( - r"([^[])#([0-9]{4,})", - r"\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)", - self.entry, - ) - # 2) issue URL w/o markdown link - # including #issuecomment-1 or #event-12 - entry = re.sub( - r"([^(])(https://github.com/ClickHouse/ClickHouse/issues/([0-9]{4,})[-#a-z0-9]*)", - r"\1[#\3](\2)", - entry, - ) - # It's possible that we face a secondary rate limit. - # In this case we should sleep until we get it - while True: - try: - user_name = self.user.name if self.user.name else self.user.login - break - except UnknownObjectException: - user_name = self.user.login - break - except RateLimitExceededException: - gh.sleep_on_rate_limit() - return ( - f"* {entry} [#{self.number}]({self.html_url}) " - f"([{user_name}]({self.user.html_url}))." - ) - - # Sort PR descriptions by numbers - def __eq__(self, other) -> bool: - if not isinstance(self, type(other)): - return NotImplemented - return self.number == other.number - - def __lt__(self, other: "Description") -> bool: - return self.number < other.number - - -def get_descriptions(prs: PullRequests) -> Dict[str, List[Description]]: - descriptions = {} # type: Dict[str, List[Description]] - repos = {} # type: Dict[str, Repository] - for pr in prs: - # See https://github.com/PyGithub/PyGithub/issues/2202, - # obj._rawData doesn't spend additional API requests - # We'll save some requests - # pylint: disable=protected-access - repo_name = pr._rawData["base"]["repo"]["full_name"] - # pylint: enable=protected-access - if repo_name not in repos: - repos[repo_name] = pr.base.repo - in_changelog = False - merge_commit = pr.merge_commit_sha - if merge_commit is None: - logging.warning("PR %s does not have merge-commit, skipping", pr.number) - continue - - in_changelog = merge_commit in SHA_IN_CHANGELOG - if in_changelog: - desc = generate_description(pr, repos[repo_name]) - if desc: - if desc.category not in descriptions: - descriptions[desc.category] = [] - descriptions[desc.category].append(desc) - - for descs in descriptions.values(): - descs.sort() - - return descriptions - - -def parse_args() -> argparse.Namespace: - parser = argparse.ArgumentParser( - formatter_class=argparse.ArgumentDefaultsHelpFormatter, - description="Generate a changelog in Markdown format between given tags. " - "It fetches all tags and unshallow the git repository automatically", - ) - parser.add_argument( - "-v", - "--verbose", - action="count", - default=0, - help="set the script verbosity, could be used multiple", - ) - parser.add_argument( - "--debug-helpers", - action="store_true", - help="add debug logging for git_helper and github_helper", - ) - parser.add_argument( - "--output", - type=argparse.FileType("w"), - default="-", - help="output file for changelog", - ) - parser.add_argument( - "--repo", - default="ClickHouse/ClickHouse", - help="a repository to query for pull-requests from GitHub", - ) - parser.add_argument( - "--jobs", - type=int, - default=10, - help="number of jobs to get pull-requests info from GitHub API", - ) - parser.add_argument( - "--gh-user-or-token", - help="user name or GH token to authenticate", - ) - parser.add_argument( - "--gh-password", - help="a password that should be used when user is given", - ) - parser.add_argument( - "--with-testing-tags", - action="store_true", - help="by default '*-testing' tags are ignored, this argument enables them too", - ) - parser.add_argument( - "--from", - dest="from_ref", - help="git ref for a starting point of changelog, by default is calculated " - "automatically to match a previous tag in history", - ) - parser.add_argument( - "to_ref", - metavar="TO_REF", - help="git ref for the changelog end", - ) - args = parser.parse_args() - return args - - -# This function mirrors the PR description checks in ClickhousePullRequestTrigger. -# Returns None if the PR should not be mentioned in changelog. -def generate_description(item: PullRequest, repo: Repository) -> Optional[Description]: - backport_number = item.number - if item.head.ref.startswith("backport/"): - branch_parts = item.head.ref.split("/") - if len(branch_parts) == 3: - 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) - else: - logging.warning( - "The branch %s doesn't match backport template, using PR %s as is", - item.head.ref, - item.number, - ) - description = item.body - # Don't skip empty lines because they delimit parts of description - lines = [x.strip() for x in (description.split("\n") if description else [])] - lines = [re.sub(r"\s+", " ", ln) for ln in lines] - - category = "" - entry = "" - - if lines: - i = 0 - while i < len(lines): - if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): - i += 1 - if i >= len(lines): - break - # Can have one empty line between header and the category itself. - # Filter it out. - if not lines[i]: - i += 1 - if i >= len(lines): - break - category = re.sub(r"^[-*\s]*", "", lines[i]) - i += 1 - elif re.match( - r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] - ): - i += 1 - # Can have one empty line between header and the entry itself. - # Filter it out. - if i < len(lines) and not lines[i]: - i += 1 - # All following lines until empty one are the changelog entry. - entry_lines = [] - while i < len(lines) and lines[i]: - entry_lines.append(lines[i]) - i += 1 - entry = " ".join(entry_lines) - else: - i += 1 - - # Remove excessive bullets from the entry. - if re.match(r"^[\-\*] ", entry): - entry = entry[2:] - - # Better style. - if re.match(r"^[a-z]", entry): - entry = entry.capitalize() - - if not category: - # Shouldn't happen, because description check in CI should catch such PRs. - # Fall through, so that it shows up in output and the user can fix it. - category = "NO CL CATEGORY" - - # Filter out the PR categories that are not for changelog. - if re.match( - r"(?i)((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", - category, - ): - category = "NOT FOR CHANGELOG / INSIGNIFICANT" - return Description(item.number, item.user, item.html_url, item.title, category) - - # Normalize bug fixes - if re.match( - r"(?i)bug\Wfix", - category, - ): - category = "Bug Fix (user-visible misbehavior in an official stable release)" - - # Filter out documentations changelog - if re.match( - r"(?i)doc", - category, - ): - return None - - if backport_number != item.number: - entry = f"Backported in #{backport_number}: {entry}" - - if not entry: - # Shouldn't happen, because description check in CI should catch such PRs. - category = "NO CL ENTRY" - entry = "NO CL ENTRY: '" + item.title + "'" - - entry = entry.strip() - if entry[-1] != ".": - entry += "." - - for c in categories_preferred_order: - if ratio(category.lower(), c.lower()) >= 90: - category = c - break - - return Description(item.number, item.user, item.html_url, entry, category) - - -def write_changelog( - fd: TextIO, descriptions: Dict[str, List[Description]], year: int -) -> None: - to_commit = runner(f"git rev-parse {TO_REF}^{{}}")[:11] - from_commit = runner(f"git rev-parse {FROM_REF}^{{}}")[:11] - fd.write( - f"---\nsidebar_position: 1\nsidebar_label: {year}\n---\n\n" - f"# {year} Changelog\n\n" - f"### ClickHouse release {TO_REF} ({to_commit}) FIXME " - f"as compared to {FROM_REF} ({from_commit})\n\n" - ) - - seen_categories = [] # type: List[str] - for category in categories_preferred_order: - if category in descriptions: - seen_categories.append(category) - fd.write(f"#### {category}\n") - for desc in descriptions[category]: - fd.write(f"{desc.formatted_entry}\n") - - fd.write("\n") - - for category in sorted(descriptions): - if category not in seen_categories: - fd.write(f"#### {category}\n\n") - for desc in descriptions[category]: - fd.write(f"{desc.formatted_entry}\n") - - fd.write("\n") - - -def check_refs(from_ref: Optional[str], to_ref: str, with_testing_tags: bool): - global FROM_REF, TO_REF - TO_REF = to_ref - - # Check TO_REF - runner.run(f"git rev-parse {TO_REF}") - - # Check from_ref - if from_ref is None: - # Get all tags pointing to TO_REF - tags = runner.run(f"git tag --points-at '{TO_REF}^{{}}'").split("\n") - logging.info("All tags pointing to %s:\n%s", TO_REF, tags) - if not with_testing_tags: - tags.append("*-testing") - exclude = " ".join([f"--exclude='{tag}'" for tag in tags]) - cmd = f"git describe --abbrev=0 --tags {exclude} '{TO_REF}'" - FROM_REF = runner.run(cmd) - else: - runner.run(f"git rev-parse {FROM_REF}") - FROM_REF = from_ref - - -def set_sha_in_changelog(): - global SHA_IN_CHANGELOG - SHA_IN_CHANGELOG = runner.run( - f"git log --format=format:%H {FROM_REF}..{TO_REF}" - ).split("\n") - - -def get_year(prs: PullRequests) -> int: - if not prs: - return date.today().year - return max(pr.created_at.year for pr in prs) - - -def main(): - log_levels = [logging.WARN, logging.INFO, logging.DEBUG] - args = parse_args() - logging.basicConfig( - format="%(asctime)s %(levelname)-8s [%(filename)s:%(lineno)d]:\n%(message)s", - level=log_levels[min(args.verbose, 2)], - ) - if args.debug_helpers: - logging.getLogger("github_helper").setLevel(logging.DEBUG) - logging.getLogger("git_helper").setLevel(logging.DEBUG) - # Create a cache directory - if not p.isdir(CACHE_PATH): - os.mkdir(CACHE_PATH, 0o700) - - # Get the full repo - if is_shallow(): - logging.info("Unshallow repository") - runner.run("git fetch --unshallow", stderr=DEVNULL) - logging.info("Fetching all tags") - runner.run("git fetch --tags", stderr=DEVNULL) - - check_refs(args.from_ref, args.to_ref, args.with_testing_tags) - set_sha_in_changelog() - - logging.info("Using %s..%s as changelog interval", FROM_REF, TO_REF) - - # use merge-base commit as a starting point, if used ref in another branch - base_commit = runner.run(f"git merge-base '{FROM_REF}^{{}}' '{TO_REF}^{{}}'") - # Get starting and ending dates for gathering PRs - # Add one day after and before to mitigate TZ possible issues - # `tag^{}` format gives commit ref when we have annotated tags - # format %cs gives a committer date, works better for cherry-picked commits - from_date = runner.run(f"git log -1 --format=format:%cs '{base_commit}'") - to_date = runner.run(f"git log -1 --format=format:%cs '{TO_REF}^{{}}'") - merged = ( - date.fromisoformat(from_date) - timedelta(1), - date.fromisoformat(to_date) + timedelta(1), - ) - - # Get all PRs for the given time frame - global gh - gh = GitHub( - args.gh_user_or_token, - args.gh_password, - create_cache_dir=False, - per_page=100, - pool_size=args.jobs, - ) - gh.cache_path = CACHE_PATH - query = f"type:pr repo:{args.repo} is:merged" - prs = gh.get_pulls_from_search(query=query, merged=merged, sort="created") - - descriptions = get_descriptions(prs) - changelog_year = get_year(prs) - - write_changelog(args.output, descriptions, changelog_year) +import subprocess +import sys +from pathlib import Path +SCRIPT_PATH = (Path(__file__).parents[2] / "tests/ci/changelog.py").absolute() if __name__ == "__main__": - main() + subprocess.check_call(["python3", SCRIPT_PATH, *sys.argv[1:]]) diff --git a/utils/changelog/git_helper.py b/utils/changelog/git_helper.py deleted file mode 120000 index 03b05a7eddd..00000000000 --- a/utils/changelog/git_helper.py +++ /dev/null @@ -1 +0,0 @@ -../../tests/ci/git_helper.py \ No newline at end of file diff --git a/utils/changelog/github_helper.py b/utils/changelog/github_helper.py deleted file mode 120000 index 2d44dfe8000..00000000000 --- a/utils/changelog/github_helper.py +++ /dev/null @@ -1 +0,0 @@ -../../tests/ci/github_helper.py \ No newline at end of file From 0803e87a9354c0c03e1d11f4844cab19df0248b1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jun 2024 14:20:51 +0200 Subject: [PATCH 574/856] Use GitHubCache in changelog.py --- tests/ci/changelog.py | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index b1a43b1520f..a08866eb1aa 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -3,10 +3,9 @@ import argparse import logging -import os -import os.path as p import re from datetime import date, timedelta +from pathlib import Path from subprocess import DEVNULL from typing import Dict, List, Optional, TextIO @@ -14,9 +13,12 @@ from github.GithubException import RateLimitExceededException, UnknownObjectExce from github.NamedUser import NamedUser from thefuzz.fuzz import ratio # type: ignore +from cache_utils import GitHubCache +from env_helper import TEMP_PATH from git_helper import git_runner as runner from git_helper import is_shallow from github_helper import GitHub, PullRequest, PullRequests, Repository +from s3_helper import S3Helper # This array gives the preferred category order, and is also used to # normalize category names. @@ -37,7 +39,6 @@ FROM_REF = "" TO_REF = "" SHA_IN_CHANGELOG = [] # type: List[str] gh = GitHub(create_cache_dir=False) -CACHE_PATH = p.join(p.dirname(p.realpath(__file__)), "gh_cache") class Description: @@ -375,9 +376,6 @@ def main(): if args.debug_helpers: logging.getLogger("github_helper").setLevel(logging.DEBUG) logging.getLogger("git_helper").setLevel(logging.DEBUG) - # Create a cache directory - if not p.isdir(CACHE_PATH): - os.mkdir(CACHE_PATH, 0o700) # Get the full repo if is_shallow(): @@ -413,7 +411,9 @@ def main(): per_page=100, pool_size=args.jobs, ) - gh.cache_path = CACHE_PATH + temp_path = Path(TEMP_PATH) + gh_cache = GitHubCache(gh.cache_path, temp_path, S3Helper()) + gh_cache.download() query = f"type:pr repo:{args.repo} is:merged" prs = gh.get_pulls_from_search(query=query, merged=merged, sort="created") @@ -421,6 +421,7 @@ def main(): changelog_year = get_year(prs) write_changelog(args.output, descriptions, changelog_year) + gh_cache.upload() if __name__ == "__main__": From 55350a33381ad04320fda49d3fcf6d863c2bea74 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jun 2024 18:40:41 +0200 Subject: [PATCH 575/856] Fix style issues --- tests/ci/changelog.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index a08866eb1aa..dcdc5d515b8 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -7,7 +7,7 @@ import re from datetime import date, timedelta from pathlib import Path from subprocess import DEVNULL -from typing import Dict, List, Optional, TextIO +from typing import Any, Dict, List, Optional, TextIO from github.GithubException import RateLimitExceededException, UnknownObjectException from github.NamedUser import NamedUser @@ -84,10 +84,10 @@ class Description: ) # Sort PR descriptions by numbers - def __eq__(self, other) -> bool: + def __eq__(self, other: Any) -> bool: if not isinstance(self, type(other)): - return NotImplemented - return self.number == other.number + raise NotImplementedError + return bool(self.number == other.number) def __lt__(self, other: "Description") -> bool: return self.number < other.number @@ -331,7 +331,7 @@ def write_changelog( fd.write("\n") -def check_refs(from_ref: Optional[str], to_ref: str, with_testing_tags: bool): +def check_refs(from_ref: Optional[str], to_ref: str, with_testing_tags: bool) -> None: global FROM_REF, TO_REF TO_REF = to_ref From 3cc099a88ed37e4fae9a62b207b31b6bc471fadc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jun 2024 20:05:18 +0200 Subject: [PATCH 576/856] Adjust changelog.py to a new release model with v24.6.1.1-new tags --- tests/ci/changelog.py | 46 ++++++++++++++++++++++++++++++++----------- 1 file changed, 34 insertions(+), 12 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index dcdc5d515b8..95b9ee9be27 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -15,10 +15,15 @@ from thefuzz.fuzz import ratio # type: ignore from cache_utils import GitHubCache from env_helper import TEMP_PATH -from git_helper import git_runner as runner -from git_helper import is_shallow +from git_helper import git_runner, is_shallow from github_helper import GitHub, PullRequest, PullRequests, Repository from s3_helper import S3Helper +from version_helper import ( + FILE_WITH_VERSION_PATH, + get_abs_path, + get_version_from_repo, + get_version_from_tag, +) # This array gives the preferred category order, and is also used to # normalize category names. @@ -39,6 +44,7 @@ FROM_REF = "" TO_REF = "" SHA_IN_CHANGELOG = [] # type: List[str] gh = GitHub(create_cache_dir=False) +runner = git_runner class Description: @@ -339,18 +345,34 @@ def check_refs(from_ref: Optional[str], to_ref: str, with_testing_tags: bool) -> runner.run(f"git rev-parse {TO_REF}") # Check from_ref - if from_ref is None: - # Get all tags pointing to TO_REF - tags = runner.run(f"git tag --points-at '{TO_REF}^{{}}'").split("\n") - logging.info("All tags pointing to %s:\n%s", TO_REF, tags) - if not with_testing_tags: - tags.append("*-testing") - exclude = " ".join([f"--exclude='{tag}'" for tag in tags]) - cmd = f"git describe --abbrev=0 --tags {exclude} '{TO_REF}'" - FROM_REF = runner.run(cmd) - else: + if from_ref is not None: runner.run(f"git rev-parse {FROM_REF}") FROM_REF = from_ref + return + + # Get the cmake/autogenerated_versions.txt from FROM_REF to read the version + # If the previous tag is greater than version in the FROM_REF, + # then we need to add it to tags_to_exclude + temp_cmake = "tests/ci/tmp/autogenerated_versions.txt" + cmake_version = get_abs_path(temp_cmake) + cmake_version.write_text(runner(f"git show {TO_REF}:{FILE_WITH_VERSION_PATH}")) + to_ref_version = get_version_from_repo(cmake_version) + # Get all tags pointing to TO_REF + excluded_tags = runner.run(f"git tag --points-at '{TO_REF}^{{}}'").split("\n") + logging.info("All tags pointing to %s:\n%s", TO_REF, excluded_tags) + if not with_testing_tags: + excluded_tags.append("*-testing") + while not from_ref: + exclude = " ".join([f"--exclude='{tag}'" for tag in excluded_tags]) + from_ref_tag = runner(f"git describe --abbrev=0 --tags {exclude} '{TO_REF}'") + from_ref_version = get_version_from_tag(from_ref_tag) + if from_ref_version <= to_ref_version: + from_ref = from_ref_tag + break + excluded_tags.append(from_ref_tag) + + cmake_version.unlink() + FROM_REF = from_ref def set_sha_in_changelog(): From 335ec8f3e181ae9df21a6295cfac10e51a99030a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 10 Jun 2024 13:43:22 +0200 Subject: [PATCH 577/856] Parse `changelog entry is not required` properly --- tests/ci/changelog.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 95b9ee9be27..0cf21589669 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -266,7 +266,9 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri # Filter out the PR categories that are not for changelog. if re.match( - r"(?i)((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", + r"(?i)((non|in|not|un)[-\s]*significant)|" + r"(not[ ]*for[ ]*changelog)|" + r"(changelog[ ]*entry[ ]*is[ ]*not[ ]*required)", category, ): category = "NOT FOR CHANGELOG / INSIGNIFICANT" From f4630e9daed0786b163572bb9f6d48bf509d36af Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 10 Jun 2024 14:18:13 +0200 Subject: [PATCH 578/856] Fix `Backported in` for not-for-changelog PRs --- tests/ci/changelog.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 0cf21589669..bc52a47fb38 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -264,15 +264,22 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri # Fall through, so that it shows up in output and the user can fix it. category = "NO CL CATEGORY" - # Filter out the PR categories that are not for changelog. + # Filter out documentations changelog before not-for-changelog if re.match( + r"(?i)doc", + category, + ): + return None + + # Filter out the PR categories that are not for changelog. + if re.search( r"(?i)((non|in|not|un)[-\s]*significant)|" r"(not[ ]*for[ ]*changelog)|" r"(changelog[ ]*entry[ ]*is[ ]*not[ ]*required)", category, ): category = "NOT FOR CHANGELOG / INSIGNIFICANT" - return Description(item.number, item.user, item.html_url, item.title, category) + entry = item.title # Normalize bug fixes if re.match( @@ -281,13 +288,6 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri ): category = "Bug Fix (user-visible misbehavior in an official stable release)" - # Filter out documentations changelog - if re.match( - r"(?i)doc", - category, - ): - return None - if backport_number != item.number: entry = f"Backported in #{backport_number}: {entry}" From 3dba47297cf9a071f148c6753d76b0fbd8915100 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 10 Jun 2024 14:28:33 +0200 Subject: [PATCH 579/856] Generate omit v24.1.6.52-stable.md changelog --- docs/changelogs/v24.1.6.52-stable.md | 45 ++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 docs/changelogs/v24.1.6.52-stable.md diff --git a/docs/changelogs/v24.1.6.52-stable.md b/docs/changelogs/v24.1.6.52-stable.md new file mode 100644 index 00000000000..341561e9a64 --- /dev/null +++ b/docs/changelogs/v24.1.6.52-stable.md @@ -0,0 +1,45 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.1.6.52-stable (fa09f677bc9) FIXME as compared to v24.1.5.6-stable (7f67181ff31) + +#### Improvement +* Backported in [#60292](https://github.com/ClickHouse/ClickHouse/issues/60292): Copy S3 file GCP fallback to buffer copy in case GCP returned `Internal Error` with `GATEWAY_TIMEOUT` HTTP error code. [#60164](https://github.com/ClickHouse/ClickHouse/pull/60164) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#60832](https://github.com/ClickHouse/ClickHouse/issues/60832): Update tzdata to 2024a. [#60768](https://github.com/ClickHouse/ClickHouse/pull/60768) ([Raúl Marín](https://github.com/Algunenano)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#60413](https://github.com/ClickHouse/ClickHouse/issues/60413): Fix segmentation fault in KQL parser when the input query exceeds the `max_query_size`. Also re-enable the KQL dialect. Fixes [#59036](https://github.com/ClickHouse/ClickHouse/issues/59036) and [#59037](https://github.com/ClickHouse/ClickHouse/issues/59037). [#59626](https://github.com/ClickHouse/ClickHouse/pull/59626) ([Yong Wang](https://github.com/kashwy)). +* Backported in [#60074](https://github.com/ClickHouse/ClickHouse/issues/60074): Fix error `Read beyond last offset` for `AsynchronousBoundedReadBuffer`. [#59630](https://github.com/ClickHouse/ClickHouse/pull/59630) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#60299](https://github.com/ClickHouse/ClickHouse/issues/60299): Fix having neigher acked nor nacked messages. If exception happens during read-write phase, messages will be nacked. [#59775](https://github.com/ClickHouse/ClickHouse/pull/59775) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#60066](https://github.com/ClickHouse/ClickHouse/issues/60066): Fix optimize_uniq_to_count removing the column alias. [#60026](https://github.com/ClickHouse/ClickHouse/pull/60026) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#60638](https://github.com/ClickHouse/ClickHouse/issues/60638): Fixed a bug in parallel optimization for queries with `FINAL`, which could give an incorrect result in rare cases. [#60041](https://github.com/ClickHouse/ClickHouse/pull/60041) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#60177](https://github.com/ClickHouse/ClickHouse/issues/60177): Fix cosineDistance crash with Nullable. [#60150](https://github.com/ClickHouse/ClickHouse/pull/60150) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#60279](https://github.com/ClickHouse/ClickHouse/issues/60279): Hide sensitive info for `S3Queue` table engine. [#60233](https://github.com/ClickHouse/ClickHouse/pull/60233) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#61000](https://github.com/ClickHouse/ClickHouse/issues/61000): Reduce the number of read rows from `system.numbers`. Fixes [#59418](https://github.com/ClickHouse/ClickHouse/issues/59418). [#60546](https://github.com/ClickHouse/ClickHouse/pull/60546) ([JackyWoo](https://github.com/JackyWoo)). +* Backported in [#60791](https://github.com/ClickHouse/ClickHouse/issues/60791): Fix buffer overflow that can happen if the attacker asks the HTTP server to decompress data with a composition of codecs and size triggering numeric overflow. Fix buffer overflow that can happen inside codec NONE on wrong input data. This was submitted by TIANGONG research team through our [Bug Bounty program](https://github.com/ClickHouse/ClickHouse/issues/38986). [#60731](https://github.com/ClickHouse/ClickHouse/pull/60731) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#60783](https://github.com/ClickHouse/ClickHouse/issues/60783): Functions for SQL/JSON were able to read uninitialized memory. This closes [#60017](https://github.com/ClickHouse/ClickHouse/issues/60017). Found by Fuzzer. [#60738](https://github.com/ClickHouse/ClickHouse/pull/60738) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#60803](https://github.com/ClickHouse/ClickHouse/issues/60803): Do not set aws custom metadata `x-amz-meta-*` headers on UploadPart & CompleteMultipartUpload calls. [#60748](https://github.com/ClickHouse/ClickHouse/pull/60748) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Backported in [#60820](https://github.com/ClickHouse/ClickHouse/issues/60820): Fix crash in arrayEnumerateRanked. [#60764](https://github.com/ClickHouse/ClickHouse/pull/60764) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#60841](https://github.com/ClickHouse/ClickHouse/issues/60841): Fix crash when using input() in INSERT SELECT JOIN. Closes [#60035](https://github.com/ClickHouse/ClickHouse/issues/60035). [#60765](https://github.com/ClickHouse/ClickHouse/pull/60765) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#60904](https://github.com/ClickHouse/ClickHouse/issues/60904): Avoid segfault if too many keys are skipped when reading from S3. [#60849](https://github.com/ClickHouse/ClickHouse/pull/60849) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NO CL CATEGORY + +* Backported in [#60186](https://github.com/ClickHouse/ClickHouse/issues/60186):. [#60181](https://github.com/ClickHouse/ClickHouse/pull/60181) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#60333](https://github.com/ClickHouse/ClickHouse/issues/60333): CI: Fix job failures due to jepsen artifacts. [#59890](https://github.com/ClickHouse/ClickHouse/pull/59890) ([Max K.](https://github.com/maxknv)). +* Backported in [#60034](https://github.com/ClickHouse/ClickHouse/issues/60034): Fix mark release ready. [#59994](https://github.com/ClickHouse/ClickHouse/pull/59994) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#60326](https://github.com/ClickHouse/ClickHouse/issues/60326): Ability to detect undead ZooKeeper sessions. [#60044](https://github.com/ClickHouse/ClickHouse/pull/60044) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#60363](https://github.com/ClickHouse/ClickHouse/issues/60363): CI: hot fix for gh statuses. [#60201](https://github.com/ClickHouse/ClickHouse/pull/60201) ([Max K.](https://github.com/maxknv)). +* Backported in [#60648](https://github.com/ClickHouse/ClickHouse/issues/60648): Detect io_uring in tests. [#60373](https://github.com/ClickHouse/ClickHouse/pull/60373) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#60569](https://github.com/ClickHouse/ClickHouse/issues/60569): Remove broken test while we fix it. [#60547](https://github.com/ClickHouse/ClickHouse/pull/60547) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#60756](https://github.com/ClickHouse/ClickHouse/issues/60756): Update shellcheck. [#60553](https://github.com/ClickHouse/ClickHouse/pull/60553) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#60584](https://github.com/ClickHouse/ClickHouse/issues/60584): CI: fix docker build job name. [#60554](https://github.com/ClickHouse/ClickHouse/pull/60554) ([Max K.](https://github.com/maxknv)). + From 4b90e0ecd4c3f4d6d7029415f220b0a9dc98cced Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Jun 2024 13:23:36 +0200 Subject: [PATCH 580/856] Improve splitting search_issues, solving pylint issues --- tests/ci/github_helper.py | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index eb0f6c24527..3fe72214430 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -49,38 +49,43 @@ class GitHub(github.Github): """Wrapper around search method with throttling and splitting by date. We split only by the first""" - splittable = False + splittable_arg = "" + splittable_value = [] for arg, value in kwargs.items(): if arg in ["closed", "created", "merged", "updated"]: if hasattr(value, "__iter__") and not isinstance(value, str): - assert [True for v in value if isinstance(v, (date, datetime))] + assert all(True for v in value if isinstance(v, (date, datetime))) assert len(value) == 2 kwargs[arg] = f"{value[0].isoformat()}..{value[1].isoformat()}" - if not splittable: + if not splittable_arg: # We split only by the first met splittable argument - preserved_arg = arg - preserved_value = value middle_value = value[0] + (value[1] - value[0]) / 2 - splittable = middle_value not in value + if middle_value in value: + # When the middle value in itareble value, we can't use it + # to split by dates later + continue + splittable_arg = arg + splittable_value = value continue assert isinstance(value, (date, datetime, str)) inter_result = [] # type: Issues + exception = RateLimitExceededException(0) for i in range(self.retries): try: logger.debug("Search issues, args=%s, kwargs=%s", args, kwargs) result = super().search_issues(*args, **kwargs) - if result.totalCount == 1000 and splittable: + if result.totalCount == 1000 and splittable_arg: # The hard limit is 1000. If it's splittable, then we make # two subrequests requests with less time frames logger.debug( "The search result contain exactly 1000 results, " "splitting %s=%s by middle point %s", - preserved_arg, - kwargs[preserved_arg], + splittable_arg, + kwargs[splittable_arg], middle_value, ) - kwargs[preserved_arg] = [preserved_value[0], middle_value] + kwargs[splittable_arg] = [splittable_value[0], middle_value] inter_result.extend(self.search_issues(*args, **kwargs)) if isinstance(middle_value, date): # When middle_value is a date, 2022-01-01..2022-01-03 @@ -88,9 +93,10 @@ class GitHub(github.Github): # 2022-01-02..2022-01-03, so we have results for # 2022-01-02 twicely. We split it to # 2022-01-01..2022-01-02 and 2022-01-03..2022-01-03. - # 2022-01-01..2022-01-02 aren't split, see splittable + # 2022-01-01..2022-01-02 aren't split, see splittable_arg + # definition above for kwargs.items middle_value += timedelta(days=1) - kwargs[preserved_arg] = [middle_value, preserved_value[1]] + kwargs[splittable_arg] = [middle_value, splittable_value[1]] inter_result.extend(self.search_issues(*args, **kwargs)) return inter_result From c797c8105d52c385dc1f2872e888db0b5c1462cf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Jun 2024 13:24:29 +0200 Subject: [PATCH 581/856] Allow to pass tqdm.tqdm into get_pulls_from_search --- tests/ci/github_helper.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 3fe72214430..b6407c5d531 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -6,7 +6,7 @@ from datetime import date, datetime, timedelta from os import path as p from pathlib import Path from time import sleep -from typing import List, Optional, Tuple, Union +from typing import Any, Callable, List, Optional, Tuple, Union import github import requests @@ -110,12 +110,15 @@ class GitHub(github.Github): raise exception # pylint: enable=signature-differs - def get_pulls_from_search(self, *args, **kwargs) -> PullRequests: # type: ignore + def get_pulls_from_search(self, *args: Any, **kwargs: Any) -> PullRequests: """The search api returns actually issues, so we need to fetch PullRequests""" issues = self.search_issues(*args, **kwargs) repos = {} prs = [] # type: PullRequests - for issue in issues: + progress_func = kwargs.pop( + "progress_func", lambda x: x + ) # type: Callable[[Issues], Issues] + for issue in progress_func(issues): # See https://github.com/PyGithub/PyGithub/issues/2202, # obj._rawData doesn't spend additional API requests # pylint: disable=protected-access From 94c82787c555237e823ceebcc75b24016aceba62 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Jun 2024 13:26:35 +0200 Subject: [PATCH 582/856] Add a progress function for searching PRs in changelog.py --- docker/test/style/Dockerfile | 1 + tests/ci/changelog.py | 5 ++++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 91768c8328d..54fab849301 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -33,6 +33,7 @@ RUN pip3 install \ flake8==4.0.1 \ requests \ thefuzz \ + tqdm==4.66.4 \ types-requests \ unidiff \ && rm -rf /root/.cache/pip diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index bc52a47fb38..fcb61d3f605 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -9,6 +9,7 @@ from pathlib import Path from subprocess import DEVNULL from typing import Any, Dict, List, Optional, TextIO +import tqdm # type: ignore from github.GithubException import RateLimitExceededException, UnknownObjectException from github.NamedUser import NamedUser from thefuzz.fuzz import ratio # type: ignore @@ -439,7 +440,9 @@ def main(): gh_cache = GitHubCache(gh.cache_path, temp_path, S3Helper()) gh_cache.download() query = f"type:pr repo:{args.repo} is:merged" - prs = gh.get_pulls_from_search(query=query, merged=merged, sort="created") + prs = gh.get_pulls_from_search( + query=query, merged=merged, sort="created", progress_func=tqdm.tqdm + ) descriptions = get_descriptions(prs) changelog_year = get_year(prs) From feef24b9a3e8fad38f2e6a7c52fe4a9267d1e2ce Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Jun 2024 12:51:05 +0000 Subject: [PATCH 583/856] Rename submodule "abseil" to "contrib/abseil" --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 6d64c52ce00..0020bdd006a 100644 --- a/.gitmodules +++ b/.gitmodules @@ -161,7 +161,7 @@ [submodule "contrib/xz"] path = contrib/xz url = https://github.com/xz-mirror/xz -[submodule "abseil"] +[submodule "contrib/abseil"] path = contrib/abseil-cpp url = https://github.com/ClickHouse/abseil-cpp.git [submodule "contrib/dragonbox"] From 13b23e9e9c3465902ad57c6b1e802c2d54832e42 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Jun 2024 13:05:33 +0000 Subject: [PATCH 584/856] Check submodule name in style check --- .gitmodules | 6 +++--- utils/check-style/check-style | 4 ++++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index 0020bdd006a..a3bb7eacabc 100644 --- a/.gitmodules +++ b/.gitmodules @@ -91,13 +91,13 @@ [submodule "contrib/aws"] path = contrib/aws url = https://github.com/ClickHouse/aws-sdk-cpp -[submodule "aws-c-event-stream"] +[submodule "contrib/aws-c-event-stream"] path = contrib/aws-c-event-stream url = https://github.com/awslabs/aws-c-event-stream -[submodule "aws-c-common"] +[submodule "contrib/aws-c-common"] path = contrib/aws-c-common url = https://github.com/awslabs/aws-c-common.git -[submodule "aws-checksums"] +[submodule "contrib/aws-checksums"] path = contrib/aws-checksums url = https://github.com/awslabs/aws-checksums [submodule "contrib/curl"] diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 5c05907e9dd..4c6a4b9ea39 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -243,6 +243,10 @@ done # All the submodules should be from https://github.com/ find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | grep -v -F 'https://github.com/' && echo 'All the submodules should be from https://github.com/'; done +# All submodules should be of this form: [submodule "contrib/libxyz"] (for consistency, the submodule name does matter too much) +# - restrict the check to top-level .gitmodules file +find $ROOT_PATH -maxdepth 1 -name '.gitmodules' | while read i; do grep -F '[submodule ' $i | grep -v -F 'contrib' && echo 'All submodules should have form [submodule "contrib/libxyz"]'; done + # There shouldn't be any code snippets under GPL or LGPL find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL" From 089f5bfecdf26d18a5e3dda3e3f8f2c4a86deac5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 12 Jun 2024 15:09:38 +0200 Subject: [PATCH 585/856] 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 84e81daa3ec51046d027188763e2e6c6d7a9c9f5 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 12 Jun 2024 15:09:50 +0200 Subject: [PATCH 586/856] Second pass fix remaining broken links --- .../aggregate-functions/combinators.md | 8 +-- .../parametric-functions.md | 16 +++-- .../reference/stochasticlinearregression.md | 2 +- .../reference/stochasticlogisticregression.md | 2 +- .../aggregate-functions/reference/varpop.md | 8 +-- .../aggregate-functions/reference/varsamp.md | 6 +- docs/en/sql-reference/dictionaries/index.md | 60 +++++++++---------- .../functions/bitmap-functions.md | 6 +- .../functions/date-time-functions.md | 4 +- .../en/sql-reference/functions/geo/geohash.md | 8 +-- docs/en/sql-reference/functions/geo/h3.md | 44 +++++++------- docs/en/sql-reference/functions/geo/s2.md | 10 ++-- .../functions/other-functions.md | 10 ++-- .../functions/rounding-functions.md | 2 +- .../functions/string-search-functions.md | 2 +- .../functions/type-conversion-functions.md | 30 +++++----- docs/en/sql-reference/operators/in.md | 2 +- .../table-functions/fileCluster.md | 2 +- 18 files changed, 113 insertions(+), 109 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index 5351531afdb..e30aa66b3b3 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -106,14 +106,14 @@ To work with these states, use: - [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) table engine. - [finalizeAggregation](../../sql-reference/functions/other-functions.md#function-finalizeaggregation) function. - [runningAccumulate](../../sql-reference/functions/other-functions.md#runningaccumulate) function. -- [-Merge](#aggregate_functions_combinators_merge) combinator. -- [-MergeState](#aggregate_functions_combinators_mergestate) combinator. +- [-Merge](#-merge) combinator. +- [-MergeState](#-mergestate) combinator. -## -Merge {#aggregate_functions_combinators_merge} +## -Merge If you apply this combinator, the aggregate function takes the intermediate aggregation state as an argument, combines the states to finish aggregation, and returns the resulting value. -## -MergeState {#aggregate_functions_combinators_mergestate} +## -MergeState Merges the intermediate aggregation states in the same way as the -Merge combinator. However, it does not return the resulting value, but an intermediate aggregation state, similar to the -State combinator. diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 43ded9df60a..093d88f939f 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -82,10 +82,12 @@ FROM In this case, you should remember that you do not know the histogram bin borders. -## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) {#function-sequencematch} +## sequenceMatch Checks whether the sequence contains an event chain that matches the pattern. +**Syntax** + ``` sql sequenceMatch(pattern)(timestamp, cond1, cond2, ...) ``` @@ -102,7 +104,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). +- `pattern` — Pattern string. See [Pattern syntax](#sequencematch). **Returned values** @@ -170,9 +172,9 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM **See Also** -- [sequenceCount](#function-sequencecount) +- [sequenceCount](#sequencecount) -## sequenceCount(pattern)(time, cond1, cond2, ...) {#function-sequencecount} +## sequenceCount Counts the number of event chains that matched the pattern. The function searches event chains that do not overlap. It starts to search for the next chain after the current chain is matched. @@ -180,6 +182,8 @@ Counts the number of event chains that matched the pattern. The function searche Events that occur at the same second may lay in the sequence in an undefined order affecting the result. ::: +**Syntax** + ``` sql sequenceCount(pattern)(timestamp, cond1, cond2, ...) ``` @@ -192,7 +196,7 @@ sequenceCount(pattern)(timestamp, cond1, cond2, ...) **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). +- `pattern` — Pattern string. See [Pattern syntax](#sequencematch). **Returned values** @@ -229,7 +233,7 @@ SELECT sequenceCount('(?1).*(?2)')(time, number = 1, number = 2) FROM t **See Also** -- [sequenceMatch](#function-sequencematch) +- [sequenceMatch](#sequencematch) ## windowFunnel diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md index 15533ba9fd7..7ab9e1d3256 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md @@ -72,5 +72,5 @@ The query will return a column of predicted values. Note that first argument of **See Also** -- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#agg_functions_stochasticlinearregression_parameters) +- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#stochasticlogisticregression) - [Difference between linear and logistic regressions](https://stackoverflow.com/questions/12146914/what-is-the-difference-between-linear-regression-and-logistic-regression) diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md index 0a040689681..4bf5529ddcb 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md @@ -11,7 +11,7 @@ This function implements stochastic logistic regression. It can be used for bina Parameters are exactly the same as in stochasticLinearRegression: `learning rate`, `l2 regularization coefficient`, `mini-batch size`, `method for updating weights`. -For more information see [parameters](#agg_functions-stochasticlinearregression-parameters). +For more information see [parameters](../reference/stochasticlinearregression.md/#parameters). ``` text stochasticLogisticRegression(1.0, 1.0, 10, 'SGD') diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpop.md b/docs/en/sql-reference/aggregate-functions/reference/varpop.md index d2b19fe2a3e..4e010248f6e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varpop.md @@ -6,7 +6,7 @@ sidebar_position: 32 This page covers the `varPop` and `varPopStable` functions available in ClickHouse. -## varPop {#varPop} +## 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`. @@ -27,7 +27,7 @@ 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`](#varPopStable) function. +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable`](#varpopstable) function. **Example** @@ -55,7 +55,7 @@ Result: 3 ``` -## varPopStable {#varPopStable} +## 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. @@ -76,7 +76,7 @@ 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. +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** diff --git a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md index e9ec9ba2bc1..bd1cfa5742a 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md @@ -40,7 +40,7 @@ Where: 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`](#varSampStable) function. +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varSampStable`](#varsampstable) function. **Example** @@ -66,7 +66,7 @@ Response: 0.8650000000000091 ``` -## varSampStable {#varSampStable} +## varSampStable Calculate the sample variance of a data set using a numerically stable algorithm. @@ -86,7 +86,7 @@ The `varSampStable` function returns a Float64 value representing the sample var **Implementation details** -The `varSampStable` function calculates the sample variance using the same formula as the [`varSamp`](#varSamp) function: +The `varSampStable` function calculates the sample variance using the same formula as the [`varSamp`](#varsamp) function: ```plaintext ∑(x - mean(x))^2 / (n - 1) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 437b836ec0e..4c7421d57c0 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -23,7 +23,7 @@ ClickHouse supports: If you are getting started with Dictionaries in ClickHouse we have a tutorial that covers that topic. Take a look [here](/docs/en/tutorial.md). ::: -You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](#dictionary_sources)”. +You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](#dictionary-sources)”. ClickHouse: @@ -75,14 +75,14 @@ The dictionary configuration file has the following format: ``` -You can [configure](#configuring_a_dictionary) any number of dictionaries in the same file. +You can [configure](#configuring-a-dictionary) any number of dictionaries in the same file. :::note You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../../sql-reference/functions/other-functions.md) function). This functionality is not related to dictionaries. ::: -## Configuring a Dictionary {#configuring_a_dictionary} +## Configuring a Dictionary @@ -123,7 +123,7 @@ LAYOUT(...) -- Memory layout configuration LIFETIME(...) -- Lifetime of dictionary in memory ``` -## Storing Dictionaries in Memory {#storing-dictionaries-in-memory} +## Storing Dictionaries in Memory There are a variety of ways to store dictionaries in memory. @@ -415,7 +415,7 @@ or LAYOUT(COMPLEX_KEY_HASHED_ARRAY([SHARDS 1])) ``` -### range_hashed {#range_hashed} +### range_hashed The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. @@ -679,7 +679,7 @@ When searching for a dictionary, the cache is searched first. For each block of If keys are not found in dictionary, then update cache task is created and added into update queue. Update queue properties can be controlled with settings `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates`. -For cache dictionaries, the expiration [lifetime](#lifetime) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. +For cache dictionaries, the expiration [lifetime](#refreshing-dictionary-data-using-lifetime) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the [system.dictionaries](../../operations/system-tables/dictionaries.md) table. @@ -771,7 +771,7 @@ The dictionary is not stored in memory and directly goes to the source during th The dictionary key has the [UInt64](../../sql-reference/data-types/int-uint.md) type. -All types of [sources](#dictionary_sources), except local files, are supported. +All types of [sources](#dictionary-sources), except local files, are supported. Configuration example: @@ -899,7 +899,7 @@ Other types are not supported yet. The function returns the attribute for the pr Data must completely fit into RAM. -## Refreshing dictionary data using LIFETIME {#lifetime} +## Refreshing dictionary data using LIFETIME ClickHouse periodically updates dictionaries based on the `LIFETIME` tag (defined in seconds). `LIFETIME` is the update interval for fully downloaded dictionaries and the invalidation interval for cached dictionaries. @@ -952,7 +952,7 @@ LIFETIME(MIN 300 MAX 360) If `0` and `0`, ClickHouse does not reload the dictionary by timeout. In this case, ClickHouse can reload the dictionary earlier if the dictionary configuration file was changed or the `SYSTEM RELOAD DICTIONARY` command was executed. -When updating the dictionaries, the ClickHouse server applies different logic depending on the type of [source](#dictionary_sources): +When updating the dictionaries, the ClickHouse server applies different logic depending on the type of [source](#dictionary-sources): - For a text file, it checks the time of modification. If the time differs from the previously recorded time, the dictionary is updated. - For MySQL source, the time of modification is checked using a `SHOW TABLE STATUS` query (in case of MySQL 8 you need to disable meta-information caching in MySQL by `set global information_schema_stats_expiry=0`). @@ -961,7 +961,7 @@ When updating the dictionaries, the ClickHouse server applies different logic de For other sources (ODBC, PostgreSQL, ClickHouse, etc), you can set up a query that will update the dictionaries only if they really changed, rather than each time. To do this, follow these steps: - The dictionary table must have a field that always changes when the source data is updated. -- The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. Specify the query in the `` field in the settings for the [source](#dictionary_sources). +- The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. Specify the query in the `` field in the settings for the [source](#dictionary-sources). Example of settings: @@ -1031,7 +1031,7 @@ SOURCE(CLICKHOUSE(... update_field 'added_time' update_lag 15)) ... ``` -## Dictionary Sources {#dictionary_sources} +## Dictionary Sources @@ -1065,7 +1065,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration The source is configured in the `source` section. -For source types [Local file](#local_file), [Executable file](#executable), [HTTP(s)](#https), [ClickHouse](#clickhouse) +For source types [Local file](#local-file), [Executable file](#executable-file), [HTTP(s)](#https), [ClickHouse](#clickhouse) optional settings are available: ``` xml @@ -1089,9 +1089,9 @@ SETTINGS(format_csv_allow_single_quotes = 0) Types of sources (`source_type`): -- [Local file](#local_file) -- [Executable File](#executable) -- [Executable Pool](#executable_pool) +- [Local file](#local-file) +- [Executable File](#executable-file) +- [Executable Pool](#executable-pool) - [HTTP(S)](#https) - DBMS - [ODBC](#odbc) @@ -1102,7 +1102,7 @@ Types of sources (`source_type`): - [Cassandra](#cassandra) - [PostgreSQL](#postgresql) -### Local File {#local_file} +### Local File Example of settings: @@ -1132,7 +1132,7 @@ When a dictionary with source `FILE` is created via DDL command (`CREATE DICTION - [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) -### Executable File {#executable} +### Executable File Working with executable files depends on [how the dictionary is stored in memory](#storing-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. @@ -1161,7 +1161,7 @@ Setting fields: That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled; otherwise, the DB user would be able to execute arbitrary binaries on the ClickHouse node. -### Executable Pool {#executable_pool} +### Executable Pool Executable pool allows loading data from pool of processes. This source does not work with dictionary layouts that need to load all data from source. Executable pool works if the dictionary [is stored](#ways-to-store-dictionaries-in-memory) using `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache`, `direct`, or `complex_key_direct` layouts. @@ -1196,9 +1196,9 @@ Setting fields: That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. -### HTTP(S) {#https} +### HTTP(S) -Working with an HTTP(S) server depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. +Working with an HTTP(S) server depends on [how the dictionary is stored in memory](#storing-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. Example of settings: @@ -1285,7 +1285,7 @@ Setting fields: - `db` – Name of the database. Omit it if the database name is set in the `` parameters. - `table` – Name of the table and schema if exists. - `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#lifetime). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#refreshing-dictionary-data-using-lifetime). - `query` – The custom query. Optional parameter. :::note @@ -1575,7 +1575,7 @@ Setting fields: - `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in MySQL, for example, `id > 10 AND id < 20`. Optional parameter. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#lifetime). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#refreshing-dictionary-data-using-lifetime). - `fail_on_connection_loss` – The configuration parameter that controls behavior of the server on connection loss. If `true`, an exception is thrown immediately if the connection between client and server was lost. If `false`, the ClickHouse server retries to execute the query three times before throwing an exception. Note that retrying leads to increased response times. Default value: `false`. @@ -1672,7 +1672,7 @@ Setting fields: - `db` – Name of the database. - `table` – Name of the table. - `where` – The selection criteria. May be omitted. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#lifetime). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#refreshing-dictionary-data-using-lifetime). - `secure` - Use ssl for connection. - `query` – The custom query. Optional parameter. @@ -1849,7 +1849,7 @@ Setting fields: - `db` – Name of the database. - `table` – Name of the table. - `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in PostgreSQL. For example, `id > 10 AND id < 20`. Optional parameter. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#lifetime). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#refreshing-dictionary-data-using-lifetime). - `query` – The custom query. Optional parameter. :::note @@ -1873,7 +1873,7 @@ LAYOUT(FLAT()) LIFETIME(0); ``` -## Dictionary Key and Fields {#dictionary-key-and-fields} +## Dictionary Key and Fields @@ -1963,7 +1963,7 @@ PRIMARY KEY Id ### Composite Key -The key can be a `tuple` from any types of fields. The [layout](#storig-dictionaries-in-memory) in this case must be `complex_key_hashed` or `complex_key_cache`. +The key can be a `tuple` from any types of fields. The [layout](#storing-dictionaries-in-memory) in this case must be `complex_key_hashed` or `complex_key_cache`. :::tip A composite key can consist of a single element. This makes it possible to use a string as the key, for instance. @@ -2036,11 +2036,11 @@ Configuration fields: | `type` | ClickHouse data type: [UInt8](../../sql-reference/data-types/int-uint.md), [UInt16](../../sql-reference/data-types/int-uint.md), [UInt32](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md), [Int8](../../sql-reference/data-types/int-uint.md), [Int16](../../sql-reference/data-types/int-uint.md), [Int32](../../sql-reference/data-types/int-uint.md), [Int64](../../sql-reference/data-types/int-uint.md), [Float32](../../sql-reference/data-types/float.md), [Float64](../../sql-reference/data-types/float.md), [UUID](../../sql-reference/data-types/uuid.md), [Decimal32](../../sql-reference/data-types/decimal.md), [Decimal64](../../sql-reference/data-types/decimal.md), [Decimal128](../../sql-reference/data-types/decimal.md), [Decimal256](../../sql-reference/data-types/decimal.md),[Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), [String](../../sql-reference/data-types/string.md), [Array](../../sql-reference/data-types/array.md).
ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.
[Nullable](../../sql-reference/data-types/nullable.md) is currently supported for [Flat](#flat), [Hashed](#hashed), [ComplexKeyHashed](#complex_key_hashed), [Direct](#direct), [ComplexKeyDirect](#complex_key_direct), [RangeHashed](#range_hashed), Polygon, [Cache](#cache), [ComplexKeyCache](#complex_key_cache), [SSDCache](#ssd_cache), [SSDComplexKeyCache](#complex_key_ssd_cache) dictionaries. In [IPTrie](#ip_trie) dictionaries `Nullable` types are not supported. | Yes | | `null_value` | Default value for a non-existing element.
In the example, it is an empty string. [NULL](../syntax.md#null) value can be used only for the `Nullable` types (see the previous line with types description). | Yes | | `expression` | [Expression](../../sql-reference/syntax.md#expressions) that ClickHouse executes on the value.
The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.

Default value: no expression. | No | -| `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](#hierarchical_dictionaries).

Default value: `false`. | No | +| `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](#hierarchical-dictionaries).

Default value: `false`. | No | | `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.

Default value: `false`. | No | | `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. -## Hierarchical Dictionaries {#hierarchical_dictionaries} +## Hierarchical Dictionaries ClickHouse supports hierarchical dictionaries with a [numeric key](#numeric-key). @@ -2165,7 +2165,7 @@ Points can be specified as an array or a tuple of their coordinates. In the curr The user can upload their own data in all formats supported by ClickHouse. -There are 3 types of [in-memory storage](#storig-dictionaries-in-memory) available: +There are 3 types of [in-memory storage](#storing-dictionaries-in-memory) available: - `POLYGON_SIMPLE`. This is a naive implementation, where a linear pass through all polygons is made for each query, and membership is checked for each one without using additional indexes. @@ -2435,7 +2435,7 @@ LIFETIME(0) LAYOUT(regexp_tree); ``` -## Embedded Dictionaries {#embedded-dictionaries} +## Embedded Dictionaries diff --git a/docs/en/sql-reference/functions/bitmap-functions.md b/docs/en/sql-reference/functions/bitmap-functions.md index d98d7d19d7c..d30c0f4dde4 100644 --- a/docs/en/sql-reference/functions/bitmap-functions.md +++ b/docs/en/sql-reference/functions/bitmap-functions.md @@ -104,7 +104,7 @@ bitmapSubsetLimit(bitmap, range_start, cardinality_limit) **Arguments** -- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild). +- `bitmap` – [Bitmap object](#bitmapbuild). - `range_start` – Start of the range (inclusive). [UInt32](../data-types/int-uint.md). - `cardinality_limit` – Maximum cardinality of the subset. [UInt32](../data-types/int-uint.md). @@ -134,7 +134,7 @@ subBitmap(bitmap, offset, cardinality_limit) **Arguments** -- `bitmap` – The bitmap. [Bitmap object](#bitmap_functions-bitmapbuild). +- `bitmap` – The bitmap. [Bitmap object](#bitmapbuild). - `offset` – The position of the first element of the subset. [UInt32](../data-types/int-uint.md). - `cardinality_limit` – The maximum number of elements in the subset. [UInt32](../data-types/int-uint.md). @@ -162,7 +162,7 @@ bitmapContains(bitmap, needle) **Arguments** -- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild). +- `bitmap` – [Bitmap object](#bitmapbuild). - `needle` – Searched bit value. [UInt32](../data-types/int-uint.md). **Returned values** diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index e8661b5f5c3..b532e0de8f0 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1049,7 +1049,7 @@ toLastDayOfWeek(t[, mode[, timezone]]) **Arguments** - `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) -- `mode` - determines the last day of the week as described in the [toWeek()](date-time-functions#toweek) function +- `mode` - determines the last day of the week as described in the [toWeek](#toweek) function - `timezone` - Optional parameter, it behaves like any other conversion function **Returned value** @@ -1763,7 +1763,7 @@ Result: ## fromDaysSinceYearZero32 -Like [fromDaysSinceYearZero](#fromDaysSinceYearZero) but returns a [Date32](../data-types/date32.md). +Like [fromDaysSinceYearZero](#fromdayssinceyearzero) but returns a [Date32](../data-types/date32.md). ## age diff --git a/docs/en/sql-reference/functions/geo/geohash.md b/docs/en/sql-reference/functions/geo/geohash.md index 9a3d52824f6..b6ac7a74092 100644 --- a/docs/en/sql-reference/functions/geo/geohash.md +++ b/docs/en/sql-reference/functions/geo/geohash.md @@ -4,7 +4,7 @@ sidebar_label: Geohash title: "Functions for Working with Geohash" --- -## Geohash {#geohash_description} +## Geohash [Geohash](https://en.wikipedia.org/wiki/Geohash) is the geocode system, which subdivides Earth’s surface into buckets of grid shape and encodes each cell into a short string of letters and digits. It is a hierarchical data structure, so the longer is the geohash string, the more precise is the geographic location. @@ -12,7 +12,7 @@ If you need to manually convert geographic coordinates to geohash strings, you c ## geohashEncode -Encodes latitude and longitude as a [geohash](#geohash_description)-string. +Encodes latitude and longitude as a [geohash](#geohash)-string. ``` sql geohashEncode(longitude, latitude, [precision]) @@ -42,7 +42,7 @@ SELECT geohashEncode(-5.60302734375, 42.593994140625, 0) AS res; ## geohashDecode -Decodes any [geohash](#geohash_description)-encoded string into longitude and latitude. +Decodes any [geohash](#geohash)-encoded string into longitude and latitude. **Input values** @@ -66,7 +66,7 @@ SELECT geohashDecode('ezs42') AS res; ## geohashesInBox -Returns an array of [geohash](#geohash_description)-encoded strings of given precision that fall inside and intersect boundaries of given box, basically a 2D grid flattened into array. +Returns an array of [geohash](#geohash)-encoded strings of given precision that fall inside and intersect boundaries of given box, basically a 2D grid flattened into array. **Syntax** diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index 6fce91f4d8e..5fbc2adf2fa 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -4,7 +4,7 @@ sidebar_label: H3 Indexes title: "Functions for Working with H3 Indexes" --- -## H3 Index {#h3index} +## H3 Index [H3](https://eng.uber.com/h3/) is a geographical indexing system where Earth’s surface divided into a grid of even hexagonal cells. This system is hierarchical, i. e. each hexagon on the top level ("parent") can be split into seven even but smaller ones ("children"), and so on. @@ -18,7 +18,7 @@ The full description of the H3 system is available at [the Uber Engineering site ## h3IsValid -Verifies whether the number is a valid [H3](#h3index) index. +Verifies whether the number is a valid [H3](#h3-index) index. **Syntax** @@ -53,7 +53,7 @@ Result: ## h3GetResolution -Defines the resolution of the given [H3](#h3index) index. +Defines the resolution of the given [H3](#h3-index) index. **Syntax** @@ -88,7 +88,7 @@ Result: ## h3EdgeAngle -Calculates the average length of the [H3](#h3index) hexagon edge in grades. +Calculates the average length of the [H3](#h3-index) hexagon edge in grades. **Syntax** @@ -102,7 +102,7 @@ h3EdgeAngle(resolution) **Returned values** -- The average length of the [H3](#h3index) hexagon edge in grades. [Float64](../../data-types/float.md). +- The average length of the [H3](#h3-index) hexagon edge in grades. [Float64](../../data-types/float.md). **Example** @@ -122,7 +122,7 @@ Result: ## h3EdgeLengthM -Calculates the average length of the [H3](#h3index) hexagon edge in meters. +Calculates the average length of the [H3](#h3-index) hexagon edge in meters. **Syntax** @@ -136,7 +136,7 @@ h3EdgeLengthM(resolution) **Returned values** -- The average length of the [H3](#h3index) hexagon edge in meters. [Float64](../../data-types/float.md). +- The average length of the [H3](#h3-index) hexagon edge in meters. [Float64](../../data-types/float.md). **Example** @@ -156,7 +156,7 @@ Result: ## h3EdgeLengthKm -Calculates the average length of the [H3](#h3index) hexagon edge in kilometers. +Calculates the average length of the [H3](#h3-index) hexagon edge in kilometers. **Syntax** @@ -170,7 +170,7 @@ h3EdgeLengthKm(resolution) **Returned values** -- The average length of the [H3](#h3index) hexagon edge in kilometers. [Float64](../../data-types/float.md). +- The average length of the [H3](#h3-index) hexagon edge in kilometers. [Float64](../../data-types/float.md). **Example** @@ -190,7 +190,7 @@ Result: ## geoToH3 -Returns [H3](#h3index) point index `(lon, lat)` with specified resolution. +Returns [H3](#h3-index) point index `(lon, lat)` with specified resolution. **Syntax** @@ -227,7 +227,7 @@ Result: ## h3ToGeo -Returns the centroid longitude and latitude corresponding to the provided [H3](#h3index) index. +Returns the centroid longitude and latitude corresponding to the provided [H3](#h3-index) index. **Syntax** @@ -296,7 +296,7 @@ Result: ## h3kRing - Lists all the [H3](#h3index) hexagons in the raduis of `k` from the given hexagon in random order. + Lists all the [H3](#h3-index) hexagons in the raduis of `k` from the given hexagon in random order. **Syntax** @@ -337,7 +337,7 @@ Result: ## h3GetBaseCell -Returns the base cell number of the [H3](#h3index) index. +Returns the base cell number of the [H3](#h3-index) index. **Syntax** @@ -439,7 +439,7 @@ Result: ## h3IndexesAreNeighbors -Returns whether or not the provided [H3](#h3index) indexes are neighbors. +Returns whether or not the provided [H3](#h3-index) indexes are neighbors. **Syntax** @@ -475,7 +475,7 @@ Result: ## h3ToChildren -Returns an array of child indexes for the given [H3](#h3index) index. +Returns an array of child indexes for the given [H3](#h3-index) index. **Syntax** @@ -510,7 +510,7 @@ Result: ## h3ToParent -Returns the parent (coarser) index containing the given [H3](#h3index) index. +Returns the parent (coarser) index containing the given [H3](#h3-index) index. **Syntax** @@ -611,7 +611,7 @@ Result: ## h3GetResolution -Returns the resolution of the [H3](#h3index) index. +Returns the resolution of the [H3](#h3-index) index. **Syntax** @@ -645,7 +645,7 @@ Result: ## h3IsResClassIII -Returns whether [H3](#h3index) index has a resolution with Class III orientation. +Returns whether [H3](#h3-index) index has a resolution with Class III orientation. **Syntax** @@ -680,7 +680,7 @@ Result: ## h3IsPentagon -Returns whether this [H3](#h3index) index represents a pentagonal cell. +Returns whether this [H3](#h3-index) index represents a pentagonal cell. **Syntax** @@ -715,7 +715,7 @@ Result: ## h3GetFaces -Returns icosahedron faces intersected by a given [H3](#h3index) index. +Returns icosahedron faces intersected by a given [H3](#h3-index) index. **Syntax** @@ -817,7 +817,7 @@ Result: ## h3ToCenterChild -Returns the center child (finer) [H3](#h3index) index contained by given [H3](#h3index) at the given resolution. +Returns the center child (finer) [H3](#h3-index) index contained by given [H3](#h3-index) at the given resolution. **Syntax** @@ -832,7 +832,7 @@ h3ToCenterChild(index, resolution) **Returned values** -- [H3](#h3index) index of the center child contained by given [H3](#h3index) at the given resolution. [UInt64](../../data-types/int-uint.md). +- [H3](#h3-index) index of the center child contained by given [H3](#h3-index) at the given resolution. [UInt64](../../data-types/int-uint.md). **Example** diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index bcb6b2833c9..e022ce870b0 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -5,7 +5,7 @@ sidebar_label: S2 Geometry # Functions for Working with S2 Index -## S2Index {#s2_index} +## S2Index [S2](https://s2geometry.io/) is a geographical indexing system where all geographical data is represented on a three-dimensional sphere (similar to a globe). @@ -13,7 +13,7 @@ In the S2 library points are represented as the S2 Index - a specific number whi ## geoToS2 -Returns [S2](#s2_index) point index corresponding to the provided coordinates `(longitude, latitude)`. +Returns [S2](#s2index) point index corresponding to the provided coordinates `(longitude, latitude)`. **Syntax** @@ -48,7 +48,7 @@ Result: ## s2ToGeo -Returns geo coordinates `(longitude, latitude)` corresponding to the provided [S2](#s2_index) point index. +Returns geo coordinates `(longitude, latitude)` corresponding to the provided [S2](#s2index) point index. **Syntax** @@ -84,7 +84,7 @@ Result: ## s2GetNeighbors -Returns S2 neighbor indexes corresponding to the provided [S2](#s2_index). Each cell in the S2 system is a quadrilateral bounded by four geodesics. So, each cell has 4 neighbors. +Returns S2 neighbor indexes corresponding to the provided [S2](#s2index). Each cell in the S2 system is a quadrilateral bounded by four geodesics. So, each cell has 4 neighbors. **Syntax** @@ -118,7 +118,7 @@ Result: ## s2CellsIntersect -Determines if the two provided [S2](#s2_index) cells intersect or not. +Determines if the two provided [S2](#s2index) cells intersect or not. **Syntax** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5eae8b7905e..e22dd5d827c 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -762,7 +762,7 @@ LIMIT 10 Given a size (number of bytes), this function returns a readable, rounded size with suffix (KB, MB, etc.) as string. -The opposite operations of this function are [parseReadableSize](#parseReadableSize), [parseReadableSizeOrZero](#parsereadablesizeorzero), and [parseReadableSizeOrNull](#parsereadablesizeornull). +The opposite operations of this function are [parseReadableSize](#parsereadablesize), [parseReadableSizeOrZero](#parsereadablesizeorzero), and [parseReadableSizeOrNull](#parsereadablesizeornull). **Syntax** @@ -795,7 +795,7 @@ Result: Given a size (number of bytes), this function returns a readable, rounded size with suffix (KiB, MiB, etc.) as string. -The opposite operations of this function are [parseReadableSize](#parseReadableSize), [parseReadableSizeOrZero](#parsereadablesizeorzero), and [parseReadableSizeOrNull](#parsereadablesizeornull). +The opposite operations of this function are [parseReadableSize](#parsereadablesize), [parseReadableSizeOrZero](#parsereadablesizeorzero), and [parseReadableSizeOrNull](#parsereadablesizeornull). **Syntax** @@ -921,7 +921,7 @@ SELECT └────────────────────┴────────────────────────────────────────────────┘ ``` -## parseReadableSize {#parseReadableSize} +## parseReadableSize Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. If the function is unable to parse the input value, it throws an exception. @@ -964,7 +964,7 @@ SELECT Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. If the function is unable to parse the input value, it returns `NULL`. -The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatReadableDecimalSize). +The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatreadabledecimalsize). **Syntax** @@ -1002,7 +1002,7 @@ SELECT Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. If the function is unable to parse the input value, it returns `0`. -The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatReadableDecimalSize). +The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatreadabledecimalsize). **Syntax** diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index c3a915ca195..e2f471d47eb 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -274,7 +274,7 @@ roundBankers(10.755, 2) = 10.76 **See Also** -- [round](#rounding_functions-round) +- [round](#round) ## roundToExp2 diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 5353bbf9b27..b7ba1d4feb7 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -1665,7 +1665,7 @@ Result: ## hasSubsequenceUTF8 -Like [hasSubsequence](#hasSubsequence) but assumes `haystack` and `needle` are UTF-8 encoded strings. +Like [hasSubsequence](#hassubsequence) but assumes `haystack` and `needle` are UTF-8 encoded strings. **Syntax** diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 86739ac0b8d..61e84ca72d1 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -10,7 +10,7 @@ sidebar_label: Type Conversion ClickHouse generally uses the [same behavior as C++ programs](https://en.cppreference.com/w/cpp/language/implicit_conversion). -`to` functions and [cast](#castx-t) behave differently in some cases, for example in case of [LowCardinality](../data-types/lowcardinality.md): [cast](#castx-t) removes [LowCardinality](../data-types/lowcardinality.md) trait `to` functions don't. The same with [Nullable](../data-types/nullable.md), this behaviour is not compatible with SQL standard, and it can be changed using [cast_keep_nullable](../../operations/settings/settings.md/#cast_keep_nullable) setting. +`to` functions and [cast](#cast) behave differently in some cases, for example in case of [LowCardinality](../data-types/lowcardinality.md): [cast](#cast) removes [LowCardinality](../data-types/lowcardinality.md) trait `to` functions don't. The same with [Nullable](../data-types/nullable.md), this behaviour is not compatible with SQL standard, and it can be changed using [cast_keep_nullable](../../operations/settings/settings.md/#cast_keep_nullable) setting. :::note Be aware of potential data loss if values of a datatype are converted to a smaller datatype (for example from `Int64` to `Int32`) or between @@ -996,7 +996,7 @@ Result: ## reinterpretAsUInt8 -Performs byte reinterpretation by treating the input value as a value of type UInt8. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt8. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1034,7 +1034,7 @@ Result: ## reinterpretAsUInt16 -Performs byte reinterpretation by treating the input value as a value of type UInt16. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt16. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1072,7 +1072,7 @@ Result: ## reinterpretAsUInt32 -Performs byte reinterpretation by treating the input value as a value of type UInt32. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt32. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1110,7 +1110,7 @@ Result: ## reinterpretAsUInt64 -Performs byte reinterpretation by treating the input value as a value of type UInt64. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt64. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1148,7 +1148,7 @@ Result: ## reinterpretAsUInt128 -Performs byte reinterpretation by treating the input value as a value of type UInt128. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt128. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1186,7 +1186,7 @@ Result: ## reinterpretAsUInt256 -Performs byte reinterpretation by treating the input value as a value of type UInt256. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt256. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1224,7 +1224,7 @@ Result: ## reinterpretAsInt8 -Performs byte reinterpretation by treating the input value as a value of type Int8. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int8. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1262,7 +1262,7 @@ Result: ## reinterpretAsInt16 -Performs byte reinterpretation by treating the input value as a value of type Int16. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int16. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1300,7 +1300,7 @@ Result: ## reinterpretAsInt32 -Performs byte reinterpretation by treating the input value as a value of type Int32. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int32. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1338,7 +1338,7 @@ Result: ## reinterpretAsInt64 -Performs byte reinterpretation by treating the input value as a value of type Int64. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int64. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1376,7 +1376,7 @@ Result: ## reinterpretAsInt128 -Performs byte reinterpretation by treating the input value as a value of type Int128. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int128. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1414,7 +1414,7 @@ Result: ## reinterpretAsInt256 -Performs byte reinterpretation by treating the input value as a value of type Int256. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int256. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1452,7 +1452,7 @@ Result: ## reinterpretAsFloat32 -Performs byte reinterpretation by treating the input value as a value of type Float32. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Float32. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1486,7 +1486,7 @@ Result: ## reinterpretAsFloat64 -Performs byte reinterpretation by treating the input value as a value of type Float64. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Float64. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 5c83b2363e0..ed75b1802d8 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -235,7 +235,7 @@ If `some_predicate` is not selective enough, it will return a large amount of da ### Distributed Subqueries and max_parallel_replicas -When [max_parallel_replicas](#settings-max_parallel_replicas) is greater than 1, distributed queries are further transformed. +When [max_parallel_replicas](#distributed-subqueries-and-max_parallel_replicas) is greater than 1, distributed queries are further transformed. For example, the following: diff --git a/docs/en/sql-reference/table-functions/fileCluster.md b/docs/en/sql-reference/table-functions/fileCluster.md index 3060e6c151d..62b00fadd62 100644 --- a/docs/en/sql-reference/table-functions/fileCluster.md +++ b/docs/en/sql-reference/table-functions/fileCluster.md @@ -22,7 +22,7 @@ fileCluster(cluster_name, path[, format, structure, compression_method]) **Arguments** - `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. -- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file also supports [globs](#globs_in_path). +- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file also supports [globs](#globs-in-path). - `format` — [Format](../../interfaces/formats.md#formats) of the files. Type: [String](../../sql-reference/data-types/string.md). - `structure` — Table structure in `'UserID UInt64, Name String'` format. Determines column names and types. Type: [String](../../sql-reference/data-types/string.md). - `compression_method` — Compression method. Supported compression types are `gz`, `br`, `xz`, `zst`, `lz4`, and `bz2`. From d92a2f3ef6c86c00953a8694294859c70d4a4732 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Jun 2024 13:20:50 +0000 Subject: [PATCH 587/856] Remove obsolete fix from aws submodule --- contrib/aws | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index deeaa9e7c5f..1c2946bfcb7 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit deeaa9e7c5fe690e3dacc4005d7ecfa7a66a32bb +Subproject commit 1c2946bfcb7f1e3ae0a858de0b59d4f1a7b4ccaf From ef1188d13db47d39ef7b22f69e67d6db6f8f5702 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 12 Jun 2024 15:31:42 +0200 Subject: [PATCH 588/856] Fix tests which are failing due to new setting --- tests/queries/0_stateless/00098_k_union_all.sql | 1 + tests/queries/0_stateless/00405_output_format_pretty_color.sql | 1 + tests/queries/0_stateless/00405_pretty_formats.sql | 1 + tests/queries/0_stateless/00476_pretty_formats_and_widths.sql | 2 +- tests/queries/0_stateless/00569_parse_date_time_best_effort.sql | 1 + tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql | 1 + .../0_stateless/00813_parse_date_time_best_effort_more.sql | 1 + tests/queries/0_stateless/01293_pretty_max_value_width.sql | 1 + .../0_stateless/01351_parse_date_time_best_effort_us.sql | 1 + .../0_stateless/01509_output_format_pretty_row_numbers.sql | 1 + tests/queries/0_stateless/01553_settings_early_apply.sql | 1 + tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql | 1 + .../0_stateless/02798_explain_settings_not_applied_bug.sql | 1 + ...2998_pretty_format_print_readable_number_on_single_value.sql | 1 + tests/queries/0_stateless/03022_highlight_digit_groups.sql | 1 + tests/queries/0_stateless/03156_nullable_number_tips.sql | 1 + tests/queries/0_stateless/03160_pretty_format_tty.sh | 2 +- 17 files changed, 17 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00098_k_union_all.sql b/tests/queries/0_stateless/00098_k_union_all.sql index 059d27075d7..280ad5ca2e6 100644 --- a/tests/queries/0_stateless/00098_k_union_all.sql +++ b/tests/queries/0_stateless/00098_k_union_all.sql @@ -1,4 +1,5 @@ SET output_format_pretty_color=1; +SET output_format_pretty_display_footer_column_names=0; SELECT 1 FORMAT PrettySpace; SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace; SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace; diff --git a/tests/queries/0_stateless/00405_output_format_pretty_color.sql b/tests/queries/0_stateless/00405_output_format_pretty_color.sql index bc2d0c3adbf..de83567dd5d 100644 --- a/tests/queries/0_stateless/00405_output_format_pretty_color.sql +++ b/tests/queries/0_stateless/00405_output_format_pretty_color.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_color = 0; SHOW SETTING output_format_pretty_color; diff --git a/tests/queries/0_stateless/00405_pretty_formats.sql b/tests/queries/0_stateless/00405_pretty_formats.sql index 00bb09a1c30..4715f4fc110 100644 --- a/tests/queries/0_stateless/00405_pretty_formats.sql +++ b/tests/queries/0_stateless/00405_pretty_formats.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_color = 1; SELECT number AS hello, toString(number) AS world, (hello, world) AS tuple, nullIf(hello % 3, 0) AS sometimes_nulls FROM system.numbers LIMIT 10 SETTINGS max_block_size = 5 FORMAT Pretty; diff --git a/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql b/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql index 4bace207fb5..647cf5f20ec 100644 --- a/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql +++ b/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql @@ -1,4 +1,4 @@ -SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0; +SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0, SET output_format_pretty_display_footer_column_names=0; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT Pretty; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompact; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace; diff --git a/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql b/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql index 511addb4e4d..ca423c1922a 100644 --- a/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql +++ b/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT s, parseDateTimeBestEffortOrNull(s, 'UTC') AS a, diff --git a/tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql b/tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql index 9a421361b71..24dd9336a88 100644 --- a/tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql +++ b/tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql @@ -1,2 +1,3 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_max_column_pad_width = 250; SELECT range(number) FROM system.numbers LIMIT 100 FORMAT PrettyCompactNoEscapes; diff --git a/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql b/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql index 1e3b24e60c0..7c2b75cdd70 100644 --- a/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql +++ b/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT s, parseDateTimeBestEffortOrNull(s, 'UTC') AS a, diff --git a/tests/queries/0_stateless/01293_pretty_max_value_width.sql b/tests/queries/0_stateless/01293_pretty_max_value_width.sql index a8e0f19f58e..f1dc0cd1912 100644 --- a/tests/queries/0_stateless/01293_pretty_max_value_width.sql +++ b/tests/queries/0_stateless/01293_pretty_max_value_width.sql @@ -1,4 +1,5 @@ SET output_format_pretty_color = 1, output_format_pretty_max_value_width_apply_for_single_value = 1, output_format_pretty_row_numbers = 0; +SET output_format_pretty_display_footer_column_names=0; SELECT 'привет' AS x, 'мир' AS y FORMAT Pretty; SET output_format_pretty_max_value_width = 5; diff --git a/tests/queries/0_stateless/01351_parse_date_time_best_effort_us.sql b/tests/queries/0_stateless/01351_parse_date_time_best_effort_us.sql index ca3166142f0..f83d5d96384 100644 --- a/tests/queries/0_stateless/01351_parse_date_time_best_effort_us.sql +++ b/tests/queries/0_stateless/01351_parse_date_time_best_effort_us.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT 'parseDateTimeBestEffortUS'; SELECT diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql index 0a09aef7fb2..04d02b8d389 100644 --- a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql @@ -1,5 +1,6 @@ SET output_format_pretty_color=1; SET output_format_pretty_row_numbers=0; +SET output_format_pretty_display_footer_column_names=0; SELECT * FROM numbers(10) FORMAT Pretty; SELECT * FROM numbers(10) FORMAT PrettyCompact; SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock; diff --git a/tests/queries/0_stateless/01553_settings_early_apply.sql b/tests/queries/0_stateless/01553_settings_early_apply.sql index 4c168bdb3a5..821e09f6651 100644 --- a/tests/queries/0_stateless/01553_settings_early_apply.sql +++ b/tests/queries/0_stateless/01553_settings_early_apply.sql @@ -1,3 +1,4 @@ +set output_format_pretty_display_footer_column_names=0; set output_format_write_statistics=0; select * from numbers(100) settings max_result_rows = 1; -- { serverError TOO_MANY_ROWS_OR_BYTES } diff --git a/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql b/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql index 21dc7b1a990..510ed4b126f 100644 --- a/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql +++ b/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT 'parseDateTime64BestEffortUS'; SELECT diff --git a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql index 5b9976714ea..b0c08134816 100644 --- a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql +++ b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_color=1; SET read_in_order_two_level_merge_threshold=1000000; diff --git a/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.sql b/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.sql index 5dc69488cea..46d6bb657c9 100644 --- a/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.sql +++ b/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT 1_000_000 as a FORMAT Pretty; SELECT 1_000_000 as a FORMAT PrettyNoEscapes; SELECT 1_000_000 as a FORMAT PrettyMonoBlock; diff --git a/tests/queries/0_stateless/03022_highlight_digit_groups.sql b/tests/queries/0_stateless/03022_highlight_digit_groups.sql index c48a02e712f..8c371c409c4 100644 --- a/tests/queries/0_stateless/03022_highlight_digit_groups.sql +++ b/tests/queries/0_stateless/03022_highlight_digit_groups.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_row_numbers = 0; SELECT exp10(number) * (number % 2 ? 1 : -1) FROM numbers(30) FORMAT PrettySpace SETTINGS output_format_pretty_color = 1; diff --git a/tests/queries/0_stateless/03156_nullable_number_tips.sql b/tests/queries/0_stateless/03156_nullable_number_tips.sql index e6f2fa36d86..9a494e3292b 100644 --- a/tests/queries/0_stateless/03156_nullable_number_tips.sql +++ b/tests/queries/0_stateless/03156_nullable_number_tips.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT 123456789 AS x FORMAT PrettyCompact; SELECT toNullable(123456789) AS x FORMAT PrettyCompact; SELECT toLowCardinality(toNullable(123456789)) AS x FORMAT PrettyCompact; diff --git a/tests/queries/0_stateless/03160_pretty_format_tty.sh b/tests/queries/0_stateless/03160_pretty_format_tty.sh index bbc4b96eb90..200bd52f3fa 100755 --- a/tests/queries/0_stateless/03160_pretty_format_tty.sh +++ b/tests/queries/0_stateless/03160_pretty_format_tty.sh @@ -5,4 +5,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # default output_format_pretty_max_rows is 10K -$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1" | wc -l +$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1, output_format_pretty_display_footer_column_names=0" | wc -l From 3ff69a9a030f829bef7582e95618f63ec73b897f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Jun 2024 15:36:15 +0200 Subject: [PATCH 589/856] Update utils/check-style/check-style Co-authored-by: Mikhail f. Shiryaev --- utils/check-style/check-style | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 4c6a4b9ea39..f8c6d6b0fde 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -245,7 +245,12 @@ find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | gre # All submodules should be of this form: [submodule "contrib/libxyz"] (for consistency, the submodule name does matter too much) # - restrict the check to top-level .gitmodules file -find $ROOT_PATH -maxdepth 1 -name '.gitmodules' | while read i; do grep -F '[submodule ' $i | grep -v -F 'contrib' && echo 'All submodules should have form [submodule "contrib/libxyz"]'; done +git config --file "$ROOT_PATH/.gitmodules" --get-regexp 'submodule\..+\.path' | \ +while read -r line; do + name=${line#submodule.}; name=${name%.path*} + path=${line#* } + [ "$name" != "$path" ] && echo "Submodule name '$name' is not equal to it's path '$path'" +done # There shouldn't be any code snippets under GPL or LGPL find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL" From 384c7339415340a8238dfc50cc14336bbd148a3f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 12 Jun 2024 15:40:06 +0200 Subject: [PATCH 590/856] Make QueryAnalyzer respect rewrite_count_distinct_if_with_count_distinct_implementation setting --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 07328383700..68337aeae4c 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -985,15 +985,15 @@ std::string QueryAnalyzer::rewriteAggregateFunctionNameIfNeeded( { result_aggregate_function_name = settings.count_distinct_implementation; } - else if (aggregate_function_name_lowercase == "countdistinctif" || aggregate_function_name_lowercase == "countifdistinct") + else if (settings.rewrite_count_distinct_if_with_count_distinct_implementation && + (aggregate_function_name_lowercase == "countdistinctif" || aggregate_function_name_lowercase == "countifdistinct")) { result_aggregate_function_name = settings.count_distinct_implementation; result_aggregate_function_name += "If"; } - - /// Replace aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal - if (aggregate_function_name_lowercase.ends_with("ifdistinct")) + else if (aggregate_function_name_lowercase.ends_with("ifdistinct")) { + /// Replace aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal size_t prefix_length = result_aggregate_function_name.size() - strlen("ifdistinct"); result_aggregate_function_name = result_aggregate_function_name.substr(0, prefix_length) + "DistinctIf"; } From 6a670645b2213cecee1e17e851463591dff3556f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 12 Jun 2024 15:48:33 +0200 Subject: [PATCH 591/856] 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 9d1f64e8b302d9d069c6218046788a6b236c98c2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 12 Jun 2024 16:11:21 +0200 Subject: [PATCH 592/856] Update utils/check-style/check-style --- utils/check-style/check-style | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index f8c6d6b0fde..21325ece916 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -241,7 +241,13 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do done # All the submodules should be from https://github.com/ -find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | grep -v -F 'https://github.com/' && echo 'All the submodules should be from https://github.com/'; done + +git config --file "$ROOT_PATH/.gitmodules" --get-regexp 'submodule\..+\.url' | \ +while read -r line; do + name=${line#submodule.}; name=${name%.url*} + url=${line#* } + [[ "$url" != 'https://github.com/'* ]] && echo "All the submodules should be from https://github.com/, submodule '$name' has '$url'" +done # All submodules should be of this form: [submodule "contrib/libxyz"] (for consistency, the submodule name does matter too much) # - restrict the check to top-level .gitmodules file From 3fd1918c3f6f92aeb14119a461cde2ed4038cedd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Jun 2024 16:21:08 +0200 Subject: [PATCH 593/856] Update check-style --- utils/check-style/check-style | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 21325ece916..db491c67f2c 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -240,13 +240,12 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do esac done -# All the submodules should be from https://github.com/ - +# All submodules should be from https://github.com/ git config --file "$ROOT_PATH/.gitmodules" --get-regexp 'submodule\..+\.url' | \ while read -r line; do name=${line#submodule.}; name=${name%.url*} url=${line#* } - [[ "$url" != 'https://github.com/'* ]] && echo "All the submodules should be from https://github.com/, submodule '$name' has '$url'" + [[ "$url" != 'https://github.com/'* ]] && echo "All submodules should be from https://github.com/, submodule '$name' has '$url'" done # All submodules should be of this form: [submodule "contrib/libxyz"] (for consistency, the submodule name does matter too much) From 9a4b970f22b5704aee8d816a2a543782c891ce4d Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 12 Jun 2024 16:21:49 +0200 Subject: [PATCH 594/856] Fix other failing tests --- tests/queries/0_stateless/00476_pretty_formats_and_widths.sql | 2 +- tests/queries/0_stateless/02375_pretty_formats.sql.j2 | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql b/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql index 647cf5f20ec..59809841726 100644 --- a/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql +++ b/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql @@ -1,4 +1,4 @@ -SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0, SET output_format_pretty_display_footer_column_names=0; +SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0, output_format_pretty_display_footer_column_names=0; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT Pretty; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompact; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace; diff --git a/tests/queries/0_stateless/02375_pretty_formats.sql.j2 b/tests/queries/0_stateless/02375_pretty_formats.sql.j2 index 55462ea6b61..36a4479260c 100644 --- a/tests/queries/0_stateless/02375_pretty_formats.sql.j2 +++ b/tests/queries/0_stateless/02375_pretty_formats.sql.j2 @@ -3,6 +3,6 @@ 'PrettySpaceNoEscapesMonoBlock'] -%} select '{{ format }}'; -select number as x, number + 1 as y from numbers(4) settings max_block_size=2, output_format_pretty_color=1 format {{ format }}; +select number as x, number + 1 as y from numbers(4) settings max_block_size=2, output_format_pretty_color=1, output_format_pretty_display_footer_column_names=0 format {{ format }}; {% endfor -%} From 49977e89e01afb8612332a4cd79fb89e95934921 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 12 Jun 2024 14:29:52 +0000 Subject: [PATCH 595/856] CI: Fix not-merged cherry-picks for backports --- tests/ci/cherry_pick.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 629464d0422..459be12ada0 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -532,9 +532,9 @@ class Backport: for br in branches: br.process(self.dry_run) - for br in branches: - if br.backported: - self.mark_pr_backported(pr) + if all(br.backported for br in branches): + # And check it after the running + self.mark_pr_backported(pr) def mark_pr_backported(self, pr: PullRequest) -> None: if self.dry_run: From d798fffb1d772266095f599173212874a3897ab1 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 12 Jun 2024 16:46:14 +0200 Subject: [PATCH 596/856] fix --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 0dbddf05697..8e304936747 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -471,13 +471,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( ColumnsStatistics statistics; if (context->getSettingsRef().materialize_statistics_on_insert) - { - for (auto col : metadata_snapshot->getColumns()) - LOG_INFO(log, "column col {} stats {}", col.name, col.statistics.column_name); statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()); - for (auto stats : statistics) - LOG_INFO(log, "writing stats {}", stats->columnName()); - } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) From 208f32328c8a91c17e1b4bb8d93b0b3343b054a1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 12 Jun 2024 16:56:09 +0200 Subject: [PATCH 597/856] Update tests/integration/test_replicated_database/test.py MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Raúl Marín --- tests/integration/test_replicated_database/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 73b7ae265e4..f23384b5c04 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -404,7 +404,8 @@ def test_alter_detach_part(started_cluster, engine): main_node.query(f"INSERT INTO {database}.alter_detach VALUES (123)") if engine == "MergeTree": dummy_node.query(f"INSERT INTO {database}.alter_detach VALUES (456)") - main_node.query(f"SYSTEM SYNC REPLICA {database}.alter_detach PULL") + else: + main_node.query(f"SYSTEM SYNC REPLICA {database}.alter_detach PULL") main_node.query(f"ALTER TABLE {database}.alter_detach DETACH PART '{part_name}'") detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_detach'" assert main_node.query(detached_parts_query) == f"{part_name}\n" From b8bb1675476276da200ee7c971749a0b75a9d7b6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Jun 2024 17:27:17 +0200 Subject: [PATCH 598/856] Update .gitmodules --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index a3bb7eacabc..12d865307d8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -161,7 +161,7 @@ [submodule "contrib/xz"] path = contrib/xz url = https://github.com/xz-mirror/xz -[submodule "contrib/abseil"] +[submodule "contrib/abseil-cpp"] path = contrib/abseil-cpp url = https://github.com/ClickHouse/abseil-cpp.git [submodule "contrib/dragonbox"] From 5f5a6d5f107d09e22fb1466aeef1aec395eb4b6f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 12 Jun 2024 17:51:17 +0200 Subject: [PATCH 599/856] Fix bug in short circuit optimization with cache dictionaries --- src/Dictionaries/CacheDictionary.cpp | 3 ++ ...e_complex_dict_short_circuit_bug.reference | 0 ...9_cache_complex_dict_short_circuit_bug.sql | 31 +++++++++++++++++++ 3 files changed, 34 insertions(+) create mode 100644 tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.reference create mode 100644 tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.sql diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 2842e2b8799..1816324a93b 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -511,7 +511,10 @@ MutableColumns CacheDictionary::aggregateColumns( if (default_mask) { if (key_state_from_storage.isDefault()) + { (*default_mask)[key_index] = 1; + aggregated_column->insertDefault(); + } else { (*default_mask)[key_index] = 0; diff --git a/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.reference b/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.sql b/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.sql new file mode 100644 index 00000000000..8463d13d251 --- /dev/null +++ b/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS complex_key_simple_attributes_source_short_circuit_table; +DROP DICTIONARY IF EXISTS cache_dictionary_complex_key_simple_attributes_short_circuit; + +CREATE TABLE complex_key_simple_attributes_source_short_circuit_table +( + id UInt64, + id_key String, + value_first String, + value_second String +) + ENGINE = TinyLog; + +INSERT INTO complex_key_simple_attributes_source_short_circuit_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); + +CREATE DICTIONARY cache_dictionary_complex_key_simple_attributes_short_circuit +( + `id` UInt64, + `id_key` String, + `value_first` String DEFAULT 'value_first_default', + `value_second` String DEFAULT 'value_second_default' +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(TABLE 'complex_key_simple_attributes_source_short_circuit_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10)); + +SELECT dictGetOrDefault('cache_dictionary_complex_key_simple_attributes_short_circuit', 'value_first', (number, concat(toString(number))), toString(materialize('default'))) AS value_first FROM system.numbers LIMIT 20 FORMAT Null; +SELECT dictGetOrDefault('cache_dictionary_complex_key_simple_attributes_short_circuit', 'value_first', (number, concat(toString(number))), toString(materialize('default'))) AS value_first FROM system.numbers LIMIT 20 FORMAT Null; + +DROP TABLE IF EXISTS complex_key_simple_attributes_source_short_circuit_table; +DROP DICTIONARY IF EXISTS cache_dictionary_complex_key_simple_attributes_short_circuit; From bf04aebc948b078ba2f00dcb601e89158367422c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 12 Jun 2024 18:00:19 +0200 Subject: [PATCH 600/856] Update ReplicatedMergeTreeQueue.cpp --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 9a368bd44f5..e30d63c343a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2004,8 +2004,7 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( MutationCommands commands; for (auto it = begin; it != end; ++it) { - /// FIXME uncomment this assertion after relesing 23.5 (currently it fails in Upgrade check) - /// chassert(mutation_pointer < it->second->entry->znode_name); + chassert(mutation_pointer < it->second->entry->znode_name); mutation_ids.push_back(it->second->entry->znode_name); const auto & commands_from_entry = it->second->entry->commands; commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); From d81c0826c72b3a4cac486779e493a9f0dd328222 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 12 Jun 2024 18:16:09 +0200 Subject: [PATCH 601/856] Fix incorrect condition in PrettySpaceBlockOutputFormat --- src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 40df517fec8..4a07d0f808c 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -104,10 +104,10 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port } /// Write blank line between last row and footer - if (format_settings.pretty.output_format_pretty_display_footer_column_names) + if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names) writeCString("\n", out); /// Write left blank - if (format_settings.pretty.output_format_pretty_row_numbers && format_settings.pretty.output_format_pretty_display_footer_column_names) + if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_row_numbers && format_settings.pretty.output_format_pretty_display_footer_column_names) writeString(String(row_number_width, ' '), out); /// Write footer if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names) From ae42f8635cbf4d18c10272a5c3e9ff74df60eb0e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 12 Jun 2024 16:53:32 +0000 Subject: [PATCH 602/856] 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 b7161b77d177680187b489b971f35ab856a4004b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 12 Jun 2024 19:08:23 +0200 Subject: [PATCH 603/856] Fix UniqInjectiveFunctionsEliminationPass with uniqCombined --- .../UniqInjectiveFunctionsEliminationPass.cpp | 14 +++++++------ ...tive_functions_inside_uniq_crash.reference | 2 ++ ..._injective_functions_inside_uniq_crash.sql | 21 +++++++++++++++++++ 3 files changed, 31 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.reference create mode 100644 tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp index d087fe1c7b9..2360cd3f0c2 100644 --- a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp @@ -41,9 +41,9 @@ public: return; bool replaced_argument = false; - auto & uniq_function_arguments_nodes = function_node->getArguments().getNodes(); + auto replaced_uniq_function_arguments_nodes = function_node->getArguments().getNodes(); - for (auto & uniq_function_argument_node : uniq_function_arguments_nodes) + for (auto & uniq_function_argument_node : replaced_uniq_function_arguments_nodes) { auto * uniq_function_argument_node_typed = uniq_function_argument_node->as(); if (!uniq_function_argument_node_typed || !uniq_function_argument_node_typed->isOrdinaryFunction()) @@ -67,12 +67,10 @@ public: if (!replaced_argument) return; - const auto & function_node_argument_nodes = function_node->getArguments().getNodes(); - DataTypes argument_types; - argument_types.reserve(function_node_argument_nodes.size()); + argument_types.reserve(replaced_uniq_function_arguments_nodes.size()); - for (const auto & function_node_argument : function_node_argument_nodes) + for (const auto & function_node_argument : replaced_uniq_function_arguments_nodes) argument_types.emplace_back(function_node_argument->getResultType()); AggregateFunctionProperties properties; @@ -83,6 +81,10 @@ public: function_node->getAggregateFunction()->getParameters(), properties); + /// uniqCombined returns nullable with nullable arguments so the result type might change which breaks the pass + if (!aggregate_function->getResultType()->equals(*function_node->getAggregateFunction()->getResultType())) + return; + function_node->resolveAsAggregateFunction(std::move(aggregate_function)); } }; diff --git a/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.reference b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.reference new file mode 100644 index 00000000000..e58e9764b39 --- /dev/null +++ b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.reference @@ -0,0 +1,2 @@ +100 +100 diff --git a/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql new file mode 100644 index 00000000000..50d99b851a6 --- /dev/null +++ b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql @@ -0,0 +1,21 @@ +SELECT sum(u) +FROM +( + SELECT + intDiv(number, 4096) AS k, + uniqCombined(tuple(materialize(toLowCardinality(toNullable(16))))) AS u + FROM numbers(4096 * 100) + GROUP BY k +) +SETTINGS allow_experimental_analyzer = 1, optimize_injective_functions_inside_uniq=0; + +SELECT sum(u) +FROM +( + SELECT + intDiv(number, 4096) AS k, + uniqCombined(tuple(materialize(toLowCardinality(toNullable(16))))) AS u + FROM numbers(4096 * 100) + GROUP BY k +) +SETTINGS allow_experimental_analyzer = 1, optimize_injective_functions_inside_uniq=1; From 35252c4eda20d6dfcb878d7b1522e7842d826f60 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 12 Jun 2024 18:41:46 +0100 Subject: [PATCH 604/856] impl --- .../0_stateless/03168_loop_engine_with_parallel_replicas.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql index dfcb5de9f2a..da4626ad897 100644 --- a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql +++ b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP DATABASE IF EXISTS 03147_db; CREATE DATABASE IF NOT EXISTS 03147_db; CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n; From 6d48962ca09f582478e6caa160463b34ead0091b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 10 Jun 2024 23:54:19 +0100 Subject: [PATCH 605/856] impl --- .../QueryPlan/ReadFromMergeTree.cpp | 9 ++++ src/Storages/StorageMerge.cpp | 6 +++ .../03155_test_move_to_prewhere.reference | 1 + .../03155_test_move_to_prewhere.sh | 46 +++++++++++++++++++ 4 files changed, 62 insertions(+) create mode 100644 tests/queries/0_stateless/03155_test_move_to_prewhere.reference create mode 100755 tests/queries/0_stateless/03155_test_move_to_prewhere.sh diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e469062d7e7..4fad1bbb653 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1515,6 +1515,15 @@ static void buildIndexes( void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) { + /// Sometimes a really dumb problem may happen. + /// For ReadFromMerge for example we may recursively call `applyFilters` for child reading steps (with no filters added so far). + /// Then later `optimizePrimaryKeyCondition` will try to apply filters to those child reading steps, but with no luck, + /// because we already made an `applyFilters` call that could lead to indexes initialization few lines below. + /// So effectively the right set of filters will be just ignored. + /// This is not an ultimate solution, of course, we're better to have more structured way of applying filters. + if (added_filter_nodes.nodes.empty()) + return; + if (!indexes) { filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index ed3f43367dd..55cfd1ffcd7 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1573,8 +1573,14 @@ QueryPlanRawPtrs ReadFromMerge::getChildPlans() QueryPlanRawPtrs plans; for (auto & child_plan : *child_plans) + { if (child_plan.plan.isInitialized()) + { + /// So we will see the optimized plan in EXPLAIN output + child_plan.plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); plans.push_back(&child_plan.plan); + } + } return plans; } diff --git a/tests/queries/0_stateless/03155_test_move_to_prewhere.reference b/tests/queries/0_stateless/03155_test_move_to_prewhere.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03155_test_move_to_prewhere.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03155_test_move_to_prewhere.sh b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh new file mode 100755 index 00000000000..b6980b3a23a --- /dev/null +++ b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -nq " + CREATE TABLE event_envoy + ( + timestamp_interval DateTime CODEC(DoubleDelta), + region LowCardinality(String), + cluster LowCardinality(String) + ) + ENGINE = MergeTree + ORDER BY (timestamp_interval) + SETTINGS index_granularity = 8192; + + INSERT INTO event_envoy SELECT now() - number, 'us-east-1', 'ch_super_fast' FROM numbers_mt(1e5); +" + +${CLICKHOUSE_CLIENT} -nq " + CREATE TABLE event_envoy_remote + ( + timestamp_interval DateTime CODEC(DoubleDelta), + region LowCardinality(String), + cluster LowCardinality(String) + ) AS remote('127.0.0.1', '${CLICKHOUSE_DATABASE}', event_envoy); +" + +${CLICKHOUSE_CLIENT} -q " + CREATE TABLE global_event_envoy + ( + timestamp_interval DateTime, + region LowCardinality(String), + cluster LowCardinality(String) + ) + ENGINE = Merge('${CLICKHOUSE_DATABASE}', 'event_envoy.*'); +" + +${CLICKHOUSE_CLIENT} --prefer_localhost_replica 1 -q " + EXPLAIN indexes=1 + SELECT timestamp_interval + FROM global_event_envoy + WHERE timestamp_interval <= now() - 54321 AND region = 'us-east-1' +" | grep -c 'Condition.*timestamp_interval' + From 8a49c1614e9bef2859b405408597957c73eb06bf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 12 Jun 2024 21:45:00 +0200 Subject: [PATCH 606/856] fix --- src/Core/ServerUUID.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Core/ServerUUID.cpp b/src/Core/ServerUUID.cpp index 159aa8faadf..9dfaf4fecf2 100644 --- a/src/Core/ServerUUID.cpp +++ b/src/Core/ServerUUID.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -16,7 +17,9 @@ namespace ErrorCodes UUID ServerUUID::get() { - if (server_uuid == UUIDHelpers::Nil) + if (server_uuid == UUIDHelpers::Nil && + (Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::SERVER || + Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::KEEPER)) throw Exception(ErrorCodes::LOGICAL_ERROR, "ServerUUID is not initialized yet"); return server_uuid; } From 55e0c668ae58ada0071916853aa07952c51fa15d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 12 Jun 2024 20:49:26 +0100 Subject: [PATCH 607/856] fix --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 9 --------- src/Storages/StorageMerge.cpp | 6 ------ 2 files changed, 15 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 4fad1bbb653..e469062d7e7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1515,15 +1515,6 @@ static void buildIndexes( void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) { - /// Sometimes a really dumb problem may happen. - /// For ReadFromMerge for example we may recursively call `applyFilters` for child reading steps (with no filters added so far). - /// Then later `optimizePrimaryKeyCondition` will try to apply filters to those child reading steps, but with no luck, - /// because we already made an `applyFilters` call that could lead to indexes initialization few lines below. - /// So effectively the right set of filters will be just ignored. - /// This is not an ultimate solution, of course, we're better to have more structured way of applying filters. - if (added_filter_nodes.nodes.empty()) - return; - if (!indexes) { filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 55cfd1ffcd7..ed3f43367dd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1573,14 +1573,8 @@ QueryPlanRawPtrs ReadFromMerge::getChildPlans() QueryPlanRawPtrs plans; for (auto & child_plan : *child_plans) - { if (child_plan.plan.isInitialized()) - { - /// So we will see the optimized plan in EXPLAIN output - child_plan.plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); plans.push_back(&child_plan.plan); - } - } return plans; } From f030b220272e040f6d56048eef0e4de2d1ffd2c5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 13 Jun 2024 00:05:46 +0200 Subject: [PATCH 608/856] fix build --- src/Coordination/Standalone/Context.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index 79a3e32a72d..38b810e4bfc 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -130,7 +130,8 @@ public: enum class ApplicationType : uint8_t { - KEEPER + KEEPER, + SERVER, }; void setApplicationType(ApplicationType) {} From 857a412e3b274bb5a309bebe9bfd284ae5ac8ad8 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 13 Jun 2024 01:27:54 +0000 Subject: [PATCH 609/856] address some review comments Signed-off-by: Duc Canh Le --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 4 ++-- src/Storages/Kafka/KafkaConsumer.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 56c774782c2..6a3475a1830 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -421,7 +421,7 @@ void BSONEachRowRowInputFormat::readTuple(IColumn & column, const DataTypePtr & "Cannot parse tuple column with type {} from BSON array/embedded document field: " "tuple doesn't have element with name \"{}\"", data_type->getName(), - name.toView()); + name); index = *try_get_index; } @@ -806,7 +806,7 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi else { if (seen_columns[index]) - throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name.toView()); + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name); seen_columns[index] = true; read_columns[index] = readField(*columns[index], types[index], BSONType(type)); diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index a2d047933be..4daf8652c3b 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -1,14 +1,14 @@ #pragma once -#include -#include #include +#include #include #include #include #include +#include #include namespace CurrentMetrics From 07f93fe78d946c66df2dbe6e44efa0c971a853e5 Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 13 Jun 2024 10:44:32 +0800 Subject: [PATCH 610/856] Fix docs --- docs/zh/guides/improving-query-performance/skipping-indexes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/guides/improving-query-performance/skipping-indexes.md b/docs/zh/guides/improving-query-performance/skipping-indexes.md index f9f43e46927..8eb88d859f2 100644 --- a/docs/zh/guides/improving-query-performance/skipping-indexes.md +++ b/docs/zh/guides/improving-query-performance/skipping-indexes.md @@ -123,7 +123,7 @@ Bloom filter是一种数据结构,它允许对集合成员进行高效的是 有三种基于Bloom过滤器的数据跳数索引类型: -* 基本的**bloom_filter**接受一个可选参数,该参数表示在0到1之间允许的“假阳性”率(如果未指定,则使用.025)。 +* 基本的**bloom_filter**接受一个可选参数,该参数表示在0到1之间允许的“假阳性”率(如果未指定,则使用0.025)。 * 更专业的**tokenbf_v1**。需要三个参数,用来优化布隆过滤器:(1)过滤器的大小字节(大过滤器有更少的假阳性,有更高的存储成本),(2)哈希函数的个数(更多的散列函数可以减少假阳性)。(3)布隆过滤器哈希函数的种子。有关这些参数如何影响布隆过滤器功能的更多细节,请参阅 [这里](https://hur.st/bloomfilter/) 。此索引仅适用于String、FixedString和Map类型的数据。输入表达式被分割为由非字母数字字符分隔的字符序列。例如,列值`This is a candidate for a "full text" search`将被分割为`This` `is` `a` `candidate` `for` `full` `text` `search`。它用于LIKE、EQUALS、in、hasToken()和类似的长字符串中单词和其他值的搜索。例如,一种可能的用途是在非结构的应用程序日志行列中搜索少量的类名或行号。 From 2c07a8a111012968af79f60cb6db25e259ee5757 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 13 Jun 2024 08:58:08 +0200 Subject: [PATCH 611/856] Update reference file for 03169 --- .../00305_http_and_readonly_2.reference | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/00305_http_and_readonly_2.reference diff --git a/tests/queries/0_stateless/00305_http_and_readonly_2.reference b/tests/queries/0_stateless/00305_http_and_readonly_2.reference new file mode 100644 index 00000000000..baddaa07bca --- /dev/null +++ b/tests/queries/0_stateless/00305_http_and_readonly_2.reference @@ -0,0 +1,26 @@ + name value changed + +1. max_rows_to_read 10000 1 +2. readonly 0 0 + name value changed + +1. max_rows_to_read 10000 1 +2. readonly 2 1 + name value changed + +1. max_rows_to_read 10000 1 +2. readonly 1 1 + name value changed + +1. max_rows_to_read 10000 1 +2. readonly 2 1 +Ok +Ok +0 +0 +Ok +Ok +Ok +0 +0 +0 From 47686e0c4ae633f822ffc0926e5380130ecc1743 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Thu, 9 May 2024 14:27:43 +0300 Subject: [PATCH 612/856] [feature] A setting `http_response_headers` Implementing https://github.com/ClickHouse/ClickHouse/issues/59620 Deprecating `content_type` setting (still supported). --- src/Server/HTTPHandler.cpp | 530 +++++++++++++++++++------------------ src/Server/HTTPHandler.h | 60 +++-- 2 files changed, 308 insertions(+), 282 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 02d0959ff50..3c0bbf03986 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1,8 +1,8 @@ #include +#include #include #include -#include #include #include #include @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -17,46 +18,47 @@ #include #include #include -#include -#include -#include #include +#include +#include +#include +#include +#include #include #include #include -#include #include #include #include #include #include -#include -#include -#include -#include +#include #include #include -#include #include "config.h" #include #include -#include -#include #include +#include +#include +#include +#include #include #include -#include #include #include #include +#include #include +#include +#include #if USE_SSL -#include +# include #endif @@ -65,68 +67,68 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; - extern const int CANNOT_COMPILE_REGEXP; - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_PARSE_TEXT; - extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; - extern const int CANNOT_PARSE_QUOTED_STRING; - extern const int CANNOT_PARSE_DATE; - extern const int CANNOT_PARSE_DATETIME; - extern const int CANNOT_PARSE_NUMBER; - extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; - extern const int CANNOT_PARSE_IPV4; - extern const int CANNOT_PARSE_IPV6; - extern const int CANNOT_PARSE_UUID; - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int CANNOT_SCHEDULE_TASK; - extern const int DUPLICATE_COLUMN; - extern const int ILLEGAL_COLUMN; - extern const int THERE_IS_NO_COLUMN; - extern const int UNKNOWN_ELEMENT_IN_AST; - extern const int UNKNOWN_TYPE_OF_AST_NODE; - extern const int TOO_DEEP_AST; - extern const int TOO_BIG_AST; - extern const int UNEXPECTED_AST_STRUCTURE; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +extern const int CANNOT_SCHEDULE_TASK; +extern const int CANNOT_PARSE_TEXT; +extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; +extern const int CANNOT_PARSE_QUOTED_STRING; +extern const int CANNOT_PARSE_DATE; +extern const int CANNOT_PARSE_DATETIME; +extern const int CANNOT_PARSE_NUMBER; +extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; +extern const int CANNOT_PARSE_IPV4; +extern const int CANNOT_PARSE_IPV6; +extern const int CANNOT_PARSE_UUID; +extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; +extern const int CANNOT_OPEN_FILE; +extern const int CANNOT_COMPILE_REGEXP; +extern const int DUPLICATE_COLUMN; +extern const int ILLEGAL_COLUMN; +extern const int THERE_IS_NO_COLUMN; +extern const int UNKNOWN_ELEMENT_IN_AST; +extern const int UNKNOWN_TYPE_OF_AST_NODE; +extern const int TOO_DEEP_AST; +extern const int TOO_BIG_AST; +extern const int UNEXPECTED_AST_STRUCTURE; +extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int SYNTAX_ERROR; +extern const int SYNTAX_ERROR; - extern const int INCORRECT_DATA; - extern const int TYPE_MISMATCH; +extern const int INCORRECT_DATA; +extern const int TYPE_MISMATCH; - extern const int UNKNOWN_TABLE; - extern const int UNKNOWN_FUNCTION; - extern const int UNKNOWN_IDENTIFIER; - extern const int UNKNOWN_TYPE; - extern const int UNKNOWN_STORAGE; - extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_SETTING; - extern const int UNKNOWN_DIRECTION_OF_SORTING; - extern const int UNKNOWN_AGGREGATE_FUNCTION; - extern const int UNKNOWN_FORMAT; - extern const int UNKNOWN_DATABASE_ENGINE; - extern const int UNKNOWN_TYPE_OF_QUERY; - extern const int UNKNOWN_ROLE; - extern const int NO_ELEMENTS_IN_CONFIG; +extern const int UNKNOWN_TABLE; +extern const int UNKNOWN_FUNCTION; +extern const int UNKNOWN_IDENTIFIER; +extern const int UNKNOWN_TYPE; +extern const int UNKNOWN_STORAGE; +extern const int UNKNOWN_DATABASE; +extern const int UNKNOWN_SETTING; +extern const int UNKNOWN_DIRECTION_OF_SORTING; +extern const int UNKNOWN_AGGREGATE_FUNCTION; +extern const int UNKNOWN_FORMAT; +extern const int UNKNOWN_DATABASE_ENGINE; +extern const int UNKNOWN_TYPE_OF_QUERY; +extern const int UNKNOWN_ROLE; +extern const int NO_ELEMENTS_IN_CONFIG; - extern const int QUERY_IS_TOO_LARGE; +extern const int QUERY_IS_TOO_LARGE; - extern const int NOT_IMPLEMENTED; - extern const int SOCKET_TIMEOUT; +extern const int NOT_IMPLEMENTED; +extern const int SOCKET_TIMEOUT; - extern const int UNKNOWN_USER; - extern const int WRONG_PASSWORD; - extern const int REQUIRED_PASSWORD; - extern const int AUTHENTICATION_FAILED; - extern const int SET_NON_GRANTED_ROLE; +extern const int UNKNOWN_USER; +extern const int WRONG_PASSWORD; +extern const int REQUIRED_PASSWORD; +extern const int AUTHENTICATION_FAILED; +extern const int SET_NON_GRANTED_ROLE; - extern const int INVALID_SESSION_TIMEOUT; - extern const int HTTP_LENGTH_REQUIRED; - extern const int SUPPORT_IS_DISABLED; +extern const int INVALID_SESSION_TIMEOUT; +extern const int HTTP_LENGTH_REQUIRED; +extern const int SUPPORT_IS_DISABLED; - extern const int TIMEOUT_EXCEEDED; +extern const int TIMEOUT_EXCEEDED; } namespace @@ -145,9 +147,9 @@ bool tryAddHTTPOptionHeadersFromConfig(HTTPServerResponse & response, const Poco if (config.getString("http_options_response." + config_key + ".name", "").empty()) LOG_WARNING(getLogger("processOptionsRequest"), "Empty header was found in config. It will not be processed."); else - response.add(config.getString("http_options_response." + config_key + ".name", ""), - config.getString("http_options_response." + config_key + ".value", "")); - + response.add( + config.getString("http_options_response." + config_key + ".name", ""), + config.getString("http_options_response." + config_key + ".value", "")); } } return true; @@ -196,54 +198,37 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_UNAUTHORIZED; } - else if (exception_code == ErrorCodes::UNKNOWN_USER || - exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::AUTHENTICATION_FAILED || - exception_code == ErrorCodes::SET_NON_GRANTED_ROLE) + else if ( + exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD + || exception_code == ErrorCodes::AUTHENTICATION_FAILED || exception_code == ErrorCodes::SET_NON_GRANTED_ROLE) { return HTTPResponse::HTTP_FORBIDDEN; } - else if (exception_code == ErrorCodes::BAD_ARGUMENTS || - exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP || - exception_code == ErrorCodes::CANNOT_PARSE_TEXT || - exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || - exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || - exception_code == ErrorCodes::CANNOT_PARSE_DATE || - exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || - exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || - exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING || - exception_code == ErrorCodes::CANNOT_PARSE_IPV4 || - exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || - exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || - exception_code == ErrorCodes::CANNOT_PARSE_UUID || - exception_code == ErrorCodes::DUPLICATE_COLUMN || - exception_code == ErrorCodes::ILLEGAL_COLUMN || - exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || - exception_code == ErrorCodes::THERE_IS_NO_COLUMN || - exception_code == ErrorCodes::TOO_DEEP_AST || - exception_code == ErrorCodes::TOO_BIG_AST || - exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || - exception_code == ErrorCodes::SYNTAX_ERROR || - exception_code == ErrorCodes::INCORRECT_DATA || - exception_code == ErrorCodes::TYPE_MISMATCH || - exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) + else if ( + exception_code == ErrorCodes::BAD_ARGUMENTS || exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP + || exception_code == ErrorCodes::CANNOT_PARSE_TEXT || exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE + || exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || exception_code == ErrorCodes::CANNOT_PARSE_DATE + || exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || exception_code == ErrorCodes::CANNOT_PARSE_NUMBER + || exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING || exception_code == ErrorCodes::CANNOT_PARSE_IPV4 + || exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED + || exception_code == ErrorCodes::CANNOT_PARSE_UUID || exception_code == ErrorCodes::DUPLICATE_COLUMN + || exception_code == ErrorCodes::ILLEGAL_COLUMN || exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST + || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || exception_code == ErrorCodes::THERE_IS_NO_COLUMN + || exception_code == ErrorCodes::TOO_DEEP_AST || exception_code == ErrorCodes::TOO_BIG_AST + || exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || exception_code == ErrorCodes::SYNTAX_ERROR + || exception_code == ErrorCodes::INCORRECT_DATA || exception_code == ErrorCodes::TYPE_MISMATCH + || exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) { return HTTPResponse::HTTP_BAD_REQUEST; } - else if (exception_code == ErrorCodes::UNKNOWN_TABLE || - exception_code == ErrorCodes::UNKNOWN_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || - exception_code == ErrorCodes::UNKNOWN_TYPE || - exception_code == ErrorCodes::UNKNOWN_STORAGE || - exception_code == ErrorCodes::UNKNOWN_DATABASE || - exception_code == ErrorCodes::UNKNOWN_SETTING || - exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || - exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_FORMAT || - exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY || - exception_code == ErrorCodes::UNKNOWN_ROLE) + else if ( + exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION + || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_TYPE + || exception_code == ErrorCodes::UNKNOWN_STORAGE || exception_code == ErrorCodes::UNKNOWN_DATABASE + || exception_code == ErrorCodes::UNKNOWN_SETTING || exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING + || exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || exception_code == ErrorCodes::UNKNOWN_FORMAT + || exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY + || exception_code == ErrorCodes::UNKNOWN_ROLE) { return HTTPResponse::HTTP_NOT_FOUND; } @@ -255,8 +240,7 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_NOT_IMPLEMENTED; } - else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || - exception_code == ErrorCodes::CANNOT_OPEN_FILE) + else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || exception_code == ErrorCodes::CANNOT_OPEN_FILE) { return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; } @@ -277,9 +261,7 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti } -static std::chrono::steady_clock::duration parseSessionTimeout( - const Poco::Util::AbstractConfiguration & config, - const HTMLForm & params) +static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util::AbstractConfiguration & config, const HTMLForm & params) { unsigned session_timeout = config.getInt("default_session_timeout", 60); @@ -293,14 +275,19 @@ static std::chrono::steady_clock::duration parseSessionTimeout( throw Exception(ErrorCodes::INVALID_SESSION_TIMEOUT, "Invalid session timeout: '{}'", session_timeout_str); if (session_timeout > max_session_timeout) - throw Exception(ErrorCodes::INVALID_SESSION_TIMEOUT, "Session timeout '{}' is larger than max_session_timeout: {}. " + throw Exception( + ErrorCodes::INVALID_SESSION_TIMEOUT, + "Session timeout '{}' is larger than max_session_timeout: {}. " "Maximum session timeout could be modified in configuration file.", - session_timeout_str, max_session_timeout); + session_timeout_str, + max_session_timeout); } return std::chrono::seconds(session_timeout); } +std::optional> +parseHttpResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); void HTTPHandler::pushDelayedResults(Output & used_output) { @@ -338,11 +325,12 @@ void HTTPHandler::pushDelayedResults(Output & used_output) } -HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const std::optional & content_type_override_) +HTTPHandler::HTTPHandler( + IServer & server_, const std::string & name, const std::optional> & http_response_headers_override_) : server(server_) , log(getLogger(name)) , default_settings(server.context()->getSettingsRef()) - , content_type_override(content_type_override_) + , http_response_headers_override(http_response_headers_override_) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } @@ -353,10 +341,7 @@ HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const std: HTTPHandler::~HTTPHandler() = default; -bool HTTPHandler::authenticateUser( - HTTPServerRequest & request, - HTMLForm & params, - HTTPServerResponse & response) +bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response) { using namespace Poco::Net; @@ -383,31 +368,36 @@ bool HTTPHandler::authenticateUser( { /// It is prohibited to mix different authorization schemes. if (has_http_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and Authorization HTTP header simultaneously"); + throw Exception( + ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and Authorization HTTP header simultaneously"); if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); + throw Exception( + ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); if (has_ssl_certificate_auth) { #if USE_SSL if (!password.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via password simultaneously"); + throw Exception( + ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and authentication via password simultaneously"); if (request.havePeerCertificate()) certificate_common_name = request.peerCertificate().commonName(); if (certificate_common_name.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name"); + throw Exception( + ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name"); #else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "SSL certificate authentication disabled because ClickHouse was built without SSL library"); + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "SSL certificate authentication disabled because ClickHouse was built without SSL library"); #endif } } @@ -415,9 +405,10 @@ bool HTTPHandler::authenticateUser( { /// It is prohibited to mix different authorization schemes. if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use Authorization HTTP header and authentication via parameters simultaneously"); + throw Exception( + ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use Authorization HTTP header and authentication via parameters simultaneously"); std::string scheme; std::string auth_info; @@ -438,7 +429,8 @@ bool HTTPHandler::authenticateUser( } else { - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); + throw Exception( + ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); } } else @@ -464,7 +456,8 @@ bool HTTPHandler::authenticateUser( auto * gss_acceptor_context = dynamic_cast(request_credentials.get()); if (!gss_acceptor_context) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected"); + throw Exception( + ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected"); #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wunreachable-code" @@ -500,9 +493,10 @@ bool HTTPHandler::authenticateUser( if (params.has("quota_key")) { if (!quota_key.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Invalid authentication: it is not allowed " - "to use quota key as HTTP header and as parameter simultaneously"); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid authentication: it is not allowed " + "to use quota key as HTTP header and as parameter simultaneously"); quota_key = params.get("quota_key"); } @@ -627,26 +621,29 @@ void HTTPHandler::processQuery( size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; bool enable_http_compression = params.getParsed("enable_http_compression", context->getSettingsRef().enable_http_compression); - Int64 http_zlib_compression_level = params.getParsed("http_zlib_compression_level", context->getSettingsRef().http_zlib_compression_level); + Int64 http_zlib_compression_level + = params.getParsed("http_zlib_compression_level", context->getSettingsRef().http_zlib_compression_level); - used_output.out_holder = - std::make_shared( - response, - request.getMethod() == HTTPRequest::HTTP_HEAD, - context->getServerSettings().keep_alive_timeout.totalSeconds(), - write_event); + used_output.out_holder = std::make_shared( + response, + request.getMethod() == HTTPRequest::HTTP_HEAD, + context->getServerSettings().keep_alive_timeout.totalSeconds(), + write_event); used_output.out = used_output.out_holder; used_output.out_maybe_compressed = used_output.out_holder; if (client_supports_http_compression && enable_http_compression) { used_output.out_holder->setCompressionMethodHeader(http_response_compression_method); - used_output.wrap_compressed_holder = - wrapWriteBufferWithCompressionMethod( - used_output.out_holder.get(), - http_response_compression_method, - static_cast(http_zlib_compression_level), - 0, DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0, false); + used_output.wrap_compressed_holder = wrapWriteBufferWithCompressionMethod( + used_output.out_holder.get(), + http_response_compression_method, + static_cast(http_zlib_compression_level), + 0, + DBMS_DEFAULT_BUFFER_SIZE, + nullptr, + 0, + false); used_output.out = used_output.wrap_compressed_holder; } @@ -670,16 +667,13 @@ void HTTPHandler::processQuery( { auto tmp_data = std::make_shared(server.context()->getTempDataOnDisk()); - auto create_tmp_disk_buffer = [tmp_data] (const WriteBufferPtr &) -> WriteBufferPtr - { - return tmp_data->createRawStream(); - }; + auto create_tmp_disk_buffer = [tmp_data](const WriteBufferPtr &) -> WriteBufferPtr { return tmp_data->createRawStream(); }; cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); } else { - auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed] (const WriteBufferPtr & prev_buf) + auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed](const WriteBufferPtr & prev_buf) { auto * prev_memory_buffer = typeid_cast(prev_buf.get()); if (!prev_memory_buffer) @@ -694,7 +688,8 @@ void HTTPHandler::processQuery( cascade_buffer2.emplace_back(push_memory_buffer_and_continue); } - used_output.out_delayed_and_compressed_holder = std::make_unique(std::move(cascade_buffer1), std::move(cascade_buffer2)); + used_output.out_delayed_and_compressed_holder + = std::make_unique(std::move(cascade_buffer1), std::move(cascade_buffer2)); used_output.out_maybe_delayed_and_compressed = used_output.out_delayed_and_compressed_holder.get(); } else @@ -707,7 +702,8 @@ void HTTPHandler::processQuery( int zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); auto in_post = wrapReadBufferWithCompressionMethod( wrapReadBufferReference(request.getStream()), - chooseCompressionMethod({}, http_request_compression_method_str), zstd_window_log_max); + chooseCompressionMethod({}, http_request_compression_method_str), + zstd_window_log_max); /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. @@ -723,12 +719,26 @@ void HTTPHandler::processQuery( std::unique_ptr in; - static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "role", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session"}; + static const NameSet reserved_param_names{ + "compress", + "decompress", + "user", + "password", + "quota_key", + "query_id", + "stacktrace", + "role", + "buffer_size", + "wait_end_of_query", + "session_id", + "session_timeout", + "session_check", + "client_protocol_version", + "close_session"}; Names reserved_param_suffixes; - auto param_could_be_skipped = [&] (const String & name) + auto param_could_be_skipped = [&](const String & name) { /// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience. if (name.empty()) @@ -738,10 +748,8 @@ void HTTPHandler::processQuery( return true; for (const String & suffix : reserved_param_suffixes) - { if (endsWith(name, suffix)) return true; - } return false; }; @@ -859,47 +867,47 @@ void HTTPHandler::processQuery( if (settings.add_http_cors_header && !request.get("Origin", "").empty() && !config.has("http_options_response")) used_output.out_holder->addHeaderCORS(true); - auto append_callback = [my_context = context] (ProgressCallback callback) + auto append_callback = [my_context = context](ProgressCallback callback) { auto prev = my_context->getProgressCallback(); - my_context->setProgressCallback([prev, callback] (const Progress & progress) - { - if (prev) - prev(progress); + my_context->setProgressCallback( + [prev, callback](const Progress & progress) + { + if (prev) + prev(progress); - callback(progress); - }); + callback(progress); + }); }; /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. /// Note that we add it unconditionally so the progress is available for `X-ClickHouse-Summary` - append_callback([&used_output](const Progress & progress) - { - used_output.out_holder->onProgress(progress); - }); + append_callback([&used_output](const Progress & progress) { used_output.out_holder->onProgress(progress); }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) { - append_callback([&context, &request](const Progress &) - { - /// Assume that at the point this method is called no one is reading data from the socket any more: - /// should be true for read-only queries. - if (!request.checkPeerConnected()) - context->killCurrentQuery(); - }); + append_callback( + [&context, &request](const Progress &) + { + /// Assume that at the point this method is called no one is reading data from the socket any more: + /// should be true for read-only queries. + if (!request.checkPeerConnected()) + context->killCurrentQuery(); + }); } customizeContext(request, context, *in_post_maybe_compressed); in = has_external_data ? std::move(in_param) : std::make_unique(*in_param, *in_post_maybe_compressed); - auto set_query_result = [&response, this] (const QueryResultDetails & details) + auto set_query_result = [&response, this](const QueryResultDetails & details) { response.add("X-ClickHouse-Query-Id", details.query_id); + if (http_response_headers_override) + for (auto [header_name, header_value] : *http_response_headers_override) + response.add(header_name, header_value); - if (content_type_override) - response.setContentType(*content_type_override); - else if (details.content_type) + if (response.getContentType() == Poco::Net::HTTPMessage::UNKNOWN_CONTENT_TYPE && details.content_type) response.setContentType(*details.content_type); if (details.format) @@ -909,7 +917,10 @@ void HTTPHandler::processQuery( response.add("X-ClickHouse-Timezone", *details.timezone); }; - auto handle_exception_in_output_format = [&](IOutputFormat & current_output_format, const String & format_name, const ContextPtr & context_, const std::optional & format_settings) + auto handle_exception_in_output_format = [&](IOutputFormat & current_output_format, + const String & format_name, + const ContextPtr & context_, + const std::optional & format_settings) { if (settings.http_write_exception_in_output_format && current_output_format.supportsWritingException()) { @@ -929,7 +940,8 @@ void HTTPHandler::processQuery( } else { - bool with_stacktrace = (params.getParsed("stacktrace", false) && server.config().getBool("enable_http_stacktrace", true)); + bool with_stacktrace + = (params.getParsed("stacktrace", false) && server.config().getBool("enable_http_stacktrace", true)); ExecutionStatus status = ExecutionStatus::fromCurrentException("", with_stacktrace); formatExceptionForClient(status.code, request, response, used_output); current_output_format.setException(status.message); @@ -970,7 +982,8 @@ try if (!used_output.out_holder && !used_output.exception_is_written) { /// If nothing was sent yet and we don't even know if we must compress the response. - WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT).writeln(s); + WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT) + .writeln(s); } else if (used_output.out_maybe_compressed) { @@ -1034,7 +1047,8 @@ catch (...) } } -void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) +void HTTPHandler::formatExceptionForClient( + int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) { if (used_output.out_holder) used_output.out_holder->setExceptionCode(exception_code); @@ -1101,9 +1115,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse std::string opentelemetry_traceparent = request.get("traceparent"); std::string error; if (!client_trace_context.parseTraceparentHeader(opentelemetry_traceparent, error)) - { LOG_DEBUG(log, "Failed to parse OpenTelemetry traceparent header '{}': {}", opentelemetry_traceparent, error); - } client_trace_context.tracestate = request.get("tracestate", ""); } @@ -1141,9 +1153,10 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse /// Workaround. Poco does not detect 411 Length Required case. if (request.getMethod() == HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) { - throw Exception(ErrorCodes::HTTP_LENGTH_REQUIRED, - "The Transfer-Encoding is not chunked and there " - "is no Content-Length header for POST request"); + throw Exception( + ErrorCodes::HTTP_LENGTH_REQUIRED, + "The Transfer-Encoding is not chunked and there " + "is no Content-Length header for POST request"); } processQuery(request, params, response, used_output, query_scope, write_event); @@ -1155,7 +1168,8 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse catch (...) { SCOPE_EXIT({ - request_credentials.reset(); // ...so that the next requests on the connection have to always start afresh in case of exceptions. + request_credentials + .reset(); // ...so that the next requests on the connection have to always start afresh in case of exceptions. }); /// Check if exception was thrown in used_output.finalize(). @@ -1185,15 +1199,18 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse used_output.finalize(); } -DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & param_name_, const std::optional& content_type_override_) - : HTTPHandler(server_, "DynamicQueryHandler", content_type_override_), param_name(param_name_) +DynamicQueryHandler::DynamicQueryHandler( + IServer & server_, + const std::string & param_name_, + const std::optional> & http_response_headers_override_) + : HTTPHandler(server_, "DynamicQueryHandler", http_response_headers_override_), param_name(param_name_) { } bool DynamicQueryHandler::customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) { if (key == param_name) - return true; /// do nothing + return true; /// do nothing if (startsWith(key, QUERY_PARAMETER_NAME_PREFIX)) { @@ -1227,16 +1244,10 @@ std::string DynamicQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm std::string full_query; /// Params are of both form params POST and uri (GET params) for (const auto & it : params) - { if (it.first == param_name) - { full_query += it.second; - } else - { customizeQueryParam(context, it.first, it.second); - } - } return full_query; } @@ -1247,8 +1258,8 @@ PredefinedQueryHandler::PredefinedQueryHandler( const std::string & predefined_query_, const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_, - const std::optional & content_type_override_) - : HTTPHandler(server_, "PredefinedQueryHandler", content_type_override_) + const std::optional> & http_response_headers_override_) + : HTTPHandler(server_, "PredefinedQueryHandler", http_response_headers_override_) , receive_params(receive_params_) , predefined_query(predefined_query_) , url_regex(url_regex_) @@ -1334,20 +1345,37 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo return predefined_query; } -HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix) +std::optional> +parseHttpResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +{ + std::unordered_map http_response_headers_override; + String http_response_headers_key = config_prefix + ".handler.http_response_headers"; + String http_response_headers_key_prefix = http_response_headers_key + "."; + if (config.has(http_response_headers_key)) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_prefix + ".handler.http_response_headers", keys); + for (const auto & key : keys) + http_response_headers_override[key] = config.getString(http_response_headers_key_prefix + key); + } + if (config.has(config_prefix + ".handler.content_type")) + http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = config.getString(config_prefix + ".handler.content_type"); + + if (http_response_headers_override.empty()) + return std::nullopt; + + return std::optional(std::move(http_response_headers_override)); +} + +HTTPRequestHandlerFactoryPtr +createDynamicHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query"); - std::optional content_type_override; - if (config.has(config_prefix + ".handler.content_type")) - content_type_override = config.getString(config_prefix + ".handler.content_type"); + std::optional> http_response_headers_override = parseHttpResponseHeaders(config, config_prefix); - auto creator = [&server, query_param_name, content_type_override] () -> std::unique_ptr - { - return std::make_unique(server, query_param_name, content_type_override); - }; + auto creator = [&server, query_param_name, http_response_headers_override]() -> std::unique_ptr + { return std::make_unique(server, query_param_name, http_response_headers_override); }; auto factory = std::make_shared>(std::move(creator)); factory->addFiltersFromConfig(config, config_prefix); @@ -1357,11 +1385,14 @@ HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, static inline bool capturingNamedQueryParam(NameSet receive_params, const CompiledRegexPtr & compiled_regex) { const auto & capturing_names = compiled_regex->NamedCapturingGroups(); - return std::count_if(capturing_names.begin(), capturing_names.end(), [&](const auto & iterator) - { - return std::count_if(receive_params.begin(), receive_params.end(), - [&](const auto & param_name) { return param_name == iterator.first; }); - }); + return std::count_if( + capturing_names.begin(), + capturing_names.end(), + [&](const auto & iterator) + { + return std::count_if( + receive_params.begin(), receive_params.end(), [&](const auto & param_name) { return param_name == iterator.first; }); + }); } static inline CompiledRegexPtr getCompiledRegex(const std::string & expression) @@ -1369,15 +1400,18 @@ static inline CompiledRegexPtr getCompiledRegex(const std::string & expression) auto compiled_regex = std::make_shared(expression); if (!compiled_regex->ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile re2: {} for http handling rule, error: {}. " - "Look at https://github.com/google/re2/wiki/Syntax for reference.", expression, compiled_regex->error()); + throw Exception( + ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile re2: {} for http handling rule, error: {}. " + "Look at https://github.com/google/re2/wiki/Syntax for reference.", + expression, + compiled_regex->error()); return compiled_regex; } -HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix) +HTTPRequestHandlerFactoryPtr +createPredefinedHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { if (!config.has(config_prefix + ".handler.query")) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no path '{}.handler.query' in configuration file.", config_prefix); @@ -1402,9 +1436,7 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, headers_name_with_regex.emplace(std::make_pair(header_name, regex)); } - std::optional content_type_override; - if (config.has(config_prefix + ".handler.content_type")) - content_type_override = config.getString(config_prefix + ".handler.content_type"); + std::optional> http_response_headers_override = parseHttpResponseHeaders(config, config_prefix); std::shared_ptr> factory; @@ -1418,18 +1450,12 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, auto regex = getCompiledRegex(url_expression); if (capturingNamedQueryParam(analyze_receive_params, regex)) { - auto creator = [ - &server, - analyze_receive_params, - predefined_query, - regex, - headers_name_with_regex, - content_type_override] + auto creator + = [&server, analyze_receive_params, predefined_query, regex, headers_name_with_regex, http_response_headers_override] -> std::unique_ptr { return std::make_unique( - server, analyze_receive_params, predefined_query, regex, - headers_name_with_regex, content_type_override); + server, analyze_receive_params, predefined_query, regex, headers_name_with_regex, http_response_headers_override); }; factory = std::make_shared>(std::move(creator)); factory->addFiltersFromConfig(config, config_prefix); @@ -1437,17 +1463,11 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, } } - auto creator = [ - &server, - analyze_receive_params, - predefined_query, - headers_name_with_regex, - content_type_override] + auto creator = [&server, analyze_receive_params, predefined_query, headers_name_with_regex, http_response_headers_override] -> std::unique_ptr { return std::make_unique( - server, analyze_receive_params, predefined_query, CompiledRegexPtr{}, - headers_name_with_regex, content_type_override); + server, analyze_receive_params, predefined_query, CompiledRegexPtr{}, headers_name_with_regex, http_response_headers_override); }; factory = std::make_shared>(std::move(creator)); diff --git a/src/Server/HTTPHandler.h b/src/Server/HTTPHandler.h index a96402247a2..5eba8b9d2a6 100644 --- a/src/Server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -1,21 +1,27 @@ #pragma once +#include +#include +#include +#include #include +#include #include #include #include #include #include -#include -#include #include namespace CurrentMetrics { - extern const Metric HTTPConnection; +extern const Metric HTTPConnection; } -namespace Poco { class Logger; } +namespace Poco +{ +class Logger; +} namespace DB { @@ -31,13 +37,16 @@ using CompiledRegexPtr = std::shared_ptr; class HTTPHandler : public HTTPRequestHandler { public: - HTTPHandler(IServer & server_, const std::string & name, const std::optional & content_type_override_); + HTTPHandler( + IServer & server_, + const std::string & name, + const std::optional> & http_response_headers_override_); ~HTTPHandler() override; void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; /// This method is called right before the query execution. - virtual void customizeContext(HTTPServerRequest & /* request */, ContextMutablePtr /* context */, ReadBuffer & /* body */) {} + virtual void customizeContext(HTTPServerRequest & /* request */, ContextMutablePtr /* context */, ReadBuffer & /* body */) { } virtual bool customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) = 0; @@ -113,8 +122,8 @@ private: /// See settings http_max_fields, http_max_field_name_size, http_max_field_value_size in HTMLForm. const Settings & default_settings; - /// Overrides Content-Type provided by the format of the response. - std::optional content_type_override; + /// Overrides for response headers. + std::optional> http_response_headers_override; // session is reset at the end of each request/response. std::unique_ptr session; @@ -128,10 +137,7 @@ private: // Returns false when the user is not authenticated yet, and the 'Negotiate' response is sent, // the session and request_credentials instances are preserved. // Throws an exception if authentication failed. - bool authenticateUser( - HTTPServerRequest & request, - HTMLForm & params, - HTTPServerResponse & response); + bool authenticateUser(HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response); /// Also initializes 'used_output'. void processQuery( @@ -143,17 +149,9 @@ private: const ProfileEvents::Event & write_event); void trySendExceptionToClient( - const std::string & s, - int exception_code, - HTTPServerRequest & request, - HTTPServerResponse & response, - Output & used_output); + const std::string & s, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output); - void formatExceptionForClient( - int exception_code, - HTTPServerRequest & request, - HTTPServerResponse & response, - Output & used_output); + void formatExceptionForClient(int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output); static void pushDelayedResults(Output & used_output); }; @@ -162,12 +160,16 @@ class DynamicQueryHandler : public HTTPHandler { private: std::string param_name; + public: - explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query", const std::optional& content_type_override_ = std::nullopt); + explicit DynamicQueryHandler( + IServer & server_, + const std::string & param_name_ = "query", + const std::optional> & http_response_headers_override_ = std::nullopt); std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override; - bool customizeQueryParam(ContextMutablePtr context, const std::string &key, const std::string &value) override; + bool customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) override; }; class PredefinedQueryHandler : public HTTPHandler @@ -177,11 +179,15 @@ private: std::string predefined_query; CompiledRegexPtr url_regex; std::unordered_map header_name_with_capture_regex; + public: PredefinedQueryHandler( - IServer & server_, const NameSet & receive_params_, const std::string & predefined_query_ - , const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_ - , const std::optional & content_type_override_); + IServer & server_, + const NameSet & receive_params_, + const std::string & predefined_query_, + const CompiledRegexPtr & url_regex_, + const std::unordered_map & header_name_with_regex_, + const std::optional> & http_response_headers_override_ = std::nullopt); void customizeContext(HTTPServerRequest & request, ContextMutablePtr context, ReadBuffer & body) override; From 1245be5d978e0d0b2e60cc29f571dd7e19684630 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Mon, 13 May 2024 10:59:32 +0300 Subject: [PATCH 613/856] feat-59620 tests --- .../test_http_handlers_config/test.py | 16 ++++++++++++++++ .../test_dynamic_handler/config.xml | 4 ++++ .../test_predefined_handler/config.xml | 4 ++++ .../test_static_handler/config.xml | 4 ++++ 4 files changed, 28 insertions(+) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index f6ac42a2db2..d8b58976642 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -88,6 +88,8 @@ def test_dynamic_query_handler(): "application/whatever; charset=cp1337" == res_custom_ct.headers["content-type"] ) + assert "it works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Works"] + assert "also works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Even-Multiple"] def test_predefined_query_handler(): @@ -146,6 +148,8 @@ def test_predefined_query_handler(): ) assert b"max_final_threads\t1\nmax_threads\t1\n" == res2.content assert "application/generic+one" == res2.headers["content-type"] + assert "it works" == res2.headers["X-Test-Http-Response-Headers-Works"] + assert "also works" == res2.headers["X-Test-Http-Response-Headers-Even-Multiple"] cluster.instance.query( "CREATE TABLE test_table (id UInt32, data String) Engine=TinyLog" @@ -212,6 +216,18 @@ def test_fixed_static_handler(): "test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"} ).content ) + assert ( + "it works" + == cluster.instance.http_request( + "test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"} + ).headers["X-Test-Http-Response-Headers-Works"] + ) + assert ( + "also works" + == cluster.instance.http_request( + "test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"} + ).headers["X-Test-Http-Response-Headers-Even-Multiple"] + ) def test_config_static_handler(): diff --git a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml index c9b61c21507..58fedbd9078 100644 --- a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml @@ -18,6 +18,10 @@ dynamic_query_handler get_dynamic_handler_query application/whatever; charset=cp1337 + + it works + also works + diff --git a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml index 1b8ddfab323..a7804721f12 100644 --- a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml @@ -19,6 +19,10 @@ predefined_query_handler SELECT name, value FROM system.settings WHERE name = {setting_name_1:String} OR name = {setting_name_2:String} application/generic+one + + it works + also works + diff --git a/tests/integration/test_http_handlers_config/test_static_handler/config.xml b/tests/integration/test_http_handlers_config/test_static_handler/config.xml index ff24e6dec96..76c1f588853 100644 --- a/tests/integration/test_http_handlers_config/test_static_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_static_handler/config.xml @@ -12,6 +12,10 @@ 402 text/html; charset=UTF-8 Test get static handler and fix content + + it works + also works + From a417a1c676f88dd7f793bed486f686a9187c956e Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Tue, 28 May 2024 16:13:47 +0300 Subject: [PATCH 614/856] feat-59620 Style fix --- src/Server/HTTPHandler.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 3c0bbf03986..81a873c8c49 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -287,7 +287,7 @@ static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util: } std::optional> -parseHttpResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); +parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); void HTTPHandler::pushDelayedResults(Output & used_output) { @@ -1346,7 +1346,7 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo } std::optional> -parseHttpResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { std::unordered_map http_response_headers_override; String http_response_headers_key = config_prefix + ".handler.http_response_headers"; @@ -1354,7 +1354,7 @@ parseHttpResponseHeaders(const Poco::Util::AbstractConfiguration & config, const if (config.has(http_response_headers_key)) { Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_prefix + ".handler.http_response_headers", keys); + config.keys(http_response_headers_key, keys); for (const auto & key : keys) http_response_headers_override[key] = config.getString(http_response_headers_key_prefix + key); } @@ -1372,7 +1372,7 @@ createDynamicHandlerFactory(IServer & server, const Poco::Util::AbstractConfigur { auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query"); - std::optional> http_response_headers_override = parseHttpResponseHeaders(config, config_prefix); + std::optional> http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix); auto creator = [&server, query_param_name, http_response_headers_override]() -> std::unique_ptr { return std::make_unique(server, query_param_name, http_response_headers_override); }; @@ -1436,7 +1436,7 @@ createPredefinedHandlerFactory(IServer & server, const Poco::Util::AbstractConfi headers_name_with_regex.emplace(std::make_pair(header_name, regex)); } - std::optional> http_response_headers_override = parseHttpResponseHeaders(config, config_prefix); + std::optional> http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix); std::shared_ptr> factory; From 1c9652c06bace095da1911848f7c1b4680cde164 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Tue, 28 May 2024 17:45:20 +0300 Subject: [PATCH 615/856] feat-59620 Style fix --- tests/integration/test_http_handlers_config/test.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index d8b58976642..166ff720922 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -84,10 +84,7 @@ def test_dynamic_query_handler(): headers={"XXX": "xxx"}, ) assert 200 == res_custom_ct.status_code - assert ( - "application/whatever; charset=cp1337" - == res_custom_ct.headers["content-type"] - ) + assert "application/whatever; charset=cp1337" == res_custom_ct.headers["content-type"] assert "it works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Works"] assert "also works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Even-Multiple"] From 1618ce43971cf9432eec8e1e102bd4319c97c161 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Tue, 28 May 2024 17:58:03 +0300 Subject: [PATCH 616/856] feat-59620 Style fix --- .../integration/test_http_handlers_config/test.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 166ff720922..b2efbf4bb65 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -84,9 +84,15 @@ def test_dynamic_query_handler(): headers={"XXX": "xxx"}, ) assert 200 == res_custom_ct.status_code - assert "application/whatever; charset=cp1337" == res_custom_ct.headers["content-type"] + assert ( + "application/whatever; charset=cp1337" + == res_custom_ct.headers["content-type"] + ) assert "it works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Works"] - assert "also works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Even-Multiple"] + assert ( + "also works" + == res_custom_ct.headers["X-Test-Http-Response-Headers-Even-Multiple"] + ) def test_predefined_query_handler(): @@ -146,7 +152,9 @@ def test_predefined_query_handler(): assert b"max_final_threads\t1\nmax_threads\t1\n" == res2.content assert "application/generic+one" == res2.headers["content-type"] assert "it works" == res2.headers["X-Test-Http-Response-Headers-Works"] - assert "also works" == res2.headers["X-Test-Http-Response-Headers-Even-Multiple"] + assert ( + "also works" == res2.headers["X-Test-Http-Response-Headers-Even-Multiple"] + ) cluster.instance.query( "CREATE TABLE test_table (id UInt32, data String) Engine=TinyLog" From ffa66225d07c8e1fd0c2ba2aaab486fc2e81b1d1 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Tue, 4 Jun 2024 23:59:08 +0300 Subject: [PATCH 617/856] Fix tests --- src/Server/HTTPHandler.cpp | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 81a873c8c49..00757142882 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -900,14 +900,19 @@ void HTTPHandler::processQuery( customizeContext(request, context, *in_post_maybe_compressed); in = has_external_data ? std::move(in_param) : std::make_unique(*in_param, *in_post_maybe_compressed); - auto set_query_result = [&response, this](const QueryResultDetails & details) + if (http_response_headers_override) { + for (auto [header_name, header_value] : *http_response_headers_override) + response.set(header_name, header_value); + } + + auto set_query_result = [this, &response](const QueryResultDetails & details) { response.add("X-ClickHouse-Query-Id", details.query_id); - if (http_response_headers_override) - for (auto [header_name, header_value] : *http_response_headers_override) - response.add(header_name, header_value); - if (response.getContentType() == Poco::Net::HTTPMessage::UNKNOWN_CONTENT_TYPE && details.content_type) + if (!( + http_response_headers_override.has_value() + && http_response_headers_override->contains(Poco::Net::HTTPMessage::CONTENT_TYPE) + ) && details.content_type) response.setContentType(*details.content_type); if (details.format) @@ -1362,9 +1367,9 @@ parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = config.getString(config_prefix + ".handler.content_type"); if (http_response_headers_override.empty()) - return std::nullopt; + return {}; - return std::optional(std::move(http_response_headers_override)); + return std::move(http_response_headers_override); } HTTPRequestHandlerFactoryPtr From 979b447451532715167edee437e42d03e6734c9e Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Wed, 5 Jun 2024 07:51:26 +0300 Subject: [PATCH 618/856] Fix styles --- src/Server/HTTPHandler.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 00757142882..da635f99014 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -900,10 +900,9 @@ void HTTPHandler::processQuery( customizeContext(request, context, *in_post_maybe_compressed); in = has_external_data ? std::move(in_param) : std::make_unique(*in_param, *in_post_maybe_compressed); - if (http_response_headers_override) { + if (http_response_headers_override) for (auto [header_name, header_value] : *http_response_headers_override) response.set(header_name, header_value); - } auto set_query_result = [this, &response](const QueryResultDetails & details) { From 06383d7a7a1c1a1a111090fc52e863e120232376 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Thu, 6 Jun 2024 10:50:31 +0300 Subject: [PATCH 619/856] Fix Static handler --- src/Server/HTTPHandler.cpp | 58 ++++----------- src/Server/HTTPHandler.h | 23 +++--- src/Server/HTTPHandlerFactory.cpp | 96 +++++++++++++------------ src/Server/HTTPResponseHeaderWriter.cpp | 69 ++++++++++++++++++ src/Server/HTTPResponseHeaderWriter.h | 23 ++++++ src/Server/StaticRequestHandler.cpp | 66 +++++++++-------- src/Server/StaticRequestHandler.h | 9 +-- 7 files changed, 204 insertions(+), 140 deletions(-) create mode 100644 src/Server/HTTPResponseHeaderWriter.cpp create mode 100644 src/Server/HTTPResponseHeaderWriter.h diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index da635f99014..a2af9905c72 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -27,6 +27,7 @@ #include #include #include +#include "Common/logger_useful.h" #include #include #include @@ -286,9 +287,6 @@ static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util: return std::chrono::seconds(session_timeout); } -std::optional> -parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); - void HTTPHandler::pushDelayedResults(Output & used_output) { std::vector write_buffers; @@ -325,8 +323,7 @@ void HTTPHandler::pushDelayedResults(Output & used_output) } -HTTPHandler::HTTPHandler( - IServer & server_, const std::string & name, const std::optional> & http_response_headers_override_) +HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_) : server(server_) , log(getLogger(name)) , default_settings(server.context()->getSettingsRef()) @@ -711,7 +708,8 @@ void HTTPHandler::processQuery( bool is_in_post_compressed = false; if (params.getParsed("decompress", false)) { - in_post_maybe_compressed = std::make_unique(*in_post, /* allow_different_codecs_ = */ false, /* external_data_ = */ true); + in_post_maybe_compressed + = std::make_unique(*in_post, /* allow_different_codecs_ = */ false, /* external_data_ = */ true); is_in_post_compressed = true; } else @@ -900,18 +898,14 @@ void HTTPHandler::processQuery( customizeContext(request, context, *in_post_maybe_compressed); in = has_external_data ? std::move(in_param) : std::make_unique(*in_param, *in_post_maybe_compressed); - if (http_response_headers_override) - for (auto [header_name, header_value] : *http_response_headers_override) - response.set(header_name, header_value); + applyHTTPResponseHeaders(response, http_response_headers_override); auto set_query_result = [this, &response](const QueryResultDetails & details) { response.add("X-ClickHouse-Query-Id", details.query_id); - if (!( - http_response_headers_override.has_value() - && http_response_headers_override->contains(Poco::Net::HTTPMessage::CONTENT_TYPE) - ) && details.content_type) + if (!(http_response_headers_override && http_response_headers_override->contains(Poco::Net::HTTPMessage::CONTENT_TYPE)) + && details.content_type) response.setContentType(*details.content_type); if (details.format) @@ -1125,10 +1119,8 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse // Setup tracing context for this thread auto context = session->sessionOrGlobalContext(); - thread_trace_context = std::make_unique("HTTPHandler", - client_trace_context, - context->getSettingsRef(), - context->getOpenTelemetrySpanLog()); + thread_trace_context = std::make_unique( + "HTTPHandler", client_trace_context, context->getSettingsRef(), context->getOpenTelemetrySpanLog()); thread_trace_context->root_span.kind = OpenTelemetry::SpanKind::SERVER; thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI()); @@ -1204,9 +1196,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse } DynamicQueryHandler::DynamicQueryHandler( - IServer & server_, - const std::string & param_name_, - const std::optional> & http_response_headers_override_) + IServer & server_, const std::string & param_name_, const HTTPResponseHeaderSetup & http_response_headers_override_) : HTTPHandler(server_, "DynamicQueryHandler", http_response_headers_override_), param_name(param_name_) { } @@ -1262,7 +1252,7 @@ PredefinedQueryHandler::PredefinedQueryHandler( const std::string & predefined_query_, const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_, - const std::optional> & http_response_headers_override_) + const HTTPResponseHeaderSetup & http_response_headers_override_) : HTTPHandler(server_, "PredefinedQueryHandler", http_response_headers_override_) , receive_params(receive_params_) , predefined_query(predefined_query_) @@ -1349,34 +1339,12 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo return predefined_query; } -std::optional> -parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) -{ - std::unordered_map http_response_headers_override; - String http_response_headers_key = config_prefix + ".handler.http_response_headers"; - String http_response_headers_key_prefix = http_response_headers_key + "."; - if (config.has(http_response_headers_key)) - { - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(http_response_headers_key, keys); - for (const auto & key : keys) - http_response_headers_override[key] = config.getString(http_response_headers_key_prefix + key); - } - if (config.has(config_prefix + ".handler.content_type")) - http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = config.getString(config_prefix + ".handler.content_type"); - - if (http_response_headers_override.empty()) - return {}; - - return std::move(http_response_headers_override); -} - HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query"); - std::optional> http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix); + HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix); auto creator = [&server, query_param_name, http_response_headers_override]() -> std::unique_ptr { return std::make_unique(server, query_param_name, http_response_headers_override); }; @@ -1440,7 +1408,7 @@ createPredefinedHandlerFactory(IServer & server, const Poco::Util::AbstractConfi headers_name_with_regex.emplace(std::make_pair(header_name, regex)); } - std::optional> http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix); + HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix); std::shared_ptr> factory; diff --git a/src/Server/HTTPHandler.h b/src/Server/HTTPHandler.h index 5eba8b9d2a6..1bf1dbdebf8 100644 --- a/src/Server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -13,6 +13,8 @@ #include #include +#include "HTTPResponseHeaderWriter.h" + namespace CurrentMetrics { extern const Metric HTTPConnection; @@ -37,10 +39,7 @@ using CompiledRegexPtr = std::shared_ptr; class HTTPHandler : public HTTPRequestHandler { public: - HTTPHandler( - IServer & server_, - const std::string & name, - const std::optional> & http_response_headers_override_); + HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_); ~HTTPHandler() override; void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; @@ -86,10 +85,7 @@ private: bool exception_is_written = false; std::function exception_writer; - bool hasDelayed() const - { - return out_maybe_delayed_and_compressed != out_maybe_compressed.get(); - } + bool hasDelayed() const { return out_maybe_delayed_and_compressed != out_maybe_compressed.get(); } void finalize() { @@ -103,10 +99,7 @@ private: out->finalize(); } - bool isFinalized() const - { - return finalized; - } + bool isFinalized() const { return finalized; } }; IServer & server; @@ -123,7 +116,7 @@ private: const Settings & default_settings; /// Overrides for response headers. - std::optional> http_response_headers_override; + HTTPResponseHeaderSetup http_response_headers_override; // session is reset at the end of each request/response. std::unique_ptr session; @@ -165,7 +158,7 @@ public: explicit DynamicQueryHandler( IServer & server_, const std::string & param_name_ = "query", - const std::optional> & http_response_headers_override_ = std::nullopt); + const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt); std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override; @@ -187,7 +180,7 @@ public: const std::string & predefined_query_, const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_, - const std::optional> & http_response_headers_override_ = std::nullopt); + const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt); void customizeContext(HTTPServerRequest & request, ContextMutablePtr context, ReadBuffer & body) override; diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 9a67e576345..d125e08c704 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -1,18 +1,18 @@ #include #include +#include #include #include -#include #include #include "HTTPHandler.h" -#include "Server/PrometheusMetricsWriter.h" -#include "StaticRequestHandler.h" -#include "ReplicasStatusHandler.h" #include "InterserverIOHTTPHandler.h" #include "PrometheusRequestHandler.h" +#include "ReplicasStatusHandler.h" +#include "Server/PrometheusMetricsWriter.h" +#include "StaticRequestHandler.h" #include "WebUIRequestHandler.h" @@ -21,9 +21,9 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; - extern const int UNKNOWN_ELEMENT_IN_CONFIG; - extern const int INVALID_CONFIG_PARAMETER; +extern const int LOGICAL_ERROR; +extern const int UNKNOWN_ELEMENT_IN_CONFIG; +extern const int INVALID_CONFIG_PARAMETER; } namespace @@ -35,10 +35,7 @@ private: std::string url; public: - explicit RedirectRequestHandler(std::string url_) - : url(std::move(url_)) - { - } + explicit RedirectRequestHandler(std::string url_) : url(std::move(url_)) { } void handleRequest(HTTPServerRequest &, HTTPServerResponse & response, const ProfileEvents::Event &) override { @@ -46,9 +43,8 @@ public: } }; -HTTPRequestHandlerFactoryPtr createRedirectHandlerFactory( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix) +HTTPRequestHandlerFactoryPtr +createRedirectHandlerFactory(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { std::string url = config.getString(config_prefix + ".handler.location"); @@ -74,7 +70,8 @@ static auto createPingHandlerFactory(IServer & server) auto creator = [&server]() -> std::unique_ptr { constexpr auto ping_response_expression = "Ok.\n"; - return std::make_unique(server, ping_response_expression); + return std::make_unique( + server, ping_response_expression, parseHTTPResponseHeaders("text/html; charset=UTF-8")); }; return std::make_shared>(std::move(creator)); } @@ -102,8 +99,12 @@ static inline auto createHandlersFactoryFromConfig( const auto & handler_type = config.getString(prefix + "." + key + ".handler.type", ""); if (handler_type.empty()) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Handler type in config is not specified here: " - "{}.{}.handler.type", prefix, key); + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "Handler type in config is not specified here: " + "{}.{}.handler.type", + prefix, + key); if (handler_type == "static") { @@ -154,19 +155,27 @@ static inline auto createHandlersFactoryFromConfig( main_handler_factory->addHandler(std::move(handler)); } else - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Unknown handler type '{}' in config here: {}.{}.handler.type", - handler_type, prefix, key); + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "Unknown handler type '{}' in config here: {}.{}.handler.type", + handler_type, + prefix, + key); } else - throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown element in config: " - "{}.{}, must be 'rule' or 'defaults'", prefix, key); + throw Exception( + ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, + "Unknown element in config: " + "{}.{}, must be 'rule' or 'defaults'", + prefix, + key); } return main_handler_factory; } -static inline HTTPRequestHandlerFactoryPtr -createHTTPHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & name, AsynchronousMetrics & async_metrics) +static inline HTTPRequestHandlerFactoryPtr createHTTPHandlerFactory( + IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & name, AsynchronousMetrics & async_metrics) { if (config.has("http_handlers")) { @@ -193,7 +202,8 @@ static inline HTTPRequestHandlerFactoryPtr createInterserverHTTPHandlerFactory(I } -HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & name) +HTTPRequestHandlerFactoryPtr createHandlerFactory( + IServer & server, const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & name) { if (name == "HTTPHandler-factory" || name == "HTTPSHandler-factory") return createHTTPHandlerFactory(server, config, name, async_metrics); @@ -214,7 +224,8 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS auto root_creator = [&server]() -> std::unique_ptr { constexpr auto root_response_expression = "config://http_server_default_response"; - return std::make_unique(server, root_response_expression); + return std::make_unique( + server, root_response_expression, parseHTTPResponseHeaders("text/html; charset=UTF-8")); }; auto root_handler = std::make_shared>(std::move(root_creator)); root_handler->attachStrictPath("/"); @@ -265,28 +276,23 @@ void addDefaultHandlersFactory( { addCommonDefaultHandlersFactory(factory, server); - auto dynamic_creator = [&server] () -> std::unique_ptr - { - return std::make_unique(server, "query"); - }; + auto dynamic_creator + = [&server]() -> std::unique_ptr { return std::make_unique(server, "query"); }; auto query_handler = std::make_shared>(std::move(dynamic_creator)); - query_handler->addFilter([](const auto & request) + query_handler->addFilter( + [](const auto & request) { - bool path_matches_get_or_head = startsWith(request.getURI(), "?") - || startsWith(request.getURI(), "/?") - || startsWith(request.getURI(), "/query?"); - bool is_get_or_head_request = request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; + bool path_matches_get_or_head + = startsWith(request.getURI(), "?") || startsWith(request.getURI(), "/?") || startsWith(request.getURI(), "/query?"); + bool is_get_or_head_request + = request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; - bool path_matches_post_or_options = path_matches_get_or_head - || request.getURI() == "/" - || request.getURI().empty(); - bool is_post_or_options_request = request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS; + bool path_matches_post_or_options = path_matches_get_or_head || request.getURI() == "/" || request.getURI().empty(); + bool is_post_or_options_request + = request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS; return (path_matches_get_or_head && is_get_or_head_request) || (path_matches_post_or_options && is_post_or_options_request); - } - ); + }); factory.addHandler(query_handler); /// We check that prometheus handler will be served on current (default) port. @@ -294,10 +300,8 @@ void addDefaultHandlersFactory( if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) { auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator = [&server, writer] () -> std::unique_ptr - { - return std::make_unique(server, writer); - }; + auto creator = [&server, writer]() -> std::unique_ptr + { return std::make_unique(server, writer); }; auto prometheus_handler = std::make_shared>(std::move(creator)); prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); prometheus_handler->allowGetAndHeadRequest(); diff --git a/src/Server/HTTPResponseHeaderWriter.cpp b/src/Server/HTTPResponseHeaderWriter.cpp new file mode 100644 index 00000000000..f5ab196c4b0 --- /dev/null +++ b/src/Server/HTTPResponseHeaderWriter.cpp @@ -0,0 +1,69 @@ +#include "HTTPResponseHeaderWriter.h" +#include +#include +#include + +namespace DB +{ + +std::unordered_map +baseParseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +{ + std::unordered_map http_response_headers_override; + String http_response_headers_key = config_prefix + ".handler.http_response_headers"; + String http_response_headers_key_prefix = http_response_headers_key + "."; + if (config.has(http_response_headers_key)) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(http_response_headers_key, keys); + for (const auto & key : keys) + { + http_response_headers_override[key] = config.getString(http_response_headers_key_prefix + key); + } + } + if (config.has(config_prefix + ".handler.content_type")) + http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = config.getString(config_prefix + ".handler.content_type"); + + return http_response_headers_override; +} + +HTTPResponseHeaderSetup parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +{ + std::unordered_map http_response_headers_override = baseParseHTTPResponseHeaders(config, config_prefix); + + if (http_response_headers_override.empty()) + return {}; + + return std::move(http_response_headers_override); +} + +std::unordered_map parseHTTPResponseHeaders( + const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_content_type) +{ + std::unordered_map http_response_headers_override = baseParseHTTPResponseHeaders(config, config_prefix); + + if (!http_response_headers_override.contains(Poco::Net::HTTPMessage::CONTENT_TYPE)) + http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = default_content_type; + + return http_response_headers_override; +} + +std::unordered_map parseHTTPResponseHeaders(const std::string & default_content_type) +{ + return {{{Poco::Net::HTTPMessage::CONTENT_TYPE, default_content_type}}}; +} + +void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup) +{ + if (setup) + for (auto [header_name, header_value] : *setup) + response.set(header_name, header_value); +} + +void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map & setup) +{ + for (auto [header_name, header_value] : setup) + response.set(header_name, header_value); +} + +} diff --git a/src/Server/HTTPResponseHeaderWriter.h b/src/Server/HTTPResponseHeaderWriter.h new file mode 100644 index 00000000000..066cf84eca7 --- /dev/null +++ b/src/Server/HTTPResponseHeaderWriter.h @@ -0,0 +1,23 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +using HTTPResponseHeaderSetup = std::optional>; + +HTTPResponseHeaderSetup parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); + +std::unordered_map parseHTTPResponseHeaders( + const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_content_type); + +std::unordered_map parseHTTPResponseHeaders(const std::string & default_content_type); + +void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup); + +void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map & setup); +} diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 67bf3875de4..e320507fc66 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -2,23 +2,24 @@ #include "IServer.h" #include "HTTPHandlerFactory.h" -#include "HTTPHandlerRequestFilter.h" +#include "HTTPResponseHeaderWriter.h" #include #include #include -#include #include -#include +#include #include +#include #include +#include +#include +#include #include #include -#include #include -#include namespace fs = std::filesystem; @@ -28,15 +29,16 @@ namespace DB namespace ErrorCodes { - extern const int INCORRECT_FILE_NAME; - extern const int HTTP_LENGTH_REQUIRED; - extern const int INVALID_CONFIG_PARAMETER; +extern const int INCORRECT_FILE_NAME; +extern const int HTTP_LENGTH_REQUIRED; +extern const int INVALID_CONFIG_PARAMETER; } static inline std::unique_ptr responseWriteBuffer(HTTPServerRequest & request, HTTPServerResponse & response, UInt64 keep_alive_timeout) { - auto buf = std::unique_ptr(new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout)); + auto buf = std::unique_ptr( + new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout)); /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -61,8 +63,8 @@ static inline void trySendExceptionToClient( /// If HTTP method is POST and Keep-Alive is turned on, we should 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() && !request.getStream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() && !request.getStream().eof() + && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) request.getStream().ignore(std::numeric_limits::max()); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); @@ -87,23 +89,26 @@ static inline void trySendExceptionToClient( } } -void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/) +void StaticRequestHandler::handleRequest( + HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/) { auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds(); auto out = responseWriteBuffer(request, response, keep_alive_timeout); try { - response.setContentType(content_type); + applyHTTPResponseHeaders(response, http_response_headers_override); if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) - throw Exception(ErrorCodes::HTTP_LENGTH_REQUIRED, - "The Transfer-Encoding is not chunked and there " - "is no Content-Length header for POST request"); + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() + && !request.hasContentLength()) + throw Exception( + ErrorCodes::HTTP_LENGTH_REQUIRED, + "The Transfer-Encoding is not chunked and there " + "is no Content-Length header for POST request"); setResponseDefaultHeaders(response, keep_alive_timeout); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status)); @@ -144,9 +149,10 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) else if (startsWith(response_expression, config_prefix)) { if (response_expression.size() <= config_prefix.size()) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, - "Static handling rule handler must contain a complete configuration path, for example: " - "config://config_key"); + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "Static handling rule handler must contain a complete configuration path, for example: " + "config://config_key"); const auto & config_path = response_expression.substr(config_prefix.size(), response_expression.size() - config_prefix.size()); writeString(server.config().getRawString(config_path, "Ok.\n"), out); @@ -155,23 +161,23 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) writeString(response_expression, out); } -StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & expression, int status_, const String & content_type_) - : server(server_), status(status_), content_type(content_type_), response_expression(expression) +StaticRequestHandler::StaticRequestHandler( + IServer & server_, const String & expression, const std::unordered_map & http_response_headers_override_, int status_) + : server(server_), status(status_), http_response_headers_override(http_response_headers_override_), response_expression(expression) { } -HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix) +HTTPRequestHandlerFactoryPtr +createStaticHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { int status = config.getInt(config_prefix + ".handler.status", 200); std::string response_content = config.getRawString(config_prefix + ".handler.response_content", "Ok.\n"); - std::string response_content_type = config.getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8"); - auto creator = [&server, response_content, status, response_content_type]() -> std::unique_ptr - { - return std::make_unique(server, response_content, status, response_content_type); - }; + std::unordered_map http_response_headers_override + = parseHTTPResponseHeaders(config, config_prefix, "text/plain; charset=UTF-8"); + + auto creator = [&server, http_response_headers_override, response_content, status]() -> std::unique_ptr + { return std::make_unique(server, response_content, http_response_headers_override, status); }; auto factory = std::make_shared>(std::move(creator)); diff --git a/src/Server/StaticRequestHandler.h b/src/Server/StaticRequestHandler.h index 38d774bb0aa..41fb395d969 100644 --- a/src/Server/StaticRequestHandler.h +++ b/src/Server/StaticRequestHandler.h @@ -1,9 +1,9 @@ #pragma once +#include #include #include - namespace DB { @@ -17,15 +17,16 @@ private: IServer & server; int status; - String content_type; + /// Overrides for response headers. + std::unordered_map http_response_headers_override; String response_expression; public: StaticRequestHandler( IServer & server, const String & expression, - int status_ = 200, - const String & content_type_ = "text/html; charset=UTF-8"); + const std::unordered_map & http_response_headers_override_, + int status_ = 200); void writeResponse(WriteBuffer & out); From 8119054cea7840c421bfeb0b90927ef7e5de9b97 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Thu, 6 Jun 2024 11:17:10 +0300 Subject: [PATCH 620/856] Add documentation --- docs/en/interfaces/http.md | 37 ++++++++++++++++++++++++++++++++++++- docs/ru/interfaces/http.md | 35 +++++++++++++++++++++++++++++++++++ 2 files changed, 71 insertions(+), 1 deletion(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index eb1a3ba1dbc..f5b6326fa96 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -508,7 +508,7 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`: - `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2’s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request. -- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`. +- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `http_response_headers`, `response_content`, `query`, `query_param_name`. `type` currently supports three types: [predefined_query_handler](#predefined_query_handler), [dynamic_query_handler](#dynamic_query_handler), [static](#static). - `query` — use with `predefined_query_handler` type, executes query when the handler is called. @@ -519,6 +519,8 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`: - `content_type` — use with any type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type). + - `http_response_headers` — use with any type, response headers map. Could be used to set content type as well. + - `response_content` — use with `static` type, response content sent to client, when using the prefix ‘file://’ or ‘config://’, find the content from the file or configuration sends to client. Next are the configuration methods for different `type`. @@ -616,6 +618,33 @@ Return a message. static 402 text/html; charset=UTF-8 + + en + 43 + + Say Hi! + + + + +``` + +`http_response_headers` could be used to set content type instead of `content_type`. + +``` xml + + + GET + xxx + /hi + + static + 402 + + text/html; charset=UTF-8 + en + 43 + Say Hi! @@ -696,6 +725,9 @@ Find the content from the file send to client. static text/html; charset=UTF-8 + + 737060cd8c284d8af7ad3082f209582d + file:///absolute_path_file.html @@ -706,6 +738,9 @@ Find the content from the file send to client. static text/html; charset=UTF-8 + + 737060cd8c284d8af7ad3082f209582d + file://./relative_path_file.html diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 5f11f1b430b..d9da51892f9 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -414,6 +414,8 @@ $ curl -v 'http://localhost:8123/predefined_query' - `content_type` — используется со всеми типами, возвращает [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type). + - `http_response_headers` — используется со всеми типами чтобы добавить кастомные хедеры в ответ. Может использоваться в том числе для задания хедера `Content-Type` вместо `content_type`. + - `response_content` — используется с типом`static`, содержимое ответа, отправленное клиенту, при использовании префикса ‘file://’ or ‘config://’, находит содержимое из файла или конфигурации, отправленного клиенту. Далее приведены методы настройки для различных типов. @@ -509,6 +511,33 @@ max_final_threads 2 static 402 text/html; charset=UTF-8 + + en + 43 + + Say Hi! + + + + +``` + +`http_response_headers` так же может использоваться для определения `Content-Type` вместо `content_type`. + +``` xml + + + GET + xxx + /hi + + static + 402 + + text/html; charset=UTF-8 + en + 43 + Say Hi! @@ -589,6 +618,9 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' static text/html; charset=UTF-8 + + 737060cd8c284d8af7ad3082f209582d + file:///absolute_path_file.html @@ -599,6 +631,9 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' static text/html; charset=UTF-8 + + 737060cd8c284d8af7ad3082f209582d + file://./relative_path_file.html From 7bff47701b530d1e463ad21e5a895f1e8bc31df9 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Thu, 6 Jun 2024 11:51:16 +0300 Subject: [PATCH 621/856] Fix styles --- src/Server/HTTPResponseHeaderWriter.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Server/HTTPResponseHeaderWriter.h b/src/Server/HTTPResponseHeaderWriter.h index 066cf84eca7..06281abb42d 100644 --- a/src/Server/HTTPResponseHeaderWriter.h +++ b/src/Server/HTTPResponseHeaderWriter.h @@ -1,3 +1,5 @@ +#pragma once + #include #include #include From 2aa2f2f7da202cb1ddcd9a8e5a395b3bd3821403 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Wed, 12 Jun 2024 22:55:32 +0300 Subject: [PATCH 622/856] Rollback unnecessary style fixes --- src/Server/HTTPHandler.cpp | 470 ++++++++++++++-------------- src/Server/HTTPHandler.h | 42 ++- src/Server/HTTPHandlerFactory.cpp | 90 +++--- src/Server/StaticRequestHandler.cpp | 46 ++- 4 files changed, 331 insertions(+), 317 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index a2af9905c72..f6ca69813ae 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1,8 +1,8 @@ #include -#include #include #include +#include #include #include #include @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -18,37 +17,38 @@ #include #include #include -#include #include -#include -#include #include -#include +#include +#include #include #include #include -#include "Common/logger_useful.h" +#include #include #include #include #include #include +#include +#include +#include -#include #include #include +#include #include "config.h" #include #include -#include #include #include #include -#include +#include #include #include +#include #include #include @@ -59,7 +59,7 @@ #include #if USE_SSL -# include +#include #endif @@ -68,68 +68,68 @@ namespace DB namespace ErrorCodes { -extern const int BAD_ARGUMENTS; -extern const int LOGICAL_ERROR; -extern const int CANNOT_SCHEDULE_TASK; -extern const int CANNOT_PARSE_TEXT; -extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; -extern const int CANNOT_PARSE_QUOTED_STRING; -extern const int CANNOT_PARSE_DATE; -extern const int CANNOT_PARSE_DATETIME; -extern const int CANNOT_PARSE_NUMBER; -extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; -extern const int CANNOT_PARSE_IPV4; -extern const int CANNOT_PARSE_IPV6; -extern const int CANNOT_PARSE_UUID; -extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; -extern const int CANNOT_OPEN_FILE; -extern const int CANNOT_COMPILE_REGEXP; -extern const int DUPLICATE_COLUMN; -extern const int ILLEGAL_COLUMN; -extern const int THERE_IS_NO_COLUMN; -extern const int UNKNOWN_ELEMENT_IN_AST; -extern const int UNKNOWN_TYPE_OF_AST_NODE; -extern const int TOO_DEEP_AST; -extern const int TOO_BIG_AST; -extern const int UNEXPECTED_AST_STRUCTURE; -extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int CANNOT_COMPILE_REGEXP; + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_PARSE_TEXT; + extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; + extern const int CANNOT_PARSE_QUOTED_STRING; + extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_PARSE_DATETIME; + extern const int CANNOT_PARSE_NUMBER; + extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; + extern const int CANNOT_PARSE_IPV4; + extern const int CANNOT_PARSE_IPV6; + extern const int CANNOT_PARSE_UUID; + extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; + extern const int CANNOT_SCHEDULE_TASK; + extern const int DUPLICATE_COLUMN; + extern const int ILLEGAL_COLUMN; + extern const int THERE_IS_NO_COLUMN; + extern const int UNKNOWN_ELEMENT_IN_AST; + extern const int UNKNOWN_TYPE_OF_AST_NODE; + extern const int TOO_DEEP_AST; + extern const int TOO_BIG_AST; + extern const int UNEXPECTED_AST_STRUCTURE; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; -extern const int SYNTAX_ERROR; + extern const int SYNTAX_ERROR; -extern const int INCORRECT_DATA; -extern const int TYPE_MISMATCH; + extern const int INCORRECT_DATA; + extern const int TYPE_MISMATCH; -extern const int UNKNOWN_TABLE; -extern const int UNKNOWN_FUNCTION; -extern const int UNKNOWN_IDENTIFIER; -extern const int UNKNOWN_TYPE; -extern const int UNKNOWN_STORAGE; -extern const int UNKNOWN_DATABASE; -extern const int UNKNOWN_SETTING; -extern const int UNKNOWN_DIRECTION_OF_SORTING; -extern const int UNKNOWN_AGGREGATE_FUNCTION; -extern const int UNKNOWN_FORMAT; -extern const int UNKNOWN_DATABASE_ENGINE; -extern const int UNKNOWN_TYPE_OF_QUERY; -extern const int UNKNOWN_ROLE; -extern const int NO_ELEMENTS_IN_CONFIG; + extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_FUNCTION; + extern const int UNKNOWN_IDENTIFIER; + extern const int UNKNOWN_TYPE; + extern const int UNKNOWN_STORAGE; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_SETTING; + extern const int UNKNOWN_DIRECTION_OF_SORTING; + extern const int UNKNOWN_AGGREGATE_FUNCTION; + extern const int UNKNOWN_FORMAT; + extern const int UNKNOWN_DATABASE_ENGINE; + extern const int UNKNOWN_TYPE_OF_QUERY; + extern const int UNKNOWN_ROLE; + extern const int NO_ELEMENTS_IN_CONFIG; -extern const int QUERY_IS_TOO_LARGE; + extern const int QUERY_IS_TOO_LARGE; -extern const int NOT_IMPLEMENTED; -extern const int SOCKET_TIMEOUT; + extern const int NOT_IMPLEMENTED; + extern const int SOCKET_TIMEOUT; -extern const int UNKNOWN_USER; -extern const int WRONG_PASSWORD; -extern const int REQUIRED_PASSWORD; -extern const int AUTHENTICATION_FAILED; -extern const int SET_NON_GRANTED_ROLE; + extern const int UNKNOWN_USER; + extern const int WRONG_PASSWORD; + extern const int REQUIRED_PASSWORD; + extern const int AUTHENTICATION_FAILED; + extern const int SET_NON_GRANTED_ROLE; -extern const int INVALID_SESSION_TIMEOUT; -extern const int HTTP_LENGTH_REQUIRED; -extern const int SUPPORT_IS_DISABLED; + extern const int INVALID_SESSION_TIMEOUT; + extern const int HTTP_LENGTH_REQUIRED; + extern const int SUPPORT_IS_DISABLED; -extern const int TIMEOUT_EXCEEDED; + extern const int TIMEOUT_EXCEEDED; } namespace @@ -148,9 +148,9 @@ bool tryAddHTTPOptionHeadersFromConfig(HTTPServerResponse & response, const Poco if (config.getString("http_options_response." + config_key + ".name", "").empty()) LOG_WARNING(getLogger("processOptionsRequest"), "Empty header was found in config. It will not be processed."); else - response.add( - config.getString("http_options_response." + config_key + ".name", ""), - config.getString("http_options_response." + config_key + ".value", "")); + response.add(config.getString("http_options_response." + config_key + ".name", ""), + config.getString("http_options_response." + config_key + ".value", "")); + } } return true; @@ -199,37 +199,54 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_UNAUTHORIZED; } - else if ( - exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD - || exception_code == ErrorCodes::AUTHENTICATION_FAILED || exception_code == ErrorCodes::SET_NON_GRANTED_ROLE) + else if (exception_code == ErrorCodes::UNKNOWN_USER || + exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::AUTHENTICATION_FAILED || + exception_code == ErrorCodes::SET_NON_GRANTED_ROLE) { return HTTPResponse::HTTP_FORBIDDEN; } - else if ( - exception_code == ErrorCodes::BAD_ARGUMENTS || exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP - || exception_code == ErrorCodes::CANNOT_PARSE_TEXT || exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE - || exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || exception_code == ErrorCodes::CANNOT_PARSE_DATE - || exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || exception_code == ErrorCodes::CANNOT_PARSE_NUMBER - || exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING || exception_code == ErrorCodes::CANNOT_PARSE_IPV4 - || exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED - || exception_code == ErrorCodes::CANNOT_PARSE_UUID || exception_code == ErrorCodes::DUPLICATE_COLUMN - || exception_code == ErrorCodes::ILLEGAL_COLUMN || exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST - || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || exception_code == ErrorCodes::THERE_IS_NO_COLUMN - || exception_code == ErrorCodes::TOO_DEEP_AST || exception_code == ErrorCodes::TOO_BIG_AST - || exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || exception_code == ErrorCodes::SYNTAX_ERROR - || exception_code == ErrorCodes::INCORRECT_DATA || exception_code == ErrorCodes::TYPE_MISMATCH - || exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) + else if (exception_code == ErrorCodes::BAD_ARGUMENTS || + exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP || + exception_code == ErrorCodes::CANNOT_PARSE_TEXT || + exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || + exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_DATE || + exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || + exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || + exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_IPV4 || + exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || + exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || + exception_code == ErrorCodes::CANNOT_PARSE_UUID || + exception_code == ErrorCodes::DUPLICATE_COLUMN || + exception_code == ErrorCodes::ILLEGAL_COLUMN || + exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || + exception_code == ErrorCodes::THERE_IS_NO_COLUMN || + exception_code == ErrorCodes::TOO_DEEP_AST || + exception_code == ErrorCodes::TOO_BIG_AST || + exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || + exception_code == ErrorCodes::SYNTAX_ERROR || + exception_code == ErrorCodes::INCORRECT_DATA || + exception_code == ErrorCodes::TYPE_MISMATCH || + exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) { return HTTPResponse::HTTP_BAD_REQUEST; } - else if ( - exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION - || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_TYPE - || exception_code == ErrorCodes::UNKNOWN_STORAGE || exception_code == ErrorCodes::UNKNOWN_DATABASE - || exception_code == ErrorCodes::UNKNOWN_SETTING || exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING - || exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || exception_code == ErrorCodes::UNKNOWN_FORMAT - || exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY - || exception_code == ErrorCodes::UNKNOWN_ROLE) + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + exception_code == ErrorCodes::UNKNOWN_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || + exception_code == ErrorCodes::UNKNOWN_TYPE || + exception_code == ErrorCodes::UNKNOWN_STORAGE || + exception_code == ErrorCodes::UNKNOWN_DATABASE || + exception_code == ErrorCodes::UNKNOWN_SETTING || + exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || + exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_FORMAT || + exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY || + exception_code == ErrorCodes::UNKNOWN_ROLE) { return HTTPResponse::HTTP_NOT_FOUND; } @@ -241,7 +258,8 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_NOT_IMPLEMENTED; } - else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || exception_code == ErrorCodes::CANNOT_OPEN_FILE) + else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || + exception_code == ErrorCodes::CANNOT_OPEN_FILE) { return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; } @@ -262,7 +280,9 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti } -static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util::AbstractConfiguration & config, const HTMLForm & params) +static std::chrono::steady_clock::duration parseSessionTimeout( + const Poco::Util::AbstractConfiguration & config, + const HTMLForm & params) { unsigned session_timeout = config.getInt("default_session_timeout", 60); @@ -276,17 +296,15 @@ static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util: throw Exception(ErrorCodes::INVALID_SESSION_TIMEOUT, "Invalid session timeout: '{}'", session_timeout_str); if (session_timeout > max_session_timeout) - throw Exception( - ErrorCodes::INVALID_SESSION_TIMEOUT, - "Session timeout '{}' is larger than max_session_timeout: {}. " + throw Exception(ErrorCodes::INVALID_SESSION_TIMEOUT, "Session timeout '{}' is larger than max_session_timeout: {}. " "Maximum session timeout could be modified in configuration file.", - session_timeout_str, - max_session_timeout); + session_timeout_str, max_session_timeout); } return std::chrono::seconds(session_timeout); } + void HTTPHandler::pushDelayedResults(Output & used_output) { std::vector write_buffers; @@ -338,7 +356,10 @@ HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTP HTTPHandler::~HTTPHandler() = default; -bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response) +bool HTTPHandler::authenticateUser( + HTTPServerRequest & request, + HTMLForm & params, + HTTPServerResponse & response) { using namespace Poco::Net; @@ -365,36 +386,31 @@ bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & param { /// It is prohibited to mix different authorization schemes. if (has_http_credentials) - throw Exception( - ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and Authorization HTTP header simultaneously"); + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and Authorization HTTP header simultaneously"); if (has_credentials_in_query_params) - throw Exception( - ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); if (has_ssl_certificate_auth) { #if USE_SSL if (!password.empty()) - throw Exception( - ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via password simultaneously"); + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and authentication via password simultaneously"); if (request.havePeerCertificate()) certificate_common_name = request.peerCertificate().commonName(); if (certificate_common_name.empty()) - throw Exception( - ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name"); + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name"); #else - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "SSL certificate authentication disabled because ClickHouse was built without SSL library"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "SSL certificate authentication disabled because ClickHouse was built without SSL library"); #endif } } @@ -402,10 +418,9 @@ bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & param { /// It is prohibited to mix different authorization schemes. if (has_credentials_in_query_params) - throw Exception( - ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use Authorization HTTP header and authentication via parameters simultaneously"); + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use Authorization HTTP header and authentication via parameters simultaneously"); std::string scheme; std::string auth_info; @@ -426,8 +441,7 @@ bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & param } else { - throw Exception( - ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); } } else @@ -453,8 +467,7 @@ bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & param auto * gss_acceptor_context = dynamic_cast(request_credentials.get()); if (!gss_acceptor_context) - throw Exception( - ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected"); + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected"); #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wunreachable-code" @@ -490,10 +503,9 @@ bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & param if (params.has("quota_key")) { if (!quota_key.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Invalid authentication: it is not allowed " - "to use quota key as HTTP header and as parameter simultaneously"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Invalid authentication: it is not allowed " + "to use quota key as HTTP header and as parameter simultaneously"); quota_key = params.get("quota_key"); } @@ -618,29 +630,26 @@ void HTTPHandler::processQuery( size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; bool enable_http_compression = params.getParsed("enable_http_compression", context->getSettingsRef().enable_http_compression); - Int64 http_zlib_compression_level - = params.getParsed("http_zlib_compression_level", context->getSettingsRef().http_zlib_compression_level); + Int64 http_zlib_compression_level = params.getParsed("http_zlib_compression_level", context->getSettingsRef().http_zlib_compression_level); - used_output.out_holder = std::make_shared( - response, - request.getMethod() == HTTPRequest::HTTP_HEAD, - context->getServerSettings().keep_alive_timeout.totalSeconds(), - write_event); + used_output.out_holder = + std::make_shared( + response, + request.getMethod() == HTTPRequest::HTTP_HEAD, + context->getServerSettings().keep_alive_timeout.totalSeconds(), + write_event); used_output.out = used_output.out_holder; used_output.out_maybe_compressed = used_output.out_holder; if (client_supports_http_compression && enable_http_compression) { used_output.out_holder->setCompressionMethodHeader(http_response_compression_method); - used_output.wrap_compressed_holder = wrapWriteBufferWithCompressionMethod( - used_output.out_holder.get(), - http_response_compression_method, - static_cast(http_zlib_compression_level), - 0, - DBMS_DEFAULT_BUFFER_SIZE, - nullptr, - 0, - false); + used_output.wrap_compressed_holder = + wrapWriteBufferWithCompressionMethod( + used_output.out_holder.get(), + http_response_compression_method, + static_cast(http_zlib_compression_level), + 0, DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0, false); used_output.out = used_output.wrap_compressed_holder; } @@ -664,13 +673,15 @@ void HTTPHandler::processQuery( { auto tmp_data = std::make_shared(server.context()->getTempDataOnDisk()); - auto create_tmp_disk_buffer = [tmp_data](const WriteBufferPtr &) -> WriteBufferPtr { return tmp_data->createRawStream(); }; + auto create_tmp_disk_buffer = [tmp_data] (const WriteBufferPtr &) -> WriteBufferPtr { + return tmp_data->createRawStream(); + }; cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); } else { - auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed](const WriteBufferPtr & prev_buf) + auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed] (const WriteBufferPtr & prev_buf) { auto * prev_memory_buffer = typeid_cast(prev_buf.get()); if (!prev_memory_buffer) @@ -685,8 +696,7 @@ void HTTPHandler::processQuery( cascade_buffer2.emplace_back(push_memory_buffer_and_continue); } - used_output.out_delayed_and_compressed_holder - = std::make_unique(std::move(cascade_buffer1), std::move(cascade_buffer2)); + used_output.out_delayed_and_compressed_holder = std::make_unique(std::move(cascade_buffer1), std::move(cascade_buffer2)); used_output.out_maybe_delayed_and_compressed = used_output.out_delayed_and_compressed_holder.get(); } else @@ -699,8 +709,7 @@ void HTTPHandler::processQuery( int zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); auto in_post = wrapReadBufferWithCompressionMethod( wrapReadBufferReference(request.getStream()), - chooseCompressionMethod({}, http_request_compression_method_str), - zstd_window_log_max); + chooseCompressionMethod({}, http_request_compression_method_str), zstd_window_log_max); /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. @@ -708,8 +717,7 @@ void HTTPHandler::processQuery( bool is_in_post_compressed = false; if (params.getParsed("decompress", false)) { - in_post_maybe_compressed - = std::make_unique(*in_post, /* allow_different_codecs_ = */ false, /* external_data_ = */ true); + in_post_maybe_compressed = std::make_unique(*in_post, /* allow_different_codecs_ = */ false, /* external_data_ = */ true); is_in_post_compressed = true; } else @@ -717,26 +725,12 @@ void HTTPHandler::processQuery( std::unique_ptr in; - static const NameSet reserved_param_names{ - "compress", - "decompress", - "user", - "password", - "quota_key", - "query_id", - "stacktrace", - "role", - "buffer_size", - "wait_end_of_query", - "session_id", - "session_timeout", - "session_check", - "client_protocol_version", - "close_session"}; + static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "role", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session"}; Names reserved_param_suffixes; - auto param_could_be_skipped = [&](const String & name) + auto param_could_be_skipped = [&] (const String & name) { /// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience. if (name.empty()) @@ -746,8 +740,10 @@ void HTTPHandler::processQuery( return true; for (const String & suffix : reserved_param_suffixes) + { if (endsWith(name, suffix)) return true; + } return false; }; @@ -865,34 +861,35 @@ void HTTPHandler::processQuery( if (settings.add_http_cors_header && !request.get("Origin", "").empty() && !config.has("http_options_response")) used_output.out_holder->addHeaderCORS(true); - auto append_callback = [my_context = context](ProgressCallback callback) + auto append_callback = [my_context = context] (ProgressCallback callback) { auto prev = my_context->getProgressCallback(); - my_context->setProgressCallback( - [prev, callback](const Progress & progress) - { - if (prev) - prev(progress); + my_context->setProgressCallback([prev, callback] (const Progress & progress) + { + if (prev) + prev(progress); - callback(progress); - }); + callback(progress); + }); }; /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. /// Note that we add it unconditionally so the progress is available for `X-ClickHouse-Summary` - append_callback([&used_output](const Progress & progress) { used_output.out_holder->onProgress(progress); }); + append_callback([&used_output](const Progress & progress) + { + used_output.out_holder->onProgress(progress); + }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) { - append_callback( - [&context, &request](const Progress &) - { - /// Assume that at the point this method is called no one is reading data from the socket any more: - /// should be true for read-only queries. - if (!request.checkPeerConnected()) - context->killCurrentQuery(); - }); + append_callback([&context, &request](const Progress &) + { + /// Assume that at the point this method is called no one is reading data from the socket any more: + /// should be true for read-only queries. + if (!request.checkPeerConnected()) + context->killCurrentQuery(); + }); } customizeContext(request, context, *in_post_maybe_compressed); @@ -900,7 +897,7 @@ void HTTPHandler::processQuery( applyHTTPResponseHeaders(response, http_response_headers_override); - auto set_query_result = [this, &response](const QueryResultDetails & details) + auto set_query_result = [&response, this] (const QueryResultDetails & details) { response.add("X-ClickHouse-Query-Id", details.query_id); @@ -915,10 +912,7 @@ void HTTPHandler::processQuery( response.add("X-ClickHouse-Timezone", *details.timezone); }; - auto handle_exception_in_output_format = [&](IOutputFormat & current_output_format, - const String & format_name, - const ContextPtr & context_, - const std::optional & format_settings) + auto handle_exception_in_output_format = [&](IOutputFormat & current_output_format, const String & format_name, const ContextPtr & context_, const std::optional & format_settings) { if (settings.http_write_exception_in_output_format && current_output_format.supportsWritingException()) { @@ -938,8 +932,7 @@ void HTTPHandler::processQuery( } else { - bool with_stacktrace - = (params.getParsed("stacktrace", false) && server.config().getBool("enable_http_stacktrace", true)); + bool with_stacktrace = (params.getParsed("stacktrace", false) && server.config().getBool("enable_http_stacktrace", true)); ExecutionStatus status = ExecutionStatus::fromCurrentException("", with_stacktrace); formatExceptionForClient(status.code, request, response, used_output); current_output_format.setException(status.message); @@ -980,8 +973,7 @@ try if (!used_output.out_holder && !used_output.exception_is_written) { /// If nothing was sent yet and we don't even know if we must compress the response. - WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT) - .writeln(s); + WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT).writeln(s); } else if (used_output.out_maybe_compressed) { @@ -1045,8 +1037,7 @@ catch (...) } } -void HTTPHandler::formatExceptionForClient( - int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) +void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) { if (used_output.out_holder) used_output.out_holder->setExceptionCode(exception_code); @@ -1113,14 +1104,18 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse std::string opentelemetry_traceparent = request.get("traceparent"); std::string error; if (!client_trace_context.parseTraceparentHeader(opentelemetry_traceparent, error)) + { LOG_DEBUG(log, "Failed to parse OpenTelemetry traceparent header '{}': {}", opentelemetry_traceparent, error); + } client_trace_context.tracestate = request.get("tracestate", ""); } // Setup tracing context for this thread auto context = session->sessionOrGlobalContext(); - thread_trace_context = std::make_unique( - "HTTPHandler", client_trace_context, context->getSettingsRef(), context->getOpenTelemetrySpanLog()); + thread_trace_context = std::make_unique("HTTPHandler", + client_trace_context, + context->getSettingsRef(), + context->getOpenTelemetrySpanLog()); thread_trace_context->root_span.kind = OpenTelemetry::SpanKind::SERVER; thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI()); @@ -1149,10 +1144,9 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse /// Workaround. Poco does not detect 411 Length Required case. if (request.getMethod() == HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) { - throw Exception( - ErrorCodes::HTTP_LENGTH_REQUIRED, - "The Transfer-Encoding is not chunked and there " - "is no Content-Length header for POST request"); + throw Exception(ErrorCodes::HTTP_LENGTH_REQUIRED, + "The Transfer-Encoding is not chunked and there " + "is no Content-Length header for POST request"); } processQuery(request, params, response, used_output, query_scope, write_event); @@ -1164,8 +1158,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse catch (...) { SCOPE_EXIT({ - request_credentials - .reset(); // ...so that the next requests on the connection have to always start afresh in case of exceptions. + request_credentials.reset(); // ...so that the next requests on the connection have to always start afresh in case of exceptions. }); /// Check if exception was thrown in used_output.finalize(). @@ -1204,7 +1197,7 @@ DynamicQueryHandler::DynamicQueryHandler( bool DynamicQueryHandler::customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) { if (key == param_name) - return true; /// do nothing + return true; /// do nothing if (startsWith(key, QUERY_PARAMETER_NAME_PREFIX)) { @@ -1238,10 +1231,16 @@ std::string DynamicQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm std::string full_query; /// Params are of both form params POST and uri (GET params) for (const auto & it : params) + { if (it.first == param_name) + { full_query += it.second; + } else + { customizeQueryParam(context, it.first, it.second); + } + } return full_query; } @@ -1339,8 +1338,9 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo return predefined_query; } -HTTPRequestHandlerFactoryPtr -createDynamicHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix) { auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query"); @@ -1357,14 +1357,11 @@ createDynamicHandlerFactory(IServer & server, const Poco::Util::AbstractConfigur static inline bool capturingNamedQueryParam(NameSet receive_params, const CompiledRegexPtr & compiled_regex) { const auto & capturing_names = compiled_regex->NamedCapturingGroups(); - return std::count_if( - capturing_names.begin(), - capturing_names.end(), - [&](const auto & iterator) - { - return std::count_if( - receive_params.begin(), receive_params.end(), [&](const auto & param_name) { return param_name == iterator.first; }); - }); + return std::count_if(capturing_names.begin(), capturing_names.end(), [&](const auto & iterator) + { + return std::count_if(receive_params.begin(), receive_params.end(), + [&](const auto & param_name) { return param_name == iterator.first; }); + }); } static inline CompiledRegexPtr getCompiledRegex(const std::string & expression) @@ -1372,18 +1369,15 @@ static inline CompiledRegexPtr getCompiledRegex(const std::string & expression) auto compiled_regex = std::make_shared(expression); if (!compiled_regex->ok()) - throw Exception( - ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile re2: {} for http handling rule, error: {}. " - "Look at https://github.com/google/re2/wiki/Syntax for reference.", - expression, - compiled_regex->error()); + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile re2: {} for http handling rule, error: {}. " + "Look at https://github.com/google/re2/wiki/Syntax for reference.", expression, compiled_regex->error()); return compiled_regex; } -HTTPRequestHandlerFactoryPtr -createPredefinedHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix) { if (!config.has(config_prefix + ".handler.query")) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no path '{}.handler.query' in configuration file.", config_prefix); @@ -1422,12 +1416,18 @@ createPredefinedHandlerFactory(IServer & server, const Poco::Util::AbstractConfi auto regex = getCompiledRegex(url_expression); if (capturingNamedQueryParam(analyze_receive_params, regex)) { - auto creator - = [&server, analyze_receive_params, predefined_query, regex, headers_name_with_regex, http_response_headers_override] + auto creator = [ + &server, + analyze_receive_params, + predefined_query, + regex, + headers_name_with_regex, + http_response_headers_override] -> std::unique_ptr { return std::make_unique( - server, analyze_receive_params, predefined_query, regex, headers_name_with_regex, http_response_headers_override); + server, analyze_receive_params, predefined_query, regex, + headers_name_with_regex, http_response_headers_override); }; factory = std::make_shared>(std::move(creator)); factory->addFiltersFromConfig(config, config_prefix); @@ -1435,11 +1435,17 @@ createPredefinedHandlerFactory(IServer & server, const Poco::Util::AbstractConfi } } - auto creator = [&server, analyze_receive_params, predefined_query, headers_name_with_regex, http_response_headers_override] + auto creator = [ + &server, + analyze_receive_params, + predefined_query, + headers_name_with_regex, + http_response_headers_override] -> std::unique_ptr { return std::make_unique( - server, analyze_receive_params, predefined_query, CompiledRegexPtr{}, headers_name_with_regex, http_response_headers_override); + server, analyze_receive_params, predefined_query, CompiledRegexPtr{}, + headers_name_with_regex, http_response_headers_override); }; factory = std::make_shared>(std::move(creator)); diff --git a/src/Server/HTTPHandler.h b/src/Server/HTTPHandler.h index 1bf1dbdebf8..c5551102f7a 100644 --- a/src/Server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -3,27 +3,24 @@ #include #include #include -#include #include -#include #include #include #include #include #include +#include +#include #include #include "HTTPResponseHeaderWriter.h" namespace CurrentMetrics { -extern const Metric HTTPConnection; + extern const Metric HTTPConnection; } -namespace Poco -{ -class Logger; -} +namespace Poco { class Logger; } namespace DB { @@ -45,7 +42,7 @@ public: void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; /// This method is called right before the query execution. - virtual void customizeContext(HTTPServerRequest & /* request */, ContextMutablePtr /* context */, ReadBuffer & /* body */) { } + virtual void customizeContext(HTTPServerRequest & /* request */, ContextMutablePtr /* context */, ReadBuffer & /* body */) {} virtual bool customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) = 0; @@ -85,7 +82,10 @@ private: bool exception_is_written = false; std::function exception_writer; - bool hasDelayed() const { return out_maybe_delayed_and_compressed != out_maybe_compressed.get(); } + bool hasDelayed() const + { + return out_maybe_delayed_and_compressed != out_maybe_compressed.get(); + } void finalize() { @@ -99,7 +99,10 @@ private: out->finalize(); } - bool isFinalized() const { return finalized; } + bool isFinalized() const + { + return finalized; + } }; IServer & server; @@ -130,7 +133,10 @@ private: // Returns false when the user is not authenticated yet, and the 'Negotiate' response is sent, // the session and request_credentials instances are preserved. // Throws an exception if authentication failed. - bool authenticateUser(HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response); + bool authenticateUser( + HTTPServerRequest & request, + HTMLForm & params, + HTTPServerResponse & response); /// Also initializes 'used_output'. void processQuery( @@ -142,9 +148,17 @@ private: const ProfileEvents::Event & write_event); void trySendExceptionToClient( - const std::string & s, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output); + const std::string & s, + int exception_code, + HTTPServerRequest & request, + HTTPServerResponse & response, + Output & used_output); - void formatExceptionForClient(int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output); + void formatExceptionForClient( + int exception_code, + HTTPServerRequest & request, + HTTPServerResponse & response, + Output & used_output); static void pushDelayedResults(Output & used_output); }; @@ -162,7 +176,7 @@ public: std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override; - bool customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) override; + bool customizeQueryParam(ContextMutablePtr context, const std::string &key, const std::string &value) override; }; class PredefinedQueryHandler : public HTTPHandler diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index d125e08c704..5344b2d024b 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -1,18 +1,18 @@ #include #include -#include #include #include +#include #include #include "HTTPHandler.h" -#include "InterserverIOHTTPHandler.h" -#include "PrometheusRequestHandler.h" -#include "ReplicasStatusHandler.h" #include "Server/PrometheusMetricsWriter.h" #include "StaticRequestHandler.h" +#include "ReplicasStatusHandler.h" +#include "InterserverIOHTTPHandler.h" +#include "PrometheusRequestHandler.h" #include "WebUIRequestHandler.h" @@ -21,9 +21,9 @@ namespace DB namespace ErrorCodes { -extern const int LOGICAL_ERROR; -extern const int UNKNOWN_ELEMENT_IN_CONFIG; -extern const int INVALID_CONFIG_PARAMETER; + extern const int LOGICAL_ERROR; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int INVALID_CONFIG_PARAMETER; } namespace @@ -35,7 +35,10 @@ private: std::string url; public: - explicit RedirectRequestHandler(std::string url_) : url(std::move(url_)) { } + explicit RedirectRequestHandler(std::string url_) + : url(std::move(url_)) + { + } void handleRequest(HTTPServerRequest &, HTTPServerResponse & response, const ProfileEvents::Event &) override { @@ -43,8 +46,9 @@ public: } }; -HTTPRequestHandlerFactoryPtr -createRedirectHandlerFactory(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +HTTPRequestHandlerFactoryPtr createRedirectHandlerFactory( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix) { std::string url = config.getString(config_prefix + ".handler.location"); @@ -99,12 +103,8 @@ static inline auto createHandlersFactoryFromConfig( const auto & handler_type = config.getString(prefix + "." + key + ".handler.type", ""); if (handler_type.empty()) - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, - "Handler type in config is not specified here: " - "{}.{}.handler.type", - prefix, - key); + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Handler type in config is not specified here: " + "{}.{}.handler.type", prefix, key); if (handler_type == "static") { @@ -155,27 +155,19 @@ static inline auto createHandlersFactoryFromConfig( main_handler_factory->addHandler(std::move(handler)); } else - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, - "Unknown handler type '{}' in config here: {}.{}.handler.type", - handler_type, - prefix, - key); + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Unknown handler type '{}' in config here: {}.{}.handler.type", + handler_type, prefix, key); } else - throw Exception( - ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, - "Unknown element in config: " - "{}.{}, must be 'rule' or 'defaults'", - prefix, - key); + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown element in config: " + "{}.{}, must be 'rule' or 'defaults'", prefix, key); } return main_handler_factory; } -static inline HTTPRequestHandlerFactoryPtr createHTTPHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & name, AsynchronousMetrics & async_metrics) +static inline HTTPRequestHandlerFactoryPtr +createHTTPHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & name, AsynchronousMetrics & async_metrics) { if (config.has("http_handlers")) { @@ -202,8 +194,7 @@ static inline HTTPRequestHandlerFactoryPtr createInterserverHTTPHandlerFactory(I } -HTTPRequestHandlerFactoryPtr createHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & name) +HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & name) { if (name == "HTTPHandler-factory" || name == "HTTPSHandler-factory") return createHTTPHandlerFactory(server, config, name, async_metrics); @@ -276,23 +267,28 @@ void addDefaultHandlersFactory( { addCommonDefaultHandlersFactory(factory, server); - auto dynamic_creator - = [&server]() -> std::unique_ptr { return std::make_unique(server, "query"); }; + auto dynamic_creator = [&server] () -> std::unique_ptr + { + return std::make_unique(server, "query"); + }; auto query_handler = std::make_shared>(std::move(dynamic_creator)); - query_handler->addFilter( - [](const auto & request) + query_handler->addFilter([](const auto & request) { - bool path_matches_get_or_head - = startsWith(request.getURI(), "?") || startsWith(request.getURI(), "/?") || startsWith(request.getURI(), "/query?"); - bool is_get_or_head_request - = request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; + bool path_matches_get_or_head = startsWith(request.getURI(), "?") + || startsWith(request.getURI(), "/?") + || startsWith(request.getURI(), "/query?"); + bool is_get_or_head_request = request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; - bool path_matches_post_or_options = path_matches_get_or_head || request.getURI() == "/" || request.getURI().empty(); - bool is_post_or_options_request - = request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS; + bool path_matches_post_or_options = path_matches_get_or_head + || request.getURI() == "/" + || request.getURI().empty(); + bool is_post_or_options_request = request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS; return (path_matches_get_or_head && is_get_or_head_request) || (path_matches_post_or_options && is_post_or_options_request); - }); + } + ); factory.addHandler(query_handler); /// We check that prometheus handler will be served on current (default) port. @@ -300,8 +296,10 @@ void addDefaultHandlersFactory( if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) { auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator = [&server, writer]() -> std::unique_ptr - { return std::make_unique(server, writer); }; + auto creator = [&server, writer] () -> std::unique_ptr + { + return std::make_unique(server, writer); + }; auto prometheus_handler = std::make_shared>(std::move(creator)); prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); prometheus_handler->allowGetAndHeadRequest(); diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index e320507fc66..331b7a84857 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -7,19 +7,19 @@ #include #include #include -#include #include -#include +#include #include +#include #include -#include #include -#include #include #include +#include #include +#include namespace fs = std::filesystem; @@ -29,16 +29,15 @@ namespace DB namespace ErrorCodes { -extern const int INCORRECT_FILE_NAME; -extern const int HTTP_LENGTH_REQUIRED; -extern const int INVALID_CONFIG_PARAMETER; + extern const int INCORRECT_FILE_NAME; + extern const int HTTP_LENGTH_REQUIRED; + extern const int INVALID_CONFIG_PARAMETER; } static inline std::unique_ptr responseWriteBuffer(HTTPServerRequest & request, HTTPServerResponse & response, UInt64 keep_alive_timeout) { - auto buf = std::unique_ptr( - new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout)); + auto buf = std::unique_ptr(new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout)); /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -63,8 +62,8 @@ static inline void trySendExceptionToClient( /// If HTTP method is POST and Keep-Alive is turned on, we should 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() && !request.getStream().eof() - && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + && response.getKeepAlive() && !request.getStream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) request.getStream().ignore(std::numeric_limits::max()); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); @@ -89,8 +88,7 @@ static inline void trySendExceptionToClient( } } -void StaticRequestHandler::handleRequest( - HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/) +void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/) { auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds(); auto out = responseWriteBuffer(request, response, keep_alive_timeout); @@ -103,12 +101,10 @@ void StaticRequestHandler::handleRequest( response.setChunkedTransferEncoding(true); /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() - && !request.hasContentLength()) - throw Exception( - ErrorCodes::HTTP_LENGTH_REQUIRED, - "The Transfer-Encoding is not chunked and there " - "is no Content-Length header for POST request"); + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) + throw Exception(ErrorCodes::HTTP_LENGTH_REQUIRED, + "The Transfer-Encoding is not chunked and there " + "is no Content-Length header for POST request"); setResponseDefaultHeaders(response, keep_alive_timeout); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status)); @@ -149,10 +145,9 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) else if (startsWith(response_expression, config_prefix)) { if (response_expression.size() <= config_prefix.size()) - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, - "Static handling rule handler must contain a complete configuration path, for example: " - "config://config_key"); + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, + "Static handling rule handler must contain a complete configuration path, for example: " + "config://config_key"); const auto & config_path = response_expression.substr(config_prefix.size(), response_expression.size() - config_prefix.size()); writeString(server.config().getRawString(config_path, "Ok.\n"), out); @@ -167,8 +162,9 @@ StaticRequestHandler::StaticRequestHandler( { } -HTTPRequestHandlerFactoryPtr -createStaticHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix) { int status = config.getInt(config_prefix + ".handler.status", 200); std::string response_content = config.getRawString(config_prefix + ".handler.response_content", "Ok.\n"); From b74f910aaf9b1a05fd909923afb7714d4070532d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Jun 2024 12:22:41 +0200 Subject: [PATCH 623/856] Do the replacement of the arguments --- src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp index 2360cd3f0c2..8a6276008d8 100644 --- a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp @@ -85,6 +85,7 @@ public: if (!aggregate_function->getResultType()->equals(*function_node->getAggregateFunction()->getResultType())) return; + function_node->getArguments().getNodes() = replaced_uniq_function_arguments_nodes; function_node->resolveAsAggregateFunction(std::move(aggregate_function)); } }; From 98d92b3be23d13d3d5f0d1180483d2f444acad78 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Jun 2024 12:54:31 +0200 Subject: [PATCH 624/856] Fix the descriptions on some server settings --- src/Core/ServerSettings.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 45f235116ab..47ea5c29a50 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -97,11 +97,11 @@ namespace DB \ M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ - M(UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0) \ - M(UInt64, max_view_num_to_warn, 10000lu, "If number of views is greater than this value, server will create a warning that will displayed to user.", 0) \ - M(UInt64, max_dictionary_num_to_warn, 1000lu, "If number of dictionaries is greater than this value, server will create a warning that will displayed to user.", 0) \ - M(UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \ - M(UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \ + M(UInt64, max_table_num_to_warn, 5000lu, "If the number of tables is greater than this value, the server will create a warning that will displayed to user.", 0) \ + M(UInt64, max_view_num_to_warn, 10000lu, "If the number of views is greater than this value, the server will create a warning that will displayed to user.", 0) \ + 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, 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) \ \ From 2d37f7b530a34e32e8d6debd9a0c02393b81fe04 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 13 Jun 2024 12:43:25 +0300 Subject: [PATCH 625/856] Initial commit --- src/Coordination/KeeperStorage.cpp | 14 ++++- src/Coordination/KeeperStorage.h | 8 ++- src/Coordination/tests/gtest_coordination.cpp | 60 +++++++++++++++++++ 3 files changed, 79 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 9bcd0608bf7..5920e098470 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -609,7 +609,10 @@ void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) uncommitted_auth.pop_front(); if (uncommitted_auth.empty()) session_and_auth.erase(add_auth->session_id); - + } + else if (auto * close_session = std::get_if(&front_delta.operation)) + { + closed_sessions.erase(close_session->session_id); } deltas.pop_front(); @@ -682,6 +685,10 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) session_and_auth.erase(add_auth->session_id); } } + else if (auto * close_session = std::get_if(&delta_it->operation)) + { + closed_sessions.erase(close_session->session_id); + } } if (delta_it == deltas.rend()) @@ -2366,12 +2373,15 @@ void KeeperStorage::preprocessRequest( ephemerals.erase(session_ephemerals); } + new_deltas.emplace_back(transaction.zxid, CloseSessionDelta{session_id}); + uncommitted_state.closed_sessions.insert(session_id); new_digest = calculateNodesDigest(new_digest, new_deltas); return; } - if (check_acl && !request_processor->checkAuth(*this, session_id, false)) + if ((check_acl && !request_processor->checkAuth(*this, session_id, false)) || + uncommitted_state.closed_sessions.contains(session_id)) // Is session closed but not committed yet { uncommitted_state.deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); return; diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index d9e67f799f8..d5e9a64e69c 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -314,8 +314,13 @@ public: AuthID auth_id; }; + struct CloseSessionDelta + { + int64_t session_id; + }; + using Operation = std:: - variant; + variant; struct Delta { @@ -351,6 +356,7 @@ public: std::shared_ptr tryGetNodeFromStorage(StringRef path) const; std::unordered_map> session_and_auth; + std::unordered_set closed_sessions; struct UncommittedNode { diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index d314757efc9..30a0eea3040 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2019,6 +2019,66 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte EXPECT_EQ(acls[0].permissions, 31); } +TEST_P(CoordinationTest, TestPreprocessWhenCloseSessionIsPrecommitted) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + int64_t session_id = 1; + size_t term = 0; + + auto state_machine = std::make_shared(queue, snapshots_queue, keeper_context, nullptr); + state_machine->init(); + + auto & storage = state_machine->getStorageUnsafe(); + const auto & uncommitted_state = storage.uncommitted_state; + + // Create first node for the session + String node_path_1 = "/node_1"; + std::shared_ptr create_req_1 = std::make_shared(); + create_req_1->path = node_path_1; + auto create_entry_1 = getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), create_req_1); + + state_machine->pre_commit(1, create_entry_1->get_buf()); + EXPECT_TRUE(uncommitted_state.nodes.contains(node_path_1)); + state_machine->commit(1, create_entry_1->get_buf()); + EXPECT_TRUE(storage.container.contains(node_path_1)); + + // Close session + std::shared_ptr close_req = std::make_shared(); + auto close_entry = getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), close_req); + // Pre-commit close session + state_machine->pre_commit(2, close_entry->get_buf()); + + // Try to create second node after close session is pre-committed + String node_path_2 = "/node_2"; + std::shared_ptr create_req_2 = std::make_shared(); + create_req_2->path = node_path_2; + auto create_entry_2 = getLogEntryFromZKRequest(term, session_id, state_machine->getNextZxid(), create_req_2); + + // Pre-commit creating second node + state_machine->pre_commit(3, create_entry_2->get_buf()); + // Second node wasn't created + EXPECT_FALSE(uncommitted_state.nodes.contains(node_path_2)); + + // Rollback pre-committed closing session + state_machine->rollback(3, create_entry_2->get_buf()); + state_machine->rollback(2, close_entry->get_buf()); + + // Pre-commit creating second node + state_machine->pre_commit(2, create_entry_2->get_buf()); + // Now second node was created + EXPECT_TRUE(uncommitted_state.nodes.contains(node_path_2)); + + state_machine->commit(2, create_entry_2->get_buf()); + EXPECT_TRUE(storage.container.contains(node_path_1)); + EXPECT_TRUE(storage.container.contains(node_path_2)); +} + TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) { using namespace Coordination; From 55319c760deecd2d63a4605c705e298f8290ebb8 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 10 Jun 2024 17:34:09 +0200 Subject: [PATCH 626/856] stateless tests: add test for https://github.com/ClickHouse/ClickHouse/issues/42083 --- .../03169_modify_column_data_loss.reference | 4 ++++ .../03169_modify_column_data_loss.sql | 19 +++++++++++++++++++ 2 files changed, 23 insertions(+) create mode 100644 tests/queries/0_stateless/03169_modify_column_data_loss.reference create mode 100644 tests/queries/0_stateless/03169_modify_column_data_loss.sql diff --git a/tests/queries/0_stateless/03169_modify_column_data_loss.reference b/tests/queries/0_stateless/03169_modify_column_data_loss.reference new file mode 100644 index 00000000000..2126a658c16 --- /dev/null +++ b/tests/queries/0_stateless/03169_modify_column_data_loss.reference @@ -0,0 +1,4 @@ +1 one 0 +2 two 0 +3 \N 0 +1 one 1 0 diff --git a/tests/queries/0_stateless/03169_modify_column_data_loss.sql b/tests/queries/0_stateless/03169_modify_column_data_loss.sql new file mode 100644 index 00000000000..def0a25a1b4 --- /dev/null +++ b/tests/queries/0_stateless/03169_modify_column_data_loss.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS column_modify_test; + +CREATE TABLE column_modify_test (id UInt64, val String, other_col UInt64) engine=MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part=0; +INSERT INTO column_modify_test VALUES (1,'one',0); +INSERT INTO column_modify_test VALUES (2,'two',0); + +-- on 21.9 that was done via mutations mechanism +ALTER TABLE column_modify_test MODIFY COLUMN val Nullable(String); + +INSERT INTO column_modify_test VALUES (3,Null,0); + +-- till now everythings looks ok +SELECT * FROM column_modify_test order by id, val, other_col; + +-- Now we do mutation. It will affect one of the parts, and will update columns.txt to the latest / correct state w/o updating the column file! +alter table column_modify_test update other_col=1 where id = 1 SETTINGS mutations_sync=1; + +-- row 1 is damaged now the column file & columns.txt is out of sync! +SELECT *, throwIf(val <> 'one') as issue FROM column_modify_test WHERE id = 1; From 80575fe122a75b4914e9e94fa558e2250c68b456 Mon Sep 17 00:00:00 2001 From: Grigorii Sokolik Date: Thu, 13 Jun 2024 14:15:19 +0300 Subject: [PATCH 627/856] Fix CI failure --- src/Server/HTTPResponseHeaderWriter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/HTTPResponseHeaderWriter.cpp b/src/Server/HTTPResponseHeaderWriter.cpp index f5ab196c4b0..fd29af5bdc7 100644 --- a/src/Server/HTTPResponseHeaderWriter.cpp +++ b/src/Server/HTTPResponseHeaderWriter.cpp @@ -56,13 +56,13 @@ std::unordered_map parseHTTPResponseHeaders(const std::string & void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup) { if (setup) - for (auto [header_name, header_value] : *setup) + for (const auto & [header_name, header_value] : *setup) response.set(header_name, header_value); } void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map & setup) { - for (auto [header_name, header_value] : setup) + for (const auto & [header_name, header_value] : setup) response.set(header_name, header_value); } From 2de54fa4d304ea1db42e7c4cc608bbc236b39d93 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 13 Jun 2024 12:21:49 +0100 Subject: [PATCH 628/856] 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 f4493d2544d5453cead162f776969b22ec409763 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 10 Jun 2024 18:33:00 +0200 Subject: [PATCH 629/856] stateless tests: add test for https://github.com/ClickHouse/ClickHouse/issues/59094 --- ...3170_part_offset_as_table_column.reference | 30 +++++++++++++++++++ .../03170_part_offset_as_table_column.sql | 25 ++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 tests/queries/0_stateless/03170_part_offset_as_table_column.reference create mode 100644 tests/queries/0_stateless/03170_part_offset_as_table_column.sql diff --git a/tests/queries/0_stateless/03170_part_offset_as_table_column.reference b/tests/queries/0_stateless/03170_part_offset_as_table_column.reference new file mode 100644 index 00000000000..435187cb39b --- /dev/null +++ b/tests/queries/0_stateless/03170_part_offset_as_table_column.reference @@ -0,0 +1,30 @@ +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 diff --git a/tests/queries/0_stateless/03170_part_offset_as_table_column.sql b/tests/queries/0_stateless/03170_part_offset_as_table_column.sql new file mode 100644 index 00000000000..36cbc156744 --- /dev/null +++ b/tests/queries/0_stateless/03170_part_offset_as_table_column.sql @@ -0,0 +1,25 @@ +CREATE TABLE test_table +( + `key` UInt32, + `_part_offset` DEFAULT 0 +) +ENGINE = MergeTree +ORDER BY key; + +INSERT INTO test_table (key) SELECT number +FROM numbers(10); + +set allow_experimental_analyzer=0; + +SELECT * +FROM test_table; + +set allow_experimental_analyzer=1; + +SELECT * +FROM test_table; + +SELECT + key, + _part_offset +FROM test_table; From 3f0211a2f88c3f8ebae75f7cb39469b8769c86d1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 13 Jun 2024 13:49:47 +0200 Subject: [PATCH 630/856] Update s3queue.md --- 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 f72bc79c1e5..0958680dc56 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -183,7 +183,7 @@ Default value: `30000`. ### s3queue_buckets {#buckets} -For 'Ordered' mode. Available since `24.6`. If there are several replicas of S3Queue table, each working with the same metadata directory in keeper, the value of `s3queue_buckets` needs to be equal to at least the number of replicas. If `s3queue_processing_threads` setting is used as well, it makes sense to increase the value of `s3queue_buckets` setting even futher, as it defines the actual parallelism of `S3Queue` processing. +For 'Ordered' mode. Available since `24.6`. If there are several replicas of S3Queue table, each working with the same metadata directory in keeper, the value of `s3queue_buckets` needs to be equal to at least the number of replicas. If `s3queue_processing_threads` setting is used as well, it makes sense to increase the value of `s3queue_buckets` setting even further, as it defines the actual parallelism of `S3Queue` processing. ## S3-related Settings {#s3-settings} From d364f447a780a75fbe9ad0c1395812ae48355f78 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 13 Jun 2024 14:13:39 +0200 Subject: [PATCH 631/856] Fix spacing issue and update reference file --- .../Formats/Impl/PrettySpaceBlockOutputFormat.cpp | 11 +++++++---- .../03169_display_column_names_in_footer.reference | 4 ---- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 4a07d0f808c..0a594b54b12 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -36,7 +36,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port if (format_settings.pretty.output_format_pretty_row_numbers) writeString(String(row_number_width, ' '), out); /// Names - auto write_names = [&]() -> void + auto write_names = [&](const bool is_footer) -> void { for (size_t i = 0; i < num_columns; ++i) { @@ -70,9 +70,12 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeChar(' ', out); } } - writeCString("\n\n", out); + if (!is_footer) + writeCString("\n\n", out); + else + writeCString("\n", out); }; - write_names(); + write_names(false); for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row) { @@ -111,7 +114,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeString(String(row_number_width, ' '), out); /// Write footer if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names) - write_names(); + write_names(true); total_rows += num_rows; } diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference index 426b7798090..edf0c9e4133 100644 --- a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference @@ -2068,7 +2068,6 @@ 100. 99 UInt64 1 number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) 1. 0 UInt64 0 @@ -2173,7 +2172,6 @@ 100. 99 UInt64 1 number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) 1. 0 UInt64 0 @@ -2278,7 +2276,6 @@ 100. 99 UInt64 1 number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) 1. 0 UInt64 0 @@ -2383,4 +2380,3 @@ 100. 99 UInt64 1 number toTypeName(number) mod(number, 2) - From 64f8fedc7a11302015b946d9cc65ec5672e819ff Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 Jun 2024 14:23:32 +0200 Subject: [PATCH 632/856] Fix test --- src/Backups/BackupIO_S3.cpp | 24 ++++++++++++------------ src/IO/S3Settings.cpp | 15 +++++++++++++++ src/IO/S3Settings.h | 7 +++++++ 3 files changed, 34 insertions(+), 12 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 273d4b4ebe8..56544312c26 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -128,13 +128,13 @@ BackupReaderS3::BackupReaderS3( , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} { - auto endpoint_settings = context_->getStorageS3Settings().getSettings( - s3_uri.uri.toString(), - context_->getUserName(), - /*ignore_user=*/is_internal_backup); + s3_settings.loadFromConfig(context_->getConfigRef(), "s3", context_->getSettingsRef()); - if (endpoint_settings.has_value()) - s3_settings = endpoint_settings.value(); + if (auto endpoint_settings = context_->getStorageS3Settings().getSettings( + s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup)) + { + s3_settings.updateIfChanged(*endpoint_settings); + } s3_settings.request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); s3_settings.request_settings.allow_native_copy = allow_s3_native_copy; @@ -226,13 +226,13 @@ BackupWriterS3::BackupWriterS3( , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} { - auto endpoint_settings = context_->getStorageS3Settings().getSettings( - s3_uri.uri.toString(), - context_->getUserName(), - /*ignore_user=*/is_internal_backup); + s3_settings.loadFromConfig(context_->getConfigRef(), "s3", context_->getSettingsRef()); - if (endpoint_settings.has_value()) - s3_settings = endpoint_settings.value(); + if (auto endpoint_settings = context_->getStorageS3Settings().getSettings( + s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup)) + { + s3_settings.updateIfChanged(*endpoint_settings); + } s3_settings.request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); s3_settings.request_settings.allow_native_copy = allow_s3_native_copy; diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp index e88d8133c32..a5a50c873cb 100644 --- a/src/IO/S3Settings.cpp +++ b/src/IO/S3Settings.cpp @@ -8,6 +8,21 @@ namespace DB { +void S3Settings::loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings) +{ + auth_settings = S3::AuthSettings(config, settings, config_prefix); + request_settings = S3::RequestSettings(config, settings, config_prefix); +} + +void S3Settings::updateIfChanged(const S3Settings & settings) +{ + auth_settings.updateIfChanged(settings.auth_settings); + request_settings.updateIfChanged(settings.request_settings); +} + void S3SettingsByEndpoint::loadFromConfig( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, diff --git a/src/IO/S3Settings.h b/src/IO/S3Settings.h index 58e12e48002..9eed0a5652f 100644 --- a/src/IO/S3Settings.h +++ b/src/IO/S3Settings.h @@ -21,6 +21,13 @@ struct S3Settings { S3::AuthSettings auth_settings; S3::RequestSettings request_settings; + + void loadFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings); + + void updateIfChanged(const S3Settings & settings); }; class S3SettingsByEndpoint From 9a87d790327919c70a866b623024e4349c69d66d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Jun 2024 14:25:41 +0200 Subject: [PATCH 633/856] Fix bug in short circuit optimization with direct dictionaries --- src/Dictionaries/DirectDictionary.cpp | 3 +- ...71_direct_dict_short_circuit_bug.reference | 0 .../03171_direct_dict_short_circuit_bug.sql | 37 +++++++++++++++++++ 3 files changed, 38 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.reference create mode 100644 tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.sql diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 933ffa04069..2c0f7653aff 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -175,8 +175,7 @@ Columns DirectDictionary::getColumns( if (!mask_filled) (*default_mask)[requested_key_index] = 1; - Field value{}; - result_column->insert(value); + result_column->insertDefault(); } else { diff --git a/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.reference b/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.sql b/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.sql new file mode 100644 index 00000000000..034d48d2b8b --- /dev/null +++ b/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.sql @@ -0,0 +1,37 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/65201 +SET short_circuit_function_evaluation='enable'; + +DROP DICTIONARY IF EXISTS direct_dictionary_simple_key_simple_attributes; +DROP TABLE IF EXISTS simple_key_simple_attributes_source_table; + +CREATE TABLE simple_key_simple_attributes_source_table +( + id UInt64, + value_first String, + value_second String +) + ENGINE = TinyLog; + +INSERT INTO simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0'); +INSERT INTO simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1'); +INSERT INTO simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2'); + + +CREATE DICTIONARY direct_dictionary_simple_key_simple_attributes +( + `id` UInt64, + `value_first` String DEFAULT 'value_first_default', + `value_second` String DEFAULT 'value_second_default' +) + PRIMARY KEY id + SOURCE(CLICKHOUSE(TABLE 'simple_key_simple_attributes_source_table')) + LAYOUT(DIRECT()); + +SELECT + toUInt128(1), + dictGetOrDefault('direct_dictionary_simple_key_simple_attributes', 'value_second', number, toString(toFixedString(toFixedString(toFixedString(materialize(toNullable('default')), 7), 7), toNullable(toNullable(toNullable(toUInt128(7))))))) AS value_second +FROM system.numbers LIMIT 255 +FORMAT Null; + +DROP DICTIONARY IF EXISTS direct_dictionary_simple_key_simple_attributes; +DROP TABLE IF EXISTS simple_key_simple_attributes_source_table; From cb9ea78810e1ccb894fae0831c299a7e5958f3af Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 08:13:51 +0000 Subject: [PATCH 634/856] 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 fe378edb350e3c45d00e765fd3b024abce05f24b Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 08:13:51 +0000 Subject: [PATCH 635/856] 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 1172b324e5ef3dcdbb751b066a546f4f0a81564d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 Jun 2024 13:08:30 +0000 Subject: [PATCH 636/856] Update test. --- .../0_stateless/03165_storage_merge_view_prewhere.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference index 3ee56295b2e..4cd7f2cb141 100644 --- a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference +++ b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference @@ -5,3 +5,4 @@ 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 From 88b6b2732a3f75444fcc6c578e16e61f7db9ae5b Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 Jun 2024 15:14:05 +0200 Subject: [PATCH 637/856] Sync changes --- src/Common/ProfileEvents.h | 1 + utils/check-style/check-style | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index e670b8907d2..f196ed5a04c 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -40,6 +40,7 @@ namespace ProfileEvents Timer(Counters & counters_, Event timer_event_, Event counter_event, Resolution resolution_); ~Timer() { end(); } void cancel() { watch.reset(); } + void restart() { watch.restart(); } void end(); UInt64 get(); diff --git a/utils/check-style/check-style b/utils/check-style/check-style index db491c67f2c..f5cb65eb879 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -65,6 +65,7 @@ EXTERN_TYPES_EXCLUDES=( ProfileEvents::increment ProfileEvents::incrementForLogMessage ProfileEvents::getName + ProfileEvents::Timer ProfileEvents::Type ProfileEvents::TypeEnum ProfileEvents::dumpToMapColumn @@ -242,7 +243,7 @@ done # All submodules should be from https://github.com/ git config --file "$ROOT_PATH/.gitmodules" --get-regexp 'submodule\..+\.url' | \ -while read -r line; do +while read -r line; do name=${line#submodule.}; name=${name%.url*} url=${line#* } [[ "$url" != 'https://github.com/'* ]] && echo "All submodules should be from https://github.com/, submodule '$name' has '$url'" From 82a5496f28569a46e468587a76e4686d324e5847 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 13 Jun 2024 15:28:04 +0200 Subject: [PATCH 638/856] fix --- src/Core/ServerUUID.cpp | 5 +++++ src/Core/ServerUUID.h | 2 ++ src/Interpreters/tests/gtest_filecache.cpp | 7 +++++++ 3 files changed, 14 insertions(+) diff --git a/src/Core/ServerUUID.cpp b/src/Core/ServerUUID.cpp index 9dfaf4fecf2..251b407e673 100644 --- a/src/Core/ServerUUID.cpp +++ b/src/Core/ServerUUID.cpp @@ -68,4 +68,9 @@ UUID loadServerUUID(const fs::path & server_uuid_file, Poco::Logger * log) } } +void ServerUUID::setRandomForUnitTests() +{ + server_uuid = UUIDHelpers::generateV4(); +} + } diff --git a/src/Core/ServerUUID.h b/src/Core/ServerUUID.h index 9b9963ceeeb..9c7f7d32acc 100644 --- a/src/Core/ServerUUID.h +++ b/src/Core/ServerUUID.h @@ -19,6 +19,8 @@ public: /// Loads server UUID from file or creates new one. Should be called on daemon startup. static void load(const fs::path & server_uuid_file, Poco::Logger * log); + + static void setRandomForUnitTests(); }; UUID loadServerUUID(const fs::path & server_uuid_file, Poco::Logger * log); diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index 41191ba1605..36acc319f4e 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -8,6 +8,7 @@ #include #include +#include #include #include #include @@ -333,6 +334,7 @@ public: TEST_F(FileCacheTest, LRUPolicy) { + ServerUUID::setRandomForUnitTests(); DB::ThreadStatus thread_status; /// To work with cache need query_id and query context. @@ -807,6 +809,7 @@ TEST_F(FileCacheTest, LRUPolicy) TEST_F(FileCacheTest, writeBuffer) { + ServerUUID::setRandomForUnitTests(); FileCacheSettings settings; settings.max_size = 100; settings.max_elements = 5; @@ -938,6 +941,7 @@ static size_t readAllTemporaryData(TemporaryFileStream & stream) TEST_F(FileCacheTest, temporaryData) { + ServerUUID::setRandomForUnitTests(); DB::FileCacheSettings settings; settings.max_size = 10_KiB; settings.max_file_segment_size = 1_KiB; @@ -1044,6 +1048,7 @@ TEST_F(FileCacheTest, temporaryData) TEST_F(FileCacheTest, CachedReadBuffer) { + ServerUUID::setRandomForUnitTests(); DB::ThreadStatus thread_status; /// To work with cache need query_id and query context. @@ -1120,6 +1125,7 @@ TEST_F(FileCacheTest, CachedReadBuffer) TEST_F(FileCacheTest, TemporaryDataReadBufferSize) { + ServerUUID::setRandomForUnitTests(); /// Temporary data stored in cache { DB::FileCacheSettings settings; @@ -1167,6 +1173,7 @@ TEST_F(FileCacheTest, TemporaryDataReadBufferSize) TEST_F(FileCacheTest, SLRUPolicy) { + ServerUUID::setRandomForUnitTests(); DB::ThreadStatus thread_status; std::string query_id = "query_id"; /// To work with cache need query_id and query context. From df607c535c88fa7faf5732f2f7056dc69787a6f6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 13 Jun 2024 13:30:23 +0000 Subject: [PATCH 639/856] Docs: Recommend disabling ASLR for sanitizer builds --- docs/en/development/tests.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index bbc7dac0a2a..f01b4249472 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -229,6 +229,10 @@ For production builds, clang is used, but we also test make gcc builds. For deve ## Sanitizers {#sanitizers} +:::note +Make sure to have address space layout randomization disabled when running sanitizer builds locally: `sudo sysctl kernel.randomize_va_space=0` +::: + ### Address sanitizer We run functional, integration, stress and unit tests under ASan on per-commit basis. From f1707998d214832faa57e3ae35c7343ba6025fa3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 13 Jun 2024 14:25:02 +0200 Subject: [PATCH 640/856] Fix search issues for progress_func, add zstd for GH cache --- docker/test/style/Dockerfile | 7 ++++--- tests/ci/github_helper.py | 6 +++--- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 54fab849301..6ad03852b66 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -10,14 +10,15 @@ RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ aspell \ curl \ - git \ - gh \ file \ + gh \ + git \ libxml2-utils \ + locales \ moreutils \ python3-pip \ yamllint \ - locales \ + zstd \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index b6407c5d531..431e6977091 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -112,12 +112,12 @@ class GitHub(github.Github): # pylint: enable=signature-differs def get_pulls_from_search(self, *args: Any, **kwargs: Any) -> PullRequests: """The search api returns actually issues, so we need to fetch PullRequests""" - issues = self.search_issues(*args, **kwargs) - repos = {} - prs = [] # type: PullRequests progress_func = kwargs.pop( "progress_func", lambda x: x ) # type: Callable[[Issues], Issues] + issues = self.search_issues(*args, **kwargs) + repos = {} + prs = [] # type: PullRequests for issue in progress_func(issues): # See https://github.com/PyGithub/PyGithub/issues/2202, # obj._rawData doesn't spend additional API requests From f48421fd61f426e246274f66f82bb35dfc1ebdb6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 13 Jun 2024 14:30:11 +0200 Subject: [PATCH 641/856] Running a changelog.py with a host network for AMI --- .github/workflows/tags_stable.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/tags_stable.yml b/.github/workflows/tags_stable.yml index e4fc9f0b1d3..2aa7694bc41 100644 --- a/.github/workflows/tags_stable.yml +++ b/.github/workflows/tags_stable.yml @@ -46,9 +46,10 @@ jobs: ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv ./utils/list-versions/update-docker-version.sh GID=$(id -g "${UID}") - docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 \ + # --network=host and CI=1 are required for the S3 access from a container + docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ --volume="${GITHUB_WORKSPACE}:/ClickHouse" clickhouse/style-test \ - /ClickHouse/utils/changelog/changelog.py -v --debug-helpers \ + /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ --gh-user-or-token="$GITHUB_TOKEN" --jobs=5 \ --output="/ClickHouse/docs/changelogs/${GITHUB_TAG}.md" "${GITHUB_TAG}" git add "./docs/changelogs/${GITHUB_TAG}.md" From e8ac8d46ff472cc8bbcb9e5cdbc47102cc533880 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 13 Jun 2024 15:43:30 +0200 Subject: [PATCH 642/856] Update docs/en/development/tests.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Raúl Marín --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index f01b4249472..55330602377 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -230,7 +230,7 @@ For production builds, clang is used, but we also test make gcc builds. For deve ## Sanitizers {#sanitizers} :::note -Make sure to have address space layout randomization disabled when running sanitizer builds locally: `sudo sysctl kernel.randomize_va_space=0` +If the process crashes at the start when running it locally, you might need to disable address space layout randomization: `sudo sysctl kernel.randomize_va_space=0` ::: ### Address sanitizer From 5e1ef8aefaf3477566f91a3e92b6cc844800fc57 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 13 Jun 2024 15:44:36 +0200 Subject: [PATCH 643/856] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 55330602377..8dff6f0ed1d 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -230,7 +230,7 @@ For production builds, clang is used, but we also test make gcc builds. For deve ## Sanitizers {#sanitizers} :::note -If the process crashes at the start when running it locally, you might need to disable address space layout randomization: `sudo sysctl kernel.randomize_va_space=0` +If the process (ClickHouse server or client) crashes at startup when running it locally, you might need to disable address space layout randomization: `sudo sysctl kernel.randomize_va_space=0` ::: ### Address sanitizer From 7c7be37383e015da0302708a9cd43d2ff7361496 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Jun 2024 13:49:03 +0000 Subject: [PATCH 644/856] fix tidy build --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 6afd3b6df3a..83ff025d2a6 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -253,7 +253,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr } } } - catch (...) + catch (...) // NOLINT(bugprone-empty-catch) { // Skip disk that do not have s3 throttlers } From d8b9b00624fd7698de20eb63e5c9ecbb8632a7ae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 Jun 2024 14:03:41 +0000 Subject: [PATCH 645/856] 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 0d17f2cededd98d80d5d8f1486e62899ed953f43 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 13 Jun 2024 16:14:57 +0200 Subject: [PATCH 646/856] CI: FinishCheck to set failure if workflow failed --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 2 +- .github/workflows/merge_queue.yml | 2 +- .github/workflows/pull_request.yml | 2 +- .github/workflows/release_branches.yml | 2 +- tests/ci/finish_check.py | 42 ++++++++++++++++++------- 6 files changed, 36 insertions(+), 16 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index b0380b939bb..c8c6ba30b0b 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -273,5 +273,5 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} python3 merge_pr.py diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 91dcb6a4968..f5c78a6b6a1 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -173,4 +173,4 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml index c8b2452829b..3f45daf0fb4 100644 --- a/.github/workflows/merge_queue.yml +++ b/.github/workflows/merge_queue.yml @@ -112,4 +112,4 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py ${{ (contains(needs.*.result, 'failure') && github.event_name == 'merge_group') && '--pipeline-failure' || '' }} + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index e4deaf9f35e..079208eb65a 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -191,7 +191,7 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} ############################################################################################# ###################################### JEPSEN TESTS ######################################### diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 4d45c8d8d4b..f9b8a4fa764 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -496,4 +496,4 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 12756599865..6a32ba71bed 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -1,4 +1,5 @@ #!/usr/bin/env python3 +import argparse import logging import sys @@ -20,27 +21,38 @@ from report import FAILURE, PENDING, SUCCESS, StatusType from synchronizer_utils import SYNC_BRANCH_PREFIX +def parse_args() -> argparse.Namespace: + parser = argparse.ArgumentParser( + formatter_class=argparse.ArgumentDefaultsHelpFormatter, + description="Script to merge the given PR. Additional checks for approved " + "status and green commit statuses could be done", + ) + parser.add_argument( + "--wf-status", + type=str, + default="", + help="overall workflow status [success|failure]", + ) + return parser.parse_args() + + def main(): logging.basicConfig(level=logging.INFO) + args = parse_args() - has_failure = False - - # FIXME: temporary hack to fail Mergeable Check in MQ if pipeline has any failed jobs - if len(sys.argv) > 1 and sys.argv[1] == "--pipeline-failure": - has_failure = True + has_workflow_failures = args.wf_status == FAILURE pr_info = PRInfo(need_orgs=True) gh = Github(get_best_robot_token(), per_page=100) commit = get_commit(gh, pr_info.sha) - statuses = None if pr_info.is_merge_queue: - # in MQ Mergeable check status must never be green if any failures in workflow - if has_failure: - set_mergeable_check(commit, "workflow failed", "failure") + # in MQ Mergeable check status must never be green if any failures in the workflow + if has_workflow_failures: + set_mergeable_check(commit, "workflow failed", FAILURE) else: # This must be the only place where green MCheck is set in the MQ (in the end of CI) to avoid early merge - set_mergeable_check(commit, "workflow passed", "success") + set_mergeable_check(commit, "workflow passed", SUCCESS) else: statuses = get_commit_filtered_statuses(commit) state = trigger_mergeable_check(commit, statuses, set_if_green=True) @@ -67,6 +79,7 @@ def main(): has_failure = False has_pending = False + error_cnt = 0 for status in statuses: if status.context in (StatusNames.MERGEABLE, StatusNames.CI): # do not account these statuses @@ -80,12 +93,19 @@ def main(): continue else: has_failure = True + error_cnt += 1 ci_state = SUCCESS # type: StatusType + description = "All checks finished" if has_failure: ci_state = FAILURE + description = f"All checks finished. {error_cnt} jobs failed" + elif has_workflow_failures: + ci_state = FAILURE + description = "All checks finished. Workflow has failures." elif has_pending: print("ERROR: CI must not have pending jobs by the time of finish check") + description = "ERROR: workflow has pending jobs" ci_state = FAILURE if ci_status.state == PENDING: @@ -93,7 +113,7 @@ def main(): commit, ci_state, ci_status.target_url, - "All checks finished", + description, StatusNames.CI, pr_info, dump_to_file=True, From c79433e6f4864169762c8548c5bab071fd70bf69 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Thu, 13 Jun 2024 17:18:34 +0300 Subject: [PATCH 647/856] 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 a77eda6388bbb3c887d1b621e6a191269c6ba5c5 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 13 Jun 2024 16:27:41 +0200 Subject: [PATCH 648/856] fix pylint --- tests/ci/finish_check.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 6a32ba71bed..2f624fd91f8 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -1,7 +1,6 @@ #!/usr/bin/env python3 import argparse import logging -import sys from github import Github From fc255456cee541f70f54c4365c410d053d073c22 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 13 Jun 2024 16:36:07 +0200 Subject: [PATCH 649/856] comment --- tests/ci/finish_check.py | 119 ++++++++++++++++++++------------------- 1 file changed, 60 insertions(+), 59 deletions(-) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 2f624fd91f8..0d59c3b43a4 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -52,71 +52,72 @@ def main(): else: # This must be the only place where green MCheck is set in the MQ (in the end of CI) to avoid early merge set_mergeable_check(commit, "workflow passed", SUCCESS) - else: - statuses = get_commit_filtered_statuses(commit) - state = trigger_mergeable_check(commit, statuses, set_if_green=True) + return - # Process upstream StatusNames.SYNC - if ( - pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/") - and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY - ): - upstream_pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) - update_upstream_sync_status( - upstream_pr_number, - pr_info.number, - gh, - state, - can_set_green_mergeable_status=True, - ) + statuses = get_commit_filtered_statuses(commit) + state = trigger_mergeable_check(commit, statuses, set_if_green=True) - ci_running_statuses = [s for s in statuses if s.context == StatusNames.CI] - if not ci_running_statuses: - return - # Take the latest status - ci_status = ci_running_statuses[-1] + # Process upstream StatusNames.SYNC + if ( + pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/") + and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY + ): + upstream_pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) + update_upstream_sync_status( + upstream_pr_number, + pr_info.number, + gh, + state, + can_set_green_mergeable_status=True, + ) - has_failure = False - has_pending = False - error_cnt = 0 - for status in statuses: - if status.context in (StatusNames.MERGEABLE, StatusNames.CI): - # do not account these statuses + ci_running_statuses = [s for s in statuses if s.context == StatusNames.CI] + if not ci_running_statuses: + return + # Take the latest status + ci_status = ci_running_statuses[-1] + + has_failure = False + has_pending = False + error_cnt = 0 + for status in statuses: + if status.context in (StatusNames.MERGEABLE, StatusNames.CI): + # do not account these statuses + continue + if status.state == PENDING: + if status.context == StatusNames.SYNC: + # do not account sync status if pending - it's a different WF continue - if status.state == PENDING: - if status.context == StatusNames.SYNC: - # do not account sync status if pending - it's a different WF - continue - has_pending = True - elif status.state == SUCCESS: - continue - else: - has_failure = True - error_cnt += 1 + has_pending = True + elif status.state == SUCCESS: + continue + else: + has_failure = True + error_cnt += 1 - ci_state = SUCCESS # type: StatusType - description = "All checks finished" - if has_failure: - ci_state = FAILURE - description = f"All checks finished. {error_cnt} jobs failed" - elif has_workflow_failures: - ci_state = FAILURE - description = "All checks finished. Workflow has failures." - elif has_pending: - print("ERROR: CI must not have pending jobs by the time of finish check") - description = "ERROR: workflow has pending jobs" - ci_state = FAILURE + ci_state = SUCCESS # type: StatusType + description = "All checks finished" + if has_failure: + ci_state = FAILURE + description = f"All checks finished. {error_cnt} jobs failed" + elif has_workflow_failures: + ci_state = FAILURE + description = "All checks finished. Workflow has failures." + elif has_pending: + print("ERROR: CI must not have pending jobs by the time of finish check") + description = "ERROR: workflow has pending jobs" + ci_state = FAILURE - if ci_status.state == PENDING: - post_commit_status( - commit, - ci_state, - ci_status.target_url, - description, - StatusNames.CI, - pr_info, - dump_to_file=True, - ) + if ci_status.state == PENDING: + post_commit_status( + commit, + ci_state, + ci_status.target_url, + description, + StatusNames.CI, + pr_info, + dump_to_file=True, + ) if __name__ == "__main__": From 2997509dbff9c49f4e99f05a5d32f9c802aa51a8 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Thu, 13 Jun 2024 17:44:30 +0300 Subject: [PATCH 650/856] 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 651/856] 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 652/856] 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 08c755c7a856329351e0d58535036e540cfa09b8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 13 Jun 2024 18:15:42 +0200 Subject: [PATCH 653/856] Modify test not to use mod, which seems to be failing stateless tests --- ...9_display_column_names_in_footer.reference | 4734 ++++++++--------- .../03169_display_column_names_in_footer.sql | 36 +- 2 files changed, 2375 insertions(+), 2395 deletions(-) diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference index edf0c9e4133..1bccb3ae7f5 100644 --- a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference @@ -1,2382 +1,2362 @@ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ -49. │ 48 │ UInt64 │ 0 │ - └────────┴────────────────────┴────────────────┘ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -10. │ 9 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -100. │ 99 │ UInt64 │ 1 │ - └────────┴────────────────────┴────────────────┘ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -100. │ 99 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -100. │ 99 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -100. │ 99 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -100. │ 99 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━┓ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━┩ - 1. │ 0 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 2. │ 1 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 3. │ 2 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 4. │ 3 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 5. │ 4 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 6. │ 5 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 7. │ 6 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 8. │ 7 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 9. │ 8 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 10. │ 9 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 11. │ 10 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 12. │ 11 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 13. │ 12 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 14. │ 13 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 15. │ 14 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 16. │ 15 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 17. │ 16 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 18. │ 17 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 19. │ 18 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 20. │ 19 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 21. │ 20 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 22. │ 21 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 23. │ 22 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 24. │ 23 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 25. │ 24 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 26. │ 25 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 27. │ 26 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 28. │ 27 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 29. │ 28 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 30. │ 29 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 31. │ 30 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 32. │ 31 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 33. │ 32 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 34. │ 33 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 35. │ 34 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 36. │ 35 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 37. │ 36 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 38. │ 37 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 39. │ 38 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 40. │ 39 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 41. │ 40 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 42. │ 41 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 43. │ 42 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 44. │ 43 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 45. │ 44 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 46. │ 45 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 47. │ 46 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 48. │ 47 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 49. │ 48 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 50. │ 49 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 51. │ 50 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 52. │ 51 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 53. │ 52 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 54. │ 53 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 55. │ 54 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 56. │ 55 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 57. │ 56 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 58. │ 57 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 59. │ 58 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 60. │ 59 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 61. │ 60 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 62. │ 61 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 63. │ 62 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 64. │ 63 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 65. │ 64 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 66. │ 65 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 67. │ 66 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 68. │ 67 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 69. │ 68 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 70. │ 69 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 71. │ 70 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 72. │ 71 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 73. │ 72 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 74. │ 73 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 75. │ 74 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 76. │ 75 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 77. │ 76 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 78. │ 77 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 79. │ 78 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 80. │ 79 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 81. │ 80 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 82. │ 81 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 83. │ 82 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 84. │ 83 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 85. │ 84 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 86. │ 85 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 87. │ 86 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 88. │ 87 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 89. │ 88 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 90. │ 89 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 91. │ 90 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 92. │ 91 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 93. │ 92 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 94. │ 93 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 95. │ 94 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 96. │ 95 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 97. │ 96 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ - 98. │ 97 │ UInt64 │ 1 │ - ├────────┼────────────────────┼────────────────┤ - 99. │ 98 │ UInt64 │ 0 │ - ├────────┼────────────────────┼────────────────┤ -100. │ 99 │ UInt64 │ 1 │ - ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━╋━━━━━━━━━━━━━━━━┫ - ┃ number ┃ toTypeName(number) ┃ mod(number, 2) ┃ - ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━┛ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ -100. │ 99 │ UInt64 │ 1 │ - └────────┴────────────────────┴────────────────┘ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ -100. │ 99 │ UInt64 │ 1 │ - └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ -100. │ 99 │ UInt64 │ 1 │ - └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ -100. │ 99 │ UInt64 │ 1 │ - └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ - ┌─number─┬─toTypeName(number)─┬─mod(number, 2)─┐ - 1. │ 0 │ UInt64 │ 0 │ - 2. │ 1 │ UInt64 │ 1 │ - 3. │ 2 │ UInt64 │ 0 │ - 4. │ 3 │ UInt64 │ 1 │ - 5. │ 4 │ UInt64 │ 0 │ - 6. │ 5 │ UInt64 │ 1 │ - 7. │ 6 │ UInt64 │ 0 │ - 8. │ 7 │ UInt64 │ 1 │ - 9. │ 8 │ UInt64 │ 0 │ - 10. │ 9 │ UInt64 │ 1 │ - 11. │ 10 │ UInt64 │ 0 │ - 12. │ 11 │ UInt64 │ 1 │ - 13. │ 12 │ UInt64 │ 0 │ - 14. │ 13 │ UInt64 │ 1 │ - 15. │ 14 │ UInt64 │ 0 │ - 16. │ 15 │ UInt64 │ 1 │ - 17. │ 16 │ UInt64 │ 0 │ - 18. │ 17 │ UInt64 │ 1 │ - 19. │ 18 │ UInt64 │ 0 │ - 20. │ 19 │ UInt64 │ 1 │ - 21. │ 20 │ UInt64 │ 0 │ - 22. │ 21 │ UInt64 │ 1 │ - 23. │ 22 │ UInt64 │ 0 │ - 24. │ 23 │ UInt64 │ 1 │ - 25. │ 24 │ UInt64 │ 0 │ - 26. │ 25 │ UInt64 │ 1 │ - 27. │ 26 │ UInt64 │ 0 │ - 28. │ 27 │ UInt64 │ 1 │ - 29. │ 28 │ UInt64 │ 0 │ - 30. │ 29 │ UInt64 │ 1 │ - 31. │ 30 │ UInt64 │ 0 │ - 32. │ 31 │ UInt64 │ 1 │ - 33. │ 32 │ UInt64 │ 0 │ - 34. │ 33 │ UInt64 │ 1 │ - 35. │ 34 │ UInt64 │ 0 │ - 36. │ 35 │ UInt64 │ 1 │ - 37. │ 36 │ UInt64 │ 0 │ - 38. │ 37 │ UInt64 │ 1 │ - 39. │ 38 │ UInt64 │ 0 │ - 40. │ 39 │ UInt64 │ 1 │ - 41. │ 40 │ UInt64 │ 0 │ - 42. │ 41 │ UInt64 │ 1 │ - 43. │ 42 │ UInt64 │ 0 │ - 44. │ 43 │ UInt64 │ 1 │ - 45. │ 44 │ UInt64 │ 0 │ - 46. │ 45 │ UInt64 │ 1 │ - 47. │ 46 │ UInt64 │ 0 │ - 48. │ 47 │ UInt64 │ 1 │ - 49. │ 48 │ UInt64 │ 0 │ - 50. │ 49 │ UInt64 │ 1 │ - 51. │ 50 │ UInt64 │ 0 │ - 52. │ 51 │ UInt64 │ 1 │ - 53. │ 52 │ UInt64 │ 0 │ - 54. │ 53 │ UInt64 │ 1 │ - 55. │ 54 │ UInt64 │ 0 │ - 56. │ 55 │ UInt64 │ 1 │ - 57. │ 56 │ UInt64 │ 0 │ - 58. │ 57 │ UInt64 │ 1 │ - 59. │ 58 │ UInt64 │ 0 │ - 60. │ 59 │ UInt64 │ 1 │ - 61. │ 60 │ UInt64 │ 0 │ - 62. │ 61 │ UInt64 │ 1 │ - 63. │ 62 │ UInt64 │ 0 │ - 64. │ 63 │ UInt64 │ 1 │ - 65. │ 64 │ UInt64 │ 0 │ - 66. │ 65 │ UInt64 │ 1 │ - 67. │ 66 │ UInt64 │ 0 │ - 68. │ 67 │ UInt64 │ 1 │ - 69. │ 68 │ UInt64 │ 0 │ - 70. │ 69 │ UInt64 │ 1 │ - 71. │ 70 │ UInt64 │ 0 │ - 72. │ 71 │ UInt64 │ 1 │ - 73. │ 72 │ UInt64 │ 0 │ - 74. │ 73 │ UInt64 │ 1 │ - 75. │ 74 │ UInt64 │ 0 │ - 76. │ 75 │ UInt64 │ 1 │ - 77. │ 76 │ UInt64 │ 0 │ - 78. │ 77 │ UInt64 │ 1 │ - 79. │ 78 │ UInt64 │ 0 │ - 80. │ 79 │ UInt64 │ 1 │ - 81. │ 80 │ UInt64 │ 0 │ - 82. │ 81 │ UInt64 │ 1 │ - 83. │ 82 │ UInt64 │ 0 │ - 84. │ 83 │ UInt64 │ 1 │ - 85. │ 84 │ UInt64 │ 0 │ - 86. │ 85 │ UInt64 │ 1 │ - 87. │ 86 │ UInt64 │ 0 │ - 88. │ 87 │ UInt64 │ 1 │ - 89. │ 88 │ UInt64 │ 0 │ - 90. │ 89 │ UInt64 │ 1 │ - 91. │ 90 │ UInt64 │ 0 │ - 92. │ 91 │ UInt64 │ 1 │ - 93. │ 92 │ UInt64 │ 0 │ - 94. │ 93 │ UInt64 │ 1 │ - 95. │ 94 │ UInt64 │ 0 │ - 96. │ 95 │ UInt64 │ 1 │ - 97. │ 96 │ UInt64 │ 0 │ - 98. │ 97 │ UInt64 │ 1 │ - 99. │ 98 │ UInt64 │ 0 │ -100. │ 99 │ UInt64 │ 1 │ - └─number─┴─toTypeName(number)─┴─mod(number, 2)─┘ - number toTypeName(number) mod(number, 2) + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ +10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ +11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ +12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ +13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ +14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ +15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ +16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ +17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ +18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ +19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ +20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ +21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ +22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ +23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ +24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ +25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ +26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ +27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ +28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ +29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ +30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ +31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ +32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ +33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ +34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ +35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ +36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ +37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ +38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ +39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ +40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ +41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ +42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ +43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ +44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ +45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ +46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ +47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ +48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ +49. │ 48 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ +10. │ 9 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + number toTypeName(number) - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 -100. 99 UInt64 1 - number toTypeName(number) mod(number, 2) + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + number toTypeName(number) - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 -100. 99 UInt64 1 + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + number toTypeName(number) - number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + number toTypeName(number) - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 -100. 99 UInt64 1 + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + number toTypeName(number) - number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) - - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 -100. 99 UInt64 1 - - number toTypeName(number) mod(number, 2) - number toTypeName(number) mod(number, 2) - - 1. 0 UInt64 0 - 2. 1 UInt64 1 - 3. 2 UInt64 0 - 4. 3 UInt64 1 - 5. 4 UInt64 0 - 6. 5 UInt64 1 - 7. 6 UInt64 0 - 8. 7 UInt64 1 - 9. 8 UInt64 0 - 10. 9 UInt64 1 - 11. 10 UInt64 0 - 12. 11 UInt64 1 - 13. 12 UInt64 0 - 14. 13 UInt64 1 - 15. 14 UInt64 0 - 16. 15 UInt64 1 - 17. 16 UInt64 0 - 18. 17 UInt64 1 - 19. 18 UInt64 0 - 20. 19 UInt64 1 - 21. 20 UInt64 0 - 22. 21 UInt64 1 - 23. 22 UInt64 0 - 24. 23 UInt64 1 - 25. 24 UInt64 0 - 26. 25 UInt64 1 - 27. 26 UInt64 0 - 28. 27 UInt64 1 - 29. 28 UInt64 0 - 30. 29 UInt64 1 - 31. 30 UInt64 0 - 32. 31 UInt64 1 - 33. 32 UInt64 0 - 34. 33 UInt64 1 - 35. 34 UInt64 0 - 36. 35 UInt64 1 - 37. 36 UInt64 0 - 38. 37 UInt64 1 - 39. 38 UInt64 0 - 40. 39 UInt64 1 - 41. 40 UInt64 0 - 42. 41 UInt64 1 - 43. 42 UInt64 0 - 44. 43 UInt64 1 - 45. 44 UInt64 0 - 46. 45 UInt64 1 - 47. 46 UInt64 0 - 48. 47 UInt64 1 - 49. 48 UInt64 0 - 50. 49 UInt64 1 - 51. 50 UInt64 0 - 52. 51 UInt64 1 - 53. 52 UInt64 0 - 54. 53 UInt64 1 - 55. 54 UInt64 0 - 56. 55 UInt64 1 - 57. 56 UInt64 0 - 58. 57 UInt64 1 - 59. 58 UInt64 0 - 60. 59 UInt64 1 - 61. 60 UInt64 0 - 62. 61 UInt64 1 - 63. 62 UInt64 0 - 64. 63 UInt64 1 - 65. 64 UInt64 0 - 66. 65 UInt64 1 - 67. 66 UInt64 0 - 68. 67 UInt64 1 - 69. 68 UInt64 0 - 70. 69 UInt64 1 - 71. 70 UInt64 0 - 72. 71 UInt64 1 - 73. 72 UInt64 0 - 74. 73 UInt64 1 - 75. 74 UInt64 0 - 76. 75 UInt64 1 - 77. 76 UInt64 0 - 78. 77 UInt64 1 - 79. 78 UInt64 0 - 80. 79 UInt64 1 - 81. 80 UInt64 0 - 82. 81 UInt64 1 - 83. 82 UInt64 0 - 84. 83 UInt64 1 - 85. 84 UInt64 0 - 86. 85 UInt64 1 - 87. 86 UInt64 0 - 88. 87 UInt64 1 - 89. 88 UInt64 0 - 90. 89 UInt64 1 - 91. 90 UInt64 0 - 92. 91 UInt64 1 - 93. 92 UInt64 0 - 94. 93 UInt64 1 - 95. 94 UInt64 0 - 96. 95 UInt64 1 - 97. 96 UInt64 0 - 98. 97 UInt64 1 - 99. 98 UInt64 0 -100. 99 UInt64 1 - - number toTypeName(number) mod(number, 2) + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.sql b/tests/queries/0_stateless/03169_display_column_names_in_footer.sql index 46779045dca..9e4ec09c21c 100644 --- a/tests/queries/0_stateless/03169_display_column_names_in_footer.sql +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.sql @@ -1,19 +1,19 @@ -- https://github.com/ClickHouse/ClickHouse/issues/65035 -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 49) FORMAT Pretty; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 10) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names_min_rows=9; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names=0; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapes; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact SETTINGS output_format_pretty_display_footer_column_names=0; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapes; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapesMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace SETTINGS output_format_pretty_display_footer_column_names=0; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapes; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceMonoBlock; -SELECT *, toTypeName(*), mod(*,2) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapesMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 49) FORMAT Pretty; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 10) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names_min_rows=9; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapes; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapes; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapesMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapes; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapesMonoBlock; From f82c173c209e2cc7b2e8aa3e9ad134e86c4c02cf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 Jun 2024 16:23:35 +0000 Subject: [PATCH 654/856] 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 833d138383073cdeb431e1c647d0169b28750379 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 13 Jun 2024 16:28:12 +0000 Subject: [PATCH 655/856] Update version_date.tsv and changelogs after v24.3.4.147-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.3.4.147-lts.md | 100 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 + 5 files changed, 105 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.3.4.147-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index b3271d94184..7a57a592724 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.5.1.1763" +ARG VERSION="24.5.2.34" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 3f3b880c8f3..3456a4ee2b9 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.5.1.1763" +ARG VERSION="24.5.2.34" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 5fd22ee9b51..0c17cc10b08 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.5.1.1763" +ARG VERSION="24.5.2.34" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.3.4.147-lts.md b/docs/changelogs/v24.3.4.147-lts.md new file mode 100644 index 00000000000..7d77fb29977 --- /dev/null +++ b/docs/changelogs/v24.3.4.147-lts.md @@ -0,0 +1,100 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.4.147-lts (31a7bdc346d) FIXME as compared to v24.3.3.102-lts (7e7f3bdd9be) + +#### Improvement +* Backported in [#63465](https://github.com/ClickHouse/ClickHouse/issues/63465): Make rabbitmq nack broken messages. Closes [#45350](https://github.com/ClickHouse/ClickHouse/issues/45350). [#60312](https://github.com/ClickHouse/ClickHouse/pull/60312) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#64290](https://github.com/ClickHouse/ClickHouse/issues/64290): Fix logical-error when undoing quorum insert transaction. [#61953](https://github.com/ClickHouse/ClickHouse/pull/61953) ([Han Fei](https://github.com/hanfei1991)). + +#### Build/Testing/Packaging Improvement +* Backported in [#63610](https://github.com/ClickHouse/ClickHouse/issues/63610): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#65128](https://github.com/ClickHouse/ClickHouse/issues/65128): 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 [#64277](https://github.com/ClickHouse/ClickHouse/issues/64277): Fix queries with FINAL give wrong result when table does not use adaptive granularity. [#62432](https://github.com/ClickHouse/ClickHouse/pull/62432) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#63716](https://github.com/ClickHouse/ClickHouse/issues/63716): Fix excessive memory usage for queries with nested lambdas. Fixes [#62036](https://github.com/ClickHouse/ClickHouse/issues/62036). [#62462](https://github.com/ClickHouse/ClickHouse/pull/62462) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63247](https://github.com/ClickHouse/ClickHouse/issues/63247): Fix size checks when updating materialized nested columns ( fixes [#62731](https://github.com/ClickHouse/ClickHouse/issues/62731) ). [#62773](https://github.com/ClickHouse/ClickHouse/pull/62773) ([Eliot Hautefeuille](https://github.com/hileef)). +* Backported in [#62984](https://github.com/ClickHouse/ClickHouse/issues/62984): Fix the `Unexpected return type` error for queries that read from `StorageBuffer` with `PREWHERE` when the source table has different types. Fixes [#62545](https://github.com/ClickHouse/ClickHouse/issues/62545). [#62916](https://github.com/ClickHouse/ClickHouse/pull/62916) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63185](https://github.com/ClickHouse/ClickHouse/issues/63185): Sanity check: Clamp values instead of throwing. [#63119](https://github.com/ClickHouse/ClickHouse/pull/63119) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63293](https://github.com/ClickHouse/ClickHouse/issues/63293): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63411](https://github.com/ClickHouse/ClickHouse/issues/63411): Fix a misbehavior when SQL security defaults don't load for old tables during server startup. [#63209](https://github.com/ClickHouse/ClickHouse/pull/63209) ([pufit](https://github.com/pufit)). +* Backported in [#63616](https://github.com/ClickHouse/ClickHouse/issues/63616): Fix bug which could potentially lead to rare LOGICAL_ERROR during SELECT query with message: `Unexpected return type from materialize. Expected type_XXX. Got type_YYY.` Introduced in [#59379](https://github.com/ClickHouse/ClickHouse/issues/59379). [#63353](https://github.com/ClickHouse/ClickHouse/pull/63353) ([alesapin](https://github.com/alesapin)). +* Backported in [#63455](https://github.com/ClickHouse/ClickHouse/issues/63455): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)). +* Backported in [#63603](https://github.com/ClickHouse/ClickHouse/issues/63603): Fix backup of projection part in case projection was removed from table metadata, but part still has projection. [#63426](https://github.com/ClickHouse/ClickHouse/pull/63426) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#63508](https://github.com/ClickHouse/ClickHouse/issues/63508): Fix 'Every derived table must have its own alias' error for MYSQL dictionary source, close [#63341](https://github.com/ClickHouse/ClickHouse/issues/63341). [#63481](https://github.com/ClickHouse/ClickHouse/pull/63481) ([vdimir](https://github.com/vdimir)). +* Backported in [#63595](https://github.com/ClickHouse/ClickHouse/issues/63595): Avoid segafult in `MergeTreePrefetchedReadPool` while fetching projection parts. [#63513](https://github.com/ClickHouse/ClickHouse/pull/63513) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63748](https://github.com/ClickHouse/ClickHouse/issues/63748): Read only the necessary columns from VIEW (new analyzer). Closes [#62594](https://github.com/ClickHouse/ClickHouse/issues/62594). [#63688](https://github.com/ClickHouse/ClickHouse/pull/63688) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63770](https://github.com/ClickHouse/ClickHouse/issues/63770): Fix [#63539](https://github.com/ClickHouse/ClickHouse/issues/63539). Forbid WINDOW redefinition in new analyzer. [#63694](https://github.com/ClickHouse/ClickHouse/pull/63694) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64189](https://github.com/ClickHouse/ClickHouse/issues/64189): Fix `Not found column` and `CAST AS Map from array requires nested tuple of 2 elements` exceptions for distributed queries which use `Map(Nothing, Nothing)` type. Fixes [#63637](https://github.com/ClickHouse/ClickHouse/issues/63637). [#63753](https://github.com/ClickHouse/ClickHouse/pull/63753) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63845](https://github.com/ClickHouse/ClickHouse/issues/63845): Fix possible `ILLEGAL_COLUMN` error in `partial_merge` join, close [#37928](https://github.com/ClickHouse/ClickHouse/issues/37928). [#63755](https://github.com/ClickHouse/ClickHouse/pull/63755) ([vdimir](https://github.com/vdimir)). +* Backported in [#63906](https://github.com/ClickHouse/ClickHouse/issues/63906): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#63989](https://github.com/ClickHouse/ClickHouse/issues/63989): Fix incorrect select query result when parallel replicas were used to read from a Materialized View. [#63861](https://github.com/ClickHouse/ClickHouse/pull/63861) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#64031](https://github.com/ClickHouse/ClickHouse/issues/64031): Fix a error `Database name is empty` for remote queries with lambdas over the cluster with modified default database. Fixes [#63471](https://github.com/ClickHouse/ClickHouse/issues/63471). [#63864](https://github.com/ClickHouse/ClickHouse/pull/63864) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64559](https://github.com/ClickHouse/ClickHouse/issues/64559): Fix SIGSEGV due to CPU/Real (`query_profiler_real_time_period_ns`/`query_profiler_cpu_time_period_ns`) profiler (has been an issue since 2022, that leads to periodic server crashes, especially if you were using distributed engine). [#63865](https://github.com/ClickHouse/ClickHouse/pull/63865) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#64009](https://github.com/ClickHouse/ClickHouse/issues/64009): Fix analyzer - IN function with arbitrary deep sub-selects in materialized view to use insertion block. [#63930](https://github.com/ClickHouse/ClickHouse/pull/63930) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#64236](https://github.com/ClickHouse/ClickHouse/issues/64236): Fix resolve of unqualified COLUMNS matcher. Preserve the input columns order and forbid usage of unknown identifiers. [#63962](https://github.com/ClickHouse/ClickHouse/pull/63962) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64106](https://github.com/ClickHouse/ClickHouse/issues/64106): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64168](https://github.com/ClickHouse/ClickHouse/issues/64168): Add missing settings to recoverLostReplica. [#64040](https://github.com/ClickHouse/ClickHouse/pull/64040) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64320](https://github.com/ClickHouse/ClickHouse/issues/64320): 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)). +* Backported in [#64380](https://github.com/ClickHouse/ClickHouse/issues/64380): 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)). +* Backported in [#64567](https://github.com/ClickHouse/ClickHouse/issues/64567): 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)). +* Backported in [#64270](https://github.com/ClickHouse/ClickHouse/issues/64270): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64339](https://github.com/ClickHouse/ClickHouse/issues/64339): 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)). +* Backported in [#64259](https://github.com/ClickHouse/ClickHouse/issues/64259): Ignore `text_log` config when using Keeper. [#64218](https://github.com/ClickHouse/ClickHouse/pull/64218) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#64688](https://github.com/ClickHouse/ClickHouse/issues/64688): 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)). +* Backported in [#64725](https://github.com/ClickHouse/ClickHouse/issues/64725): Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#64621](https://github.com/ClickHouse/ClickHouse/issues/64621): 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)). +* Backported in [#64678](https://github.com/ClickHouse/ClickHouse/issues/64678): 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)). +* Backported in [#64831](https://github.com/ClickHouse/ClickHouse/issues/64831): 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)). +* Backported in [#64940](https://github.com/ClickHouse/ClickHouse/issues/64940): Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64869](https://github.com/ClickHouse/ClickHouse/issues/64869): 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)). +* Backported in [#64980](https://github.com/ClickHouse/ClickHouse/issues/64980): 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 [#64972](https://github.com/ClickHouse/ClickHouse/issues/64972): 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 [#65070](https://github.com/ClickHouse/ClickHouse/issues/65070): 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 [#65175](https://github.com/ClickHouse/ClickHouse/issues/65175): 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)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#64587](https://github.com/ClickHouse/ClickHouse/issues/64587): 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)). +* Backported in [#64878](https://github.com/ClickHouse/ClickHouse/issues/64878): 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)). + +#### NO CL CATEGORY + +* Backported in [#63304](https://github.com/ClickHouse/ClickHouse/issues/63304):. [#63297](https://github.com/ClickHouse/ClickHouse/pull/63297) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#63708](https://github.com/ClickHouse/ClickHouse/issues/63708):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#64363](https://github.com/ClickHouse/ClickHouse/issues/64363) to 24.3: Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts"'. [#64907](https://github.com/ClickHouse/ClickHouse/pull/64907) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#63751](https://github.com/ClickHouse/ClickHouse/issues/63751): group_by_use_nulls strikes back. [#62922](https://github.com/ClickHouse/ClickHouse/pull/62922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63558](https://github.com/ClickHouse/ClickHouse/issues/63558): Try fix segfault in `MergeTreeReadPoolBase::createTask`. [#63323](https://github.com/ClickHouse/ClickHouse/pull/63323) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63336](https://github.com/ClickHouse/ClickHouse/issues/63336): The commit url has different pattern. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63374](https://github.com/ClickHouse/ClickHouse/issues/63374): Add tags for the test 03000_traverse_shadow_system_data_paths.sql to make it stable. [#63366](https://github.com/ClickHouse/ClickHouse/pull/63366) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#63625](https://github.com/ClickHouse/ClickHouse/issues/63625): Workaround for `oklch()` inside canvas bug for firefox. [#63404](https://github.com/ClickHouse/ClickHouse/pull/63404) ([Sergei Trifonov](https://github.com/serxa)). +* Backported in [#63569](https://github.com/ClickHouse/ClickHouse/issues/63569): Add `jwcrypto` to integration tests runner. [#63551](https://github.com/ClickHouse/ClickHouse/pull/63551) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Backported in [#63649](https://github.com/ClickHouse/ClickHouse/issues/63649): Fix `02362_part_log_merge_algorithm` flaky test. [#63635](https://github.com/ClickHouse/ClickHouse/pull/63635) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Backported in [#63762](https://github.com/ClickHouse/ClickHouse/issues/63762): Cancel S3 reads properly when parallel reads are used. [#63687](https://github.com/ClickHouse/ClickHouse/pull/63687) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63741](https://github.com/ClickHouse/ClickHouse/issues/63741): Userspace page cache: don't collect stats if cache is unused. [#63730](https://github.com/ClickHouse/ClickHouse/pull/63730) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#63826](https://github.com/ClickHouse/ClickHouse/issues/63826): Fix `test_odbc_interaction` for arm64 on linux. [#63787](https://github.com/ClickHouse/ClickHouse/pull/63787) ([alesapin](https://github.com/alesapin)). +* Backported in [#63895](https://github.com/ClickHouse/ClickHouse/issues/63895): Fix `test_catboost_evaluate` for aarch64. [#63789](https://github.com/ClickHouse/ClickHouse/pull/63789) ([alesapin](https://github.com/alesapin)). +* Backported in [#63887](https://github.com/ClickHouse/ClickHouse/issues/63887): Fix `test_disk_types` for aarch64. [#63832](https://github.com/ClickHouse/ClickHouse/pull/63832) ([alesapin](https://github.com/alesapin)). +* Backported in [#63879](https://github.com/ClickHouse/ClickHouse/issues/63879): Fix `test_short_strings_aggregation` for arm. [#63836](https://github.com/ClickHouse/ClickHouse/pull/63836) ([alesapin](https://github.com/alesapin)). +* Backported in [#63916](https://github.com/ClickHouse/ClickHouse/issues/63916): Disable `test_non_default_compression/test.py::test_preconfigured_deflateqpl_codec` on arm. [#63839](https://github.com/ClickHouse/ClickHouse/pull/63839) ([alesapin](https://github.com/alesapin)). +* Backported in [#63969](https://github.com/ClickHouse/ClickHouse/issues/63969): fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#64047](https://github.com/ClickHouse/ClickHouse/issues/64047): Do not create new release in release branch automatically. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64076](https://github.com/ClickHouse/ClickHouse/issues/64076): Files without shebang have mime 'text/plain' or 'inode/x-empty'. [#64062](https://github.com/ClickHouse/ClickHouse/pull/64062) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64142](https://github.com/ClickHouse/ClickHouse/issues/64142): Fix sanitizers. [#64090](https://github.com/ClickHouse/ClickHouse/pull/64090) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#64159](https://github.com/ClickHouse/ClickHouse/issues/64159): Add retries in `git submodule update`. [#64125](https://github.com/ClickHouse/ClickHouse/pull/64125) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#64473](https://github.com/ClickHouse/ClickHouse/issues/64473): Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts. [#64363](https://github.com/ClickHouse/ClickHouse/pull/64363) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65113](https://github.com/ClickHouse/ClickHouse/issues/65113): 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 [#64999](https://github.com/ClickHouse/ClickHouse/issues/64999): Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 2f96daf4887..fe4adf1b446 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,6 +1,8 @@ +v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.4.147-lts 2024-06-13 v24.3.3.102-lts 2024-05-01 v24.3.2.23-lts 2024-04-03 v24.3.1.2672-lts 2024-03-27 From 11020f71f30389991bde012fc33d8dc0d05fde80 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 13 Jun 2024 13:38:53 +0000 Subject: [PATCH 656/856] Update version_date.tsv and changelogs after v24.5.2.34-stable --- docs/changelogs/v24.5.2.34-stable.md | 38 ++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 docs/changelogs/v24.5.2.34-stable.md diff --git a/docs/changelogs/v24.5.2.34-stable.md b/docs/changelogs/v24.5.2.34-stable.md new file mode 100644 index 00000000000..2db05a5f5dc --- /dev/null +++ b/docs/changelogs/v24.5.2.34-stable.md @@ -0,0 +1,38 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.2.34-stable (45589aeee49) FIXME as compared to v24.5.1.1763-stable (647c154a94d) + +#### Improvement +* Backported in [#65096](https://github.com/ClickHouse/ClickHouse/issues/65096): 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)). + +#### Build/Testing/Packaging Improvement +* Backported in [#65132](https://github.com/ClickHouse/ClickHouse/issues/65132): 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 [#64729](https://github.com/ClickHouse/ClickHouse/issues/64729): Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#65061](https://github.com/ClickHouse/ClickHouse/issues/65061): 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)). +* Backported in [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088): 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)). +* Backported in [#64944](https://github.com/ClickHouse/ClickHouse/issues/64944): Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64873](https://github.com/ClickHouse/ClickHouse/issues/64873): 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)). +* Backported in [#64984](https://github.com/ClickHouse/ClickHouse/issues/64984): 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 [#64976](https://github.com/ClickHouse/ClickHouse/issues/64976): 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 [#65074](https://github.com/ClickHouse/ClickHouse/issues/65074): 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 [#65179](https://github.com/ClickHouse/ClickHouse/issues/65179): 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 [#65163](https://github.com/ClickHouse/ClickHouse/issues/65163): 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)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#64882](https://github.com/ClickHouse/ClickHouse/issues/64882): 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)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65002](https://github.com/ClickHouse/ClickHouse/issues/65002): Be more graceful with existing tables with `inverted` indexes. [#64656](https://github.com/ClickHouse/ClickHouse/pull/64656) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#65115](https://github.com/ClickHouse/ClickHouse/issues/65115): 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 [#64796](https://github.com/ClickHouse/ClickHouse/issues/64796): Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). + From 2077a7ca077989016b986028f11ee26e5ffecfd0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 13 Jun 2024 19:17:29 +0200 Subject: [PATCH 657/856] Update reference file... again --- .../00305_http_and_readonly_2.reference | 26 ------------ ...9_display_column_names_in_footer.reference | 40 ++++++++++++++----- 2 files changed, 30 insertions(+), 36 deletions(-) delete mode 100644 tests/queries/0_stateless/00305_http_and_readonly_2.reference diff --git a/tests/queries/0_stateless/00305_http_and_readonly_2.reference b/tests/queries/0_stateless/00305_http_and_readonly_2.reference deleted file mode 100644 index baddaa07bca..00000000000 --- a/tests/queries/0_stateless/00305_http_and_readonly_2.reference +++ /dev/null @@ -1,26 +0,0 @@ - name value changed - -1. max_rows_to_read 10000 1 -2. readonly 0 0 - name value changed - -1. max_rows_to_read 10000 1 -2. readonly 2 1 - name value changed - -1. max_rows_to_read 10000 1 -2. readonly 1 1 - name value changed - -1. max_rows_to_read 10000 1 -2. readonly 2 1 -Ok -Ok -0 -0 -Ok -Ok -Ok -0 -0 -0 diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference index 1bccb3ae7f5..7a9f413a900 100644 --- a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference @@ -121,7 +121,9 @@ 9. │ 8 │ UInt64 │ ├────────┼────────────────────┤ 10. │ 9 │ UInt64 │ - └────────┴────────────────────┘ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ ┃ number ┃ toTypeName(number) ┃ ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ @@ -527,7 +529,9 @@ 99. │ 98 │ UInt64 │ ├────────┼────────────────────┤ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ ┃ number ┃ toTypeName(number) ┃ ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ @@ -730,7 +734,9 @@ 99. │ 98 │ UInt64 │ ├────────┼────────────────────┤ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ ┃ number ┃ toTypeName(number) ┃ ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ @@ -933,7 +939,9 @@ 99. │ 98 │ UInt64 │ ├────────┼────────────────────┤ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ ┃ number ┃ toTypeName(number) ┃ ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ @@ -1136,7 +1144,9 @@ 99. │ 98 │ UInt64 │ ├────────┼────────────────────┤ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ ┃ number ┃ toTypeName(number) ┃ ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ @@ -1339,7 +1349,9 @@ 99. │ 98 │ UInt64 │ ├────────┼────────────────────┤ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ ┌─number─┬─toTypeName(number)─┐ 1. │ 0 │ UInt64 │ 2. │ 1 │ UInt64 │ @@ -1543,7 +1555,7 @@ 98. │ 97 │ UInt64 │ 99. │ 98 │ UInt64 │ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + └─number─┴─toTypeName(number)─┘ ┌─number─┬─toTypeName(number)─┐ 1. │ 0 │ UInt64 │ 2. │ 1 │ UInt64 │ @@ -1645,7 +1657,7 @@ 98. │ 97 │ UInt64 │ 99. │ 98 │ UInt64 │ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + └─number─┴─toTypeName(number)─┘ ┌─number─┬─toTypeName(number)─┐ 1. │ 0 │ UInt64 │ 2. │ 1 │ UInt64 │ @@ -1747,7 +1759,7 @@ 98. │ 97 │ UInt64 │ 99. │ 98 │ UInt64 │ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + └─number─┴─toTypeName(number)─┘ ┌─number─┬─toTypeName(number)─┐ 1. │ 0 │ UInt64 │ 2. │ 1 │ UInt64 │ @@ -1849,7 +1861,7 @@ 98. │ 97 │ UInt64 │ 99. │ 98 │ UInt64 │ 100. │ 99 │ UInt64 │ - └────────┴────────────────────┘ + └─number─┴─toTypeName(number)─┘ number toTypeName(number) 1. 0 UInt64 @@ -2054,6 +2066,8 @@ 98. 97 UInt64 99. 98 UInt64 100. 99 UInt64 + + number toTypeName(number) number toTypeName(number) 1. 0 UInt64 @@ -2156,6 +2170,8 @@ 98. 97 UInt64 99. 98 UInt64 100. 99 UInt64 + + number toTypeName(number) number toTypeName(number) 1. 0 UInt64 @@ -2258,6 +2274,8 @@ 98. 97 UInt64 99. 98 UInt64 100. 99 UInt64 + + number toTypeName(number) number toTypeName(number) 1. 0 UInt64 @@ -2360,3 +2378,5 @@ 98. 97 UInt64 99. 98 UInt64 100. 99 UInt64 + + number toTypeName(number) From fb32a99578b57cf185f6e868879aaf2ff218419d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 Jun 2024 19:13:13 +0200 Subject: [PATCH 658/856] 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 3bba8e80f1dc55226bf99c7a2e238585e1afed60 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 13 Jun 2024 17:51:46 +0000 Subject: [PATCH 659/856] Update version_date.tsv and changelogs after v24.5.3.5-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.5.3.5-stable.md | 14 ++++++++++++++ utils/list-versions/version_date.tsv | 3 +++ 5 files changed, 20 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.5.3.5-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index b3271d94184..24f38740ff5 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.5.1.1763" +ARG VERSION="24.5.3.5" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 3f3b880c8f3..c71319a2a7e 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.5.1.1763" +ARG VERSION="24.5.3.5" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 5fd22ee9b51..ed8cf3d657d 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.5.1.1763" +ARG VERSION="24.5.3.5" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.5.3.5-stable.md b/docs/changelogs/v24.5.3.5-stable.md new file mode 100644 index 00000000000..4606e58d0a4 --- /dev/null +++ b/docs/changelogs/v24.5.3.5-stable.md @@ -0,0 +1,14 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.3.5-stable (e0eb66f8e17) FIXME as compared to v24.5.2.34-stable (45589aeee49) + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65227](https://github.com/ClickHouse/ClickHouse/issues/65227): Capture weak_ptr of ContextAccess for safety. [#65051](https://github.com/ClickHouse/ClickHouse/pull/65051) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#65219](https://github.com/ClickHouse/ClickHouse/issues/65219): 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 2f96daf4887..719c25bdc95 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,6 +1,9 @@ +v24.5.3.5-stable 2024-06-13 +v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.4.147-lts 2024-06-13 v24.3.3.102-lts 2024-05-01 v24.3.2.23-lts 2024-04-03 v24.3.1.2672-lts 2024-03-27 From 3506ed14f447a3e1ffbbda0b38c097569d2da202 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Jun 2024 19:56:54 +0200 Subject: [PATCH 660/856] Add validation when creating a user with bcrypt_hash --- src/Access/AuthenticationData.cpp | 14 +++++++++++++- .../0_stateless/03172_bcrypt_validation.reference | 0 .../0_stateless/03172_bcrypt_validation.sql | 2 ++ 3 files changed, 15 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03172_bcrypt_validation.reference create mode 100644 tests/queries/0_stateless/03172_bcrypt_validation.sql diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index a32215f3d92..814ee72c74b 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -31,6 +31,7 @@ namespace DB { namespace ErrorCodes { + extern const int AUTHENTICATION_FAILED; extern const int SUPPORT_IS_DISABLED; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; @@ -90,8 +91,10 @@ bool AuthenticationData::Util::checkPasswordBcrypt(std::string_view password [[m { #if USE_BCRYPT int ret = bcrypt_checkpw(password.data(), reinterpret_cast(password_bcrypt.data())); + /// Before 24.6 we didn't validate hashes on creation, so it could be that the stored hash is invalid + /// and it could not be decoded by the library if (ret == -1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "BCrypt library failed: bcrypt_checkpw returned {}", ret); + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Internal failure decoding Bcrypt hash"); return (ret == 0); #else throw Exception( @@ -230,6 +233,15 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password hash for the 'BCRYPT_PASSWORD' authentication type has length {} " "but must be 59 or 60 bytes.", hash.size()); + + auto resized = hash; + resized.resize(64); + + /// Verify that it is a valid hash + int ret = bcrypt_checkpw("", reinterpret_cast(resized.data())); + if (ret == -1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not decode the provided hash with 'bcrypt_hash'"); + password_hash = hash; password_hash.resize(64); return; diff --git a/tests/queries/0_stateless/03172_bcrypt_validation.reference b/tests/queries/0_stateless/03172_bcrypt_validation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03172_bcrypt_validation.sql b/tests/queries/0_stateless/03172_bcrypt_validation.sql new file mode 100644 index 00000000000..2c34a7d60d1 --- /dev/null +++ b/tests/queries/0_stateless/03172_bcrypt_validation.sql @@ -0,0 +1,2 @@ +DROP USER IF EXISTS 03172_user_invalid_bcrypt_hash; +CREATE USER 03172_user_invalid_bcrypt_hash IDENTIFIED WITH bcrypt_hash BY '012345678901234567890123456789012345678901234567890123456789'; -- { serverError BAD_ARGUMENTS } From 3f7286dc51bcffe499f1da45fdfe75ae4504aa3c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 Jun 2024 18:08:00 +0000 Subject: [PATCH 661/856] 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 662/856] 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 5f686fdb0782ba2abdca380e98b49bacb64f297f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 13 Jun 2024 18:34:14 +0000 Subject: [PATCH 663/856] Less flaky 01019_alter_materialized_view_consistent --- .../01019_alter_materialized_view_consistent.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh b/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh index 3a2eac1f38f..26c2e54e637 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh @@ -54,10 +54,10 @@ function alter_thread() { for i in {0..5}; do ALTER[$i]="ALTER TABLE mv MODIFY QUERY SELECT v == 1 as test, v as case FROM src_a;" done - # Insert 3 ALTERs to src_b, one in the first half of the array and two in arbitrary positions. - ALTER[$RANDOM % 3]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" - ALTER[$RANDOM % 6]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" - ALTER[$RANDOM % 6]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" + # Insert 3 ALTERs to src_b randomly in each third of array. + ALTER[$RANDOM % 2]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" + ALTER[$RANDOM % 2 + 2]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" + ALTER[$RANDOM % 2 + 4]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" i=0 while true; do From 5bc879c07c77f4ddaf4498ba6db52ecffc49fb3e Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 13 Jun 2024 16:40:01 +0200 Subject: [PATCH 664/856] comment --- tests/ci/finish_check.py | 28 +++++++++++----------------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 0d59c3b43a4..904b565ad86 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -81,17 +81,12 @@ def main(): has_pending = False error_cnt = 0 for status in statuses: - if status.context in (StatusNames.MERGEABLE, StatusNames.CI): + if status.context in (StatusNames.MERGEABLE, StatusNames.CI, StatusNames.SYNC): # do not account these statuses continue if status.state == PENDING: - if status.context == StatusNames.SYNC: - # do not account sync status if pending - it's a different WF - continue has_pending = True - elif status.state == SUCCESS: - continue - else: + elif status.state != SUCCESS: has_failure = True error_cnt += 1 @@ -108,16 +103,15 @@ def main(): description = "ERROR: workflow has pending jobs" ci_state = FAILURE - if ci_status.state == PENDING: - post_commit_status( - commit, - ci_state, - ci_status.target_url, - description, - StatusNames.CI, - pr_info, - dump_to_file=True, - ) + post_commit_status( + commit, + ci_state, + ci_status.target_url, + description, + StatusNames.CI, + pr_info, + dump_to_file=True, + ) if __name__ == "__main__": From 02677892e33fef568df1b1317d766e81e9c8d9ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 13 Jun 2024 21:03:02 +0200 Subject: [PATCH 665/856] Fix libunwind in CI --- docker/test/fuzzer/run-fuzzer.sh | 1 + docker/test/stateless/attach_gdb.lib | 1 + 2 files changed, 2 insertions(+) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index b8f967ed9c2..6191aeaf304 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -208,6 +208,7 @@ handle SIGPIPE nostop noprint pass handle SIGTERM nostop noprint pass handle SIGUSR1 nostop noprint pass handle SIGUSR2 nostop noprint pass +handle SIGSEGV nostop pass handle SIG$RTMIN nostop noprint pass info signals continue diff --git a/docker/test/stateless/attach_gdb.lib b/docker/test/stateless/attach_gdb.lib index d288288bb17..eb54f920b98 100644 --- a/docker/test/stateless/attach_gdb.lib +++ b/docker/test/stateless/attach_gdb.lib @@ -20,6 +20,7 @@ handle SIGPIPE nostop noprint pass handle SIGTERM nostop noprint pass handle SIGUSR1 nostop noprint pass handle SIGUSR2 nostop noprint pass +handle SIGSEGV nostop pass handle SIG$RTMIN nostop noprint pass info signals continue From dfcb36506273b9c401447a3fdc0eabc2dee90f6a Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 13 Jun 2024 21:53:55 +0200 Subject: [PATCH 666/856] CI: Do not skip FinishCheck in Merge Queue --- .github/workflows/merge_queue.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml index c8b2452829b..5f8ff407e93 100644 --- a/.github/workflows/merge_queue.yml +++ b/.github/workflows/merge_queue.yml @@ -99,7 +99,7 @@ jobs: ################################# Stage Final ################################# # FinishCheck: - if: ${{ !failure() && !cancelled() }} + if: ${{ !cancelled() }} needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Tests_1] runs-on: [self-hosted, style-checker-aarch64] steps: From 99ce17fb2b0a1b5c7787470690f8e14c11005101 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 13 Jun 2024 22:15:39 +0200 Subject: [PATCH 667/856] style fix --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 079208eb65a..66ca3381a40 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -191,7 +191,7 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} ############################################################################################# ###################################### JEPSEN TESTS ######################################### From 87f451d7641e1f2b5392eeddf4c0655bae58236b Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 13 Jun 2024 22:36:57 +0200 Subject: [PATCH 668/856] Revert "Change default s3_throw_on_zero_files_match to true, document that presigned S3 URLs are not supported" --- docs/en/sql-reference/table-functions/s3.md | 11 +--------- src/Core/Settings.h | 6 +++--- src/Core/SettingsChangesHistory.h | 6 ++---- .../ObjectStorage/Azure/Configuration.cpp | 1 - .../ObjectStorage/HDFS/Configuration.cpp | 1 - .../ObjectStorage/S3/Configuration.cpp | 1 - .../ObjectStorage/StorageObjectStorage.h | 1 - .../StorageObjectStorageSource.cpp | 12 +++-------- .../StorageObjectStorageSource.h | 2 -- src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- tests/integration/test_storage_hdfs/test.py | 21 ++++++------------- ...02481_s3_throw_if_mismatch_files.reference | 4 ++-- .../02481_s3_throw_if_mismatch_files.sql | 4 ++-- ...ed_url_and_url_with_special_characters.sql | 4 ++-- .../aspell-ignore/en/aspell-dict.txt | 5 ++--- 15 files changed, 24 insertions(+), 57 deletions(-) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 7538d66996f..1a7e2b8d66a 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -248,6 +248,7 @@ FROM s3( LIMIT 5; ``` + ## Working with archives Suppose that we have several archive files with following URIs on S3: @@ -265,16 +266,6 @@ FROM s3( ); ``` -## Presigned URL - -Presigned URLs are currently not supported. Use `url()` table function instead: -```sql -SELECT * -FROM url( - 'https://example.amazonaws.com/f.csv?X-Amz-Security-Token=[...]' -) -``` - ## Virtual Columns {#virtual-columns} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index efa84f19f78..b3e83092a77 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -115,9 +115,9 @@ class IColumn; M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ - M(Bool, s3_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \ - M(Bool, hdfs_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \ - M(Bool, azure_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, azure_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, s3_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in S3 table engine", 0) \ M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 31da77fddaf..69bc8c5d207 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -88,9 +88,8 @@ static const std::map StorageObjectStorageSourc iterator = std::make_unique( object_storage, configuration, predicate, virtual_columns, local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size, - settings.throw_on_zero_files_match, settings.throw_on_zero_files_match_setting_name, - file_progress_callback); + settings.throw_on_zero_files_match, file_progress_callback); } else { @@ -426,7 +425,6 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( ObjectInfos * read_keys_, size_t list_object_keys_size, bool throw_on_zero_files_match_, - const char * throw_on_zero_files_match_setting_name_, std::function file_progress_callback_) : IIterator("GlobIterator") , WithContext(context_) @@ -434,7 +432,6 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( , configuration(configuration_) , virtual_columns(virtual_columns_) , throw_on_zero_files_match(throw_on_zero_files_match_) - , throw_on_zero_files_match_setting_name(throw_on_zero_files_match_setting_name_) , read_keys(read_keys_) , file_progress_callback(file_progress_callback_) { @@ -487,11 +484,8 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne if (first_iteration && !object_info && throw_on_zero_files_match) { throw Exception(ErrorCodes::FILE_DOESNT_EXIST, - "Can not match any files with path {}{}", - configuration->getPath(), - throw_on_zero_files_match_setting_name - ? fmt::format(" (this error can be suppressed by setting {} = false)", throw_on_zero_files_match_setting_name) - : ""); + "Can not match any files with path {}", + configuration->getPath()); } first_iteration = false; return object_info; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 5e76d8e979f..fd7c7aa7102 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -168,7 +168,6 @@ public: ObjectInfos * read_keys_, size_t list_object_keys_size, bool throw_on_zero_files_match_, - const char * throw_on_zero_files_match_setting_name_, std::function file_progress_callback_ = {}); ~GlobIterator() override = default; @@ -185,7 +184,6 @@ private: const ConfigurationPtr configuration; const NamesAndTypesList virtual_columns; const bool throw_on_zero_files_match; - const char * throw_on_zero_files_match_setting_name; size_t index = 0; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 7e26335c691..afb75a21b21 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -486,7 +486,7 @@ std::shared_ptr StorageS3Queue::createFileIterator { auto settings = configuration->getQuerySettings(local_context); auto glob_iterator = std::make_unique( - object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match, settings.throw_on_zero_files_match_setting_name); + object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match); return std::make_shared(files_metadata, std::move(glob_iterator), shutdown_called, log); } diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 818a1e7447d..47d8f44c0b7 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -111,7 +111,7 @@ def test_storage_with_multidirectory_glob(started_cluster): try: node1.query( - "SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p4/path1,p2/path3}/postfix/data{1,2}.nonexist', TSV) SETTINGS hdfs_throw_on_zero_files_match=0" + "SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p4/path1,p2/path3}/postfix/data{1,2}.nonexist', TSV)" ) assert False, "Exception have to be thrown" except Exception as ex: @@ -220,22 +220,14 @@ def test_globs_in_read_table(started_cluster): ) print("inside_table_func ", inside_table_func) assert ( - node1.query( - "select * from hdfs(" - + inside_table_func - + ") settings hdfs_throw_on_zero_files_match=0" - ) + node1.query("select * from hdfs(" + inside_table_func + ")") == paths_amount * some_data ) assert node1.query( - "select count(distinct _path) from hdfs(" - + inside_table_func - + ") settings hdfs_throw_on_zero_files_match=0" + "select count(distinct _path) from hdfs(" + inside_table_func + ")" ).rstrip() == str(paths_amount) assert node1.query( - "select count(distinct _file) from hdfs(" - + inside_table_func - + ") settings hdfs_throw_on_zero_files_match=0" + "select count(distinct _file) from hdfs(" + inside_table_func + ")" ).rstrip() == str(files_amount) @@ -643,7 +635,6 @@ def test_cluster_join(started_cluster): SELECT l.id,r.id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as l JOIN hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as r ON l.id = r.id - SETTINGS hdfs_throw_on_zero_files_match=0 """ ) assert "AMBIGUOUS_COLUMN_NAME" not in result @@ -652,13 +643,13 @@ def test_cluster_join(started_cluster): def test_cluster_macro(started_cluster): with_macro = node1.query( """ - SELECT id FROM hdfsCluster('{default_cluster_macro}', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') SETTINGS hdfs_throw_on_zero_files_match=0 + SELECT id FROM hdfsCluster('{default_cluster_macro}', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') """ ) no_macro = node1.query( """ - SELECT id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') SETTINGS hdfs_throw_on_zero_files_match=0 + SELECT id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') """ ) diff --git a/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.reference b/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.reference index 752b12ff3bd..a7096a686f5 100644 --- a/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.reference +++ b/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.reference @@ -3,5 +3,5 @@ drop table if exists test_02481_mismatch_files; create table test_02481_mismatch_files (a UInt64, b String) engine = S3(s3_conn, filename='test_02481_mismatch_files_{_partition_id}', format=Parquet) partition by a; set s3_truncate_on_insert=1; insert into test_02481_mismatch_files values (1, 'a'), (22, 'b'), (333, 'c'); -select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } -select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError FILE_DOESNT_EXIST } +select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } +select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=1; -- { serverError FILE_DOESNT_EXIST } diff --git a/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.sql b/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.sql index cd500b58946..7ec1d3ebd5f 100644 --- a/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.sql +++ b/tests/queries/0_stateless/02481_s3_throw_if_mismatch_files.sql @@ -7,6 +7,6 @@ create table test_02481_mismatch_files (a UInt64, b String) engine = S3(s3_conn, set s3_truncate_on_insert=1; insert into test_02481_mismatch_files values (1, 'a'), (22, 'b'), (333, 'c'); -select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } +select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } -select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError FILE_DOESNT_EXIST } +select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=1; -- { serverError FILE_DOESNT_EXIST } diff --git a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql index 078a5701aca..1e99eb8b83d 100644 --- a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql +++ b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql @@ -1,5 +1,5 @@ -- Tags: no-fasttest -select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet', NOSIGN) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } +select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } -select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD', NOSIGN) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_DETECT_FORMAT } +select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_DETECT_FORMAT } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index c4b70de1f65..84682689934 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2912 +personal_ws-1.1 en 2758 AArch ACLs ALTERs @@ -722,7 +722,6 @@ Postgres PostgresSQL Precompiled Preprocess -Presigned PrettyCompact PrettyCompactMonoBlock PrettyCompactNoEscapes @@ -1937,9 +1936,9 @@ loghouse london lookups loongarch +lowcardinality lowCardinalityIndices lowCardinalityKeys -lowcardinality lowerUTF lowercased lttb From cd2d825dd10d3b9ed9a13cb6d7df803a1ee84d5f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 13 Jun 2024 18:52:40 +0100 Subject: [PATCH 669/856] impl --- src/Interpreters/HashJoin.cpp | 12 +++++++++--- src/Interpreters/HashJoin.h | 3 +++ 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 75da8bbc3e7..1c2a054b2a5 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -869,6 +869,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) || (min_rows_to_compress && getTotalRowCount() >= min_rows_to_compress))) { block_to_save = block_to_save.compress(); + have_compressed = true; } data->blocks_allocated_size += block_to_save.allocatedBytes(); @@ -2317,14 +2318,19 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) } }; - for (const Block & compressed_block_right : data->blocks) + for (const Block & block_right : data->blocks) { ++block_number; if (block_number < start_right_block) continue; - auto block_right = compressed_block_right.decompress(); - process_right_block(block_right); + /// The following statement cannot be substituted with `process_right_block(!have_compressed ? block_right : block_right.decompress())` + /// because it will lead to copying of `block_right` even if its branch is taken (because common type of `block_right` and `block_right.decompress()` is `Block`). + if (!have_compressed) + process_right_block(block_right); + else + process_right_block(block_right.decompress()); + if (rows_added > max_joined_block_rows) { break; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index a0996556f9a..56a1768a7ff 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -434,7 +434,10 @@ private: /// Changes in hash table broke correspondence, /// so we must guarantee constantness of hash table during HashJoin lifetime (using method setLock) mutable JoinStuff::JoinUsedFlags used_flags; + RightTableDataPtr data; + bool have_compressed = false; + std::vector key_sizes; /// Needed to do external cross join From 39261d09a27e9ecaea1c6c3892f04a187dc730a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Jun 2024 23:32:25 +0200 Subject: [PATCH 670/856] Fix compilation without libraries --- src/Access/AuthenticationData.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index 814ee72c74b..70355fadfbd 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -237,10 +237,12 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash) auto resized = hash; resized.resize(64); +#if USE_BCRYPT /// Verify that it is a valid hash int ret = bcrypt_checkpw("", reinterpret_cast(resized.data())); if (ret == -1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not decode the provided hash with 'bcrypt_hash'"); +#endif password_hash = hash; password_hash.resize(64); From fcc16b9f5229dde2207cf3789fe800a03548b682 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Fri, 14 Jun 2024 02:08:17 +0300 Subject: [PATCH 671/856] 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 04897eb0faf2c489197a2727f41dd9d9b2033fa8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Jun 2024 13:42:28 +0000 Subject: [PATCH 672/856] Done --- src/Core/Settings.h | 3 + src/Core/SettingsChangesHistory.h | 3 + .../Serializations/SerializationTuple.cpp | 96 +++++++- src/Formats/EscapingRuleUtils.cpp | 2 +- src/Formats/FormatFactory.cpp | 3 + src/Formats/FormatSettings.h | 2 + tests/queries/0_stateless/00300_csv.reference | 6 +- .../01016_input_null_as_default.sh | 4 +- ...csv_best_effort_schema_inference.reference | 80 +++---- .../02969_auto_format_detection.reference | 209 +++++++++--------- .../02977_csv_format_support_tuple.sql | 4 + 11 files changed, 250 insertions(+), 162 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b3e83092a77..bb5e8411a23 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1009,6 +1009,8 @@ class IColumn; M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ M(Bool, format_csv_allow_single_quotes, false, "If it is set to true, allow strings in single quotes.", 0) \ M(Bool, format_csv_allow_double_quotes, true, "If it is set to true, allow strings in double quotes.", 0) \ + M(Bool, format_csv_serialize_tuple_into_separate_columns, true, "If it set to true, then Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost)", 0) \ + M(Bool, format_csv_deserialize_separate_columns_into_tuple, true, "if it set to true, then separate columns written in CSV format can be deserialized to Tuple column.", 0) \ M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ M(Bool, input_format_csv_allow_cr_end_of_line, false, "If it is set true, \\r will be allowed at end of line not followed by \\n", 0) \ M(Bool, input_format_csv_enum_as_number, false, "Treat inserted enum values in CSV formats as enum indices", 0) \ @@ -1047,6 +1049,7 @@ class IColumn; M(UInt64, input_format_max_bytes_to_read_for_schema_inference, 32 * 1024 * 1024, "The maximum bytes of data to read for automatic schema inference", 0) \ M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_csv_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference in CSV format", 0) \ + M(Bool, input_format_csv_try_infer_strings_from_quoted_tuples, true, "Interpret quoted tuples in the input data as a value of type String.", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 69bc8c5d207..c47893d79f8 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -172,6 +172,9 @@ static const std::mapserializeTextCSV(extractElementColumn(column, i), row_num, ostr, settings); + } + } + else + { + WriteBufferFromOwnString wb; + serializeText(column, row_num, wb, settings); + writeCSV(wb.str(), ostr); + } } void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - String s; - readCSV(s, istr, settings.csv); - ReadBufferFromString rb(s); - deserializeText(column, rb, settings, true); + if (settings.csv.deserialize_separate_columns_into_tuple) + { + addElementSafe(elems.size(), column, [&] + { + const size_t size = elems.size(); + for (size_t i = 0; i < size; ++i) + { + if (i != 0) + { + skipWhitespaceIfAny(istr); + assertChar(settings.csv.tuple_delimiter, istr); + skipWhitespaceIfAny(istr); + } + + auto & element_column = extractElementColumn(column, i); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextCSV(element_column, istr, settings, elems[i]); + else + elems[i]->deserializeTextCSV(element_column, istr, settings); + } + return true; + }); + } + else + { + String s; + readCSV(s, istr, settings.csv); + ReadBufferFromString rb(s); + deserializeText(column, rb, settings, true); + } } bool SerializationTuple::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - String s; - if (!tryReadCSV(s, istr, settings.csv)) - return false; - ReadBufferFromString rb(s); - return tryDeserializeText(column, rb, settings, true); + if (settings.csv.deserialize_separate_columns_into_tuple) + { + return addElementSafe(elems.size(), column, [&] + { + const size_t size = elems.size(); + for (size_t i = 0; i < size; ++i) + { + if (i != 0) + { + skipWhitespaceIfAny(istr); + if (!checkChar(settings.csv.tuple_delimiter, istr)) + return false; + skipWhitespaceIfAny(istr); + } + + auto & element_column = extractElementColumn(column, i); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) + { + if (!SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextCSV(element_column, istr, settings, elems[i])) + return false; + } + else + { + if (!elems[i]->tryDeserializeTextCSV(element_column, istr, settings)) + return false; + } + } + + return true; + }); + } + else + { + String s; + if (!tryReadCSV(s, istr, settings.csv)) + return false; + ReadBufferFromString rb(s); + return tryDeserializeText(column, rb, settings, true); + } } struct SerializeBinaryBulkStateTuple : public ISerialization::SerializeBinaryBulkState diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 9577ca2a8df..10cf279bf7e 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -303,7 +303,7 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet auto type = tryInferDataTypeForSingleField(data, format_settings); /// If we couldn't infer any type or it's a number and csv.try_infer_numbers_from_strings = 0, we determine it as a string. - if (!type || (isNumber(type) && !format_settings.csv.try_infer_numbers_from_strings)) + if (!type || (isTuple(type) && format_settings.csv.try_infer_strings_from_quoted_tuples) || (isNumber(type) && !format_settings.csv.try_infer_numbers_from_strings)) return std::make_shared(); return type; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index e90986f2236..d2a9c723d4a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -77,6 +77,8 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.avro.output_rows_in_file = settings.output_format_avro_rows_in_file; format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; + format_settings.csv.serialize_tuple_into_separate_columns = settings.format_csv_serialize_tuple_into_separate_columns; + format_settings.csv.deserialize_separate_columns_into_tuple = settings.format_csv_deserialize_separate_columns_into_tuple; format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line; format_settings.csv.allow_cr_end_of_line = settings.input_format_csv_allow_cr_end_of_line; format_settings.csv.delimiter = settings.format_csv_delimiter; @@ -94,6 +96,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.csv.try_infer_numbers_from_strings = settings.input_format_csv_try_infer_numbers_from_strings; + format_settings.csv.try_infer_strings_from_quoted_tuples = settings.input_format_csv_try_infer_strings_from_quoted_tuples; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 337aafbbe9c..8c571987e74 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -153,6 +153,8 @@ struct FormatSettings char delimiter = ','; bool allow_single_quotes = true; bool allow_double_quotes = true; + bool serialize_tuple_into_separate_columns = true; + bool deserialize_separate_columns_into_tuple = true; bool empty_as_default = false; bool crlf_end_of_line = false; bool allow_cr_end_of_line = false; diff --git a/tests/queries/0_stateless/00300_csv.reference b/tests/queries/0_stateless/00300_csv.reference index e7966a9e8d9..42cd22078c4 100644 --- a/tests/queries/0_stateless/00300_csv.reference +++ b/tests/queries/0_stateless/00300_csv.reference @@ -1,11 +1,11 @@ -"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline +"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline here" "x","y","z","a","b" -"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline +"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline here" "x","y","z","a","b" "String","UInt8","Array(UInt8)","Tuple(UInt16, Array(String))","String" -"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline +"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline here" 0,"0","[]","2000-01-01","2000-01-01 00:00:00" 1,"1","[0]","2000-01-02","2000-01-01 00:00:01" diff --git a/tests/queries/0_stateless/01016_input_null_as_default.sh b/tests/queries/0_stateless/01016_input_null_as_default.sh index 8d6a9a07435..24d93b2703c 100755 --- a/tests/queries/0_stateless/01016_input_null_as_default.sh +++ b/tests/queries/0_stateless/01016_input_null_as_default.sh @@ -11,8 +11,8 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE default_by_other_column (a Float32 DEFA echo 'CSV' echo '\N, 1, \N, "2019-07-22", "[10, 20, 30]", \N -1, world, 3, "2019-07-23", \N, "('\''tuple'\'', 3.14)" -2, \N, 123, \N, "[]", "('\''test'\'', 2.71828)" +1, world, 3, "2019-07-23", \N, tuple, 3.14 +2, \N, 123, \N, "[]", test, 2.71828 3, \N, \N, \N, \N, \N' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO null_as_default FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; $CLICKHOUSE_CLIENT --query="TRUNCATE TABLE null_as_default"; diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference index 8ad0a566c62..777f1155f0c 100644 --- a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference @@ -1,107 +1,107 @@ TSV -c1 Nullable(Int64) -c2 Nullable(String) -c3 Array(Nullable(Int64)) -c4 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) +c1 Nullable(Int64) +c2 Nullable(String) +c3 Array(Nullable(Int64)) +c4 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) 42 Some string [1,2,3,4] (1,2,3) 42 abcd [] (4,5,6) -c1 Nullable(String) +c1 Nullable(String) [({\'key\' : 42.42}, [\'String\', \'String2\'], 42.42), ({}, [], -42), ({\'key2\' : NULL}, [NULL], NULL)] [] [({}, [], 0)] [({}, [NULL], NULL)] [({}, [\'String3\'], NULL)] [({\'key3\': NULL}, []), NULL] -c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) +c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) [({'key':42.42},['String','String2'],42.42),({},[],-42),({'key2':NULL},[NULL],NULL)] [] [({},[],0)] [({},[NULL],NULL)] [({},['String3'],NULL)] [({'key3':NULL},[],NULL)] -c1 Nullable(Bool) +c1 Nullable(Bool) true false \N -c1 Array(Nullable(Bool)) +c1 Array(Nullable(Bool)) [true,NULL] [] [NULL] [false] -c1 Nullable(String) +c1 Nullable(String) [] -c1 Nullable(String) +c1 Nullable(String) {} -c1 Nullable(String) +c1 Nullable(String) () -c1 Nullable(String) +c1 Nullable(String) [1, 2, 3 -c1 Nullable(String) +c1 Nullable(String) [(1, 2, 3 4)] -c1 Nullable(String) +c1 Nullable(String) [1, 2, 3 + 4] -c1 Nullable(String) +c1 Nullable(String) (1, 2, -c1 Nullable(String) +c1 Nullable(String) [1, Some trash, 42.2] -c1 Nullable(String) +c1 Nullable(String) [1, \'String\', {\'key\' : 2}] -c1 Nullable(String) +c1 Nullable(String) {\'key\' : 1, [1] : 10} -c1 Nullable(String) +c1 Nullable(String) {}{} -c1 Nullable(String) +c1 Nullable(String) [1, 2, 3 -c1 Nullable(String) +c1 Nullable(String) [abc, def] -c1 Array(Nullable(String)) +c1 Array(Nullable(String)) ['abc','def'] -c1 Nullable(String) +c1 Nullable(String) [\'string] -c1 Nullable(String) +c1 Nullable(String) \'string -c1 Nullable(Float64) +c1 Nullable(Float64) 42.42 -c1 Nullable(String) +c1 Nullable(String) 42.42sometrash -c1 Nullable(String) +c1 Nullable(String) [42.42sometrash, 42.42] CSV -c1 Nullable(String) -c2 Nullable(String) -c3 Array(Nullable(Int64)) -c4 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) +c1 Nullable(String) +c2 Nullable(String) +c3 Array(Nullable(Int64)) +c4 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) 42 Some string [1,2,3,4] [(1,2,3)] 42\\ abcd [] [(4,5,6)] -c1 Nullable(String) +c1 Nullable(String) [({\'key\' : 42.42}, [\'String\', \'String2\'], 42.42), ({}, [], -42), ({\'key2\' : NULL}, [NULL], NULL)] [] [({}, [], 0)] [({}, [NULL], NULL)] [({}, [\'String3\'], NULL)] [({\'key3\': NULL}, []), NULL] -c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) +c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) [({'key':42.42},['String','String2'],42.42),({},[],-42),({'key2':NULL},[NULL],NULL)] [] [({},[],0)] [({},[NULL],NULL)] [({},['String3'],NULL)] [({'key3':NULL},[],NULL)] -c1 Nullable(Bool) +c1 Nullable(Bool) true false \N -c1 Array(Nullable(Bool)) +c1 Array(Nullable(Bool)) [true,NULL] [] [NULL] [false] -c1 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) -(1,2,3) -c1 Nullable(String) +c1 Nullable(String) +(1, 2, 3) +c1 Nullable(String) 123.123 -c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) +c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) [(1,2,3)] -c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) +c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) [(1,2,3)] diff --git a/tests/queries/0_stateless/02969_auto_format_detection.reference b/tests/queries/0_stateless/02969_auto_format_detection.reference index 865db11defc..9fcaef54db1 100644 --- a/tests/queries/0_stateless/02969_auto_format_detection.reference +++ b/tests/queries/0_stateless/02969_auto_format_detection.reference @@ -1,122 +1,123 @@ Parquet -a Nullable(UInt64) -b Nullable(String) -c Array(Nullable(UInt64)) -d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +a Nullable(UInt64) +b Nullable(String) +c Array(Nullable(UInt64)) +d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) ORC -a Nullable(Int64) -b Nullable(String) -c Array(Nullable(Int64)) -d Tuple(\n a Nullable(Int64),\n b Nullable(String)) +a Nullable(Int64) +b Nullable(String) +c Array(Nullable(Int64)) +d Tuple(\n a Nullable(Int64),\n b Nullable(String)) Arrow -a Nullable(UInt64) -b Nullable(String) -c Array(Nullable(UInt64)) -d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +a Nullable(UInt64) +b Nullable(String) +c Array(Nullable(UInt64)) +d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) ArrowStream -a Nullable(UInt64) -b Nullable(String) -c Array(Nullable(UInt64)) -d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +a Nullable(UInt64) +b Nullable(String) +c Array(Nullable(UInt64)) +d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) Avro -a Int64 -b String -c Array(Int64) -d Tuple(\n a Int64,\n b String) +a Int64 +b String +c Array(Int64) +d Tuple(\n a Int64,\n b String) Native -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) BSONEachRow -a Nullable(Int64) -b Nullable(String) -c Array(Nullable(Int64)) -d Tuple(\n a Nullable(Int64),\n b Nullable(String)) +a Nullable(Int64) +b Nullable(String) +c Array(Nullable(Int64)) +d Tuple(\n a Nullable(Int64),\n b Nullable(String)) JSONCompact -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) Values -c1 Nullable(UInt64) -c2 Nullable(String) -c3 Array(Nullable(UInt64)) -c4 Tuple(Nullable(UInt64), Nullable(String)) +c1 Nullable(UInt64) +c2 Nullable(String) +c3 Array(Nullable(UInt64)) +c4 Tuple(Nullable(UInt64), Nullable(String)) TSKV -a Nullable(String) -b Nullable(String) -c Array(Nullable(UInt64)) -d Nullable(String) +a Nullable(String) +b Nullable(String) +c Array(Nullable(UInt64)) +d Nullable(String) JSONObjectEachRow -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) JSONColumns -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) JSONCompactColumns -c1 Nullable(String) -c2 Nullable(String) -c3 Array(Nullable(String)) -c4 Tuple(\n a Nullable(String),\n b Nullable(String)) +c1 Nullable(String) +c2 Nullable(String) +c3 Array(Nullable(String)) +c4 Tuple(\n a Nullable(String),\n b Nullable(String)) JSONCompact -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) JSON -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) TSV -c1 Nullable(UInt64) -c2 Nullable(String) -c3 Array(Nullable(UInt64)) -c4 Tuple(Nullable(UInt64), Nullable(String)) +c1 Nullable(UInt64) +c2 Nullable(String) +c3 Array(Nullable(UInt64)) +c4 Tuple(Nullable(UInt64), Nullable(String)) CSV -c1 Nullable(UInt64) -c2 Nullable(String) -c3 Array(Nullable(UInt64)) -c4 Tuple(Nullable(UInt64), Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) +c1 Nullable(UInt64) +c2 Nullable(String) +c3 Array(Nullable(UInt64)) +c4 Nullable(UInt64) +c5 Nullable(String) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) 1 -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql index d00cc00e097..3e9a51049c7 100644 --- a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql @@ -1,5 +1,9 @@ -- Tags: no-parallel +SET format_csv_serialize_tuple_into_separate_columns = false; +SET format_csv_deserialize_separate_columns_into_tuple = false; +SET input_format_csv_try_infer_strings_from_quoted_tuples = false; + insert into function file('02977_1.csv') select '20240305', 1, ['s', 'd'], map('a', 2), tuple('222', 33, map('abc', 5)) SETTINGS engine_file_truncate_on_insert=1; desc file('02977_1.csv'); select * from file('02977_1.csv') settings max_threads=1; From a98ec2e0e9531446fd231ceb8a4fbc0c0ef0f62b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Jun 2024 15:58:29 +0200 Subject: [PATCH 673/856] Update src/Formats/EscapingRuleUtils.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/Formats/EscapingRuleUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 10cf279bf7e..36d16d8d154 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -303,7 +303,7 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet auto type = tryInferDataTypeForSingleField(data, format_settings); /// If we couldn't infer any type or it's a number and csv.try_infer_numbers_from_strings = 0, we determine it as a string. - if (!type || (isTuple(type) && format_settings.csv.try_infer_strings_from_quoted_tuples) || (isNumber(type) && !format_settings.csv.try_infer_numbers_from_strings)) + if (!type || (format_settings.csv.try_infer_strings_from_quoted_tuples && isTuple(type)) || (!format_settings.csv.try_infer_numbers_from_strings && isNumber(type))) return std::make_shared(); return type; From e48ebf55293f8dceaf42b30dc4bbb4615efad92b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Jun 2024 14:34:12 +0000 Subject: [PATCH 674/856] Fixed build and tests --- src/Core/Settings.h | 4 ++-- src/Core/SettingsChangesHistory.h | 4 ++-- src/Formats/FormatFactory.cpp | 4 ++-- src/Formats/FormatSettings.h | 1 + .../0_stateless/00309_formats.reference | Bin 18545 -> 18537 bytes .../02977_csv_format_support_tuple.sql | 4 ++-- 6 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bb5e8411a23..70cd5bf2a62 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1009,8 +1009,8 @@ class IColumn; M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ M(Bool, format_csv_allow_single_quotes, false, "If it is set to true, allow strings in single quotes.", 0) \ M(Bool, format_csv_allow_double_quotes, true, "If it is set to true, allow strings in double quotes.", 0) \ - M(Bool, format_csv_serialize_tuple_into_separate_columns, true, "If it set to true, then Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost)", 0) \ - M(Bool, format_csv_deserialize_separate_columns_into_tuple, true, "if it set to true, then separate columns written in CSV format can be deserialized to Tuple column.", 0) \ + M(Bool, output_format_csv_serialize_tuple_into_separate_columns, true, "If it set to true, then Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost)", 0) \ + M(Bool, input_format_csv_deserialize_separate_columns_into_tuple, true, "if it set to true, then separate columns written in CSV format can be deserialized to Tuple column.", 0) \ M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ M(Bool, input_format_csv_allow_cr_end_of_line, false, "If it is set true, \\r will be allowed at end of line not followed by \\n", 0) \ M(Bool, input_format_csv_enum_as_number, false, "Treat inserted enum values in CSV formats as enum indices", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index c47893d79f8..e73527c063d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -172,8 +172,8 @@ static const std::map Date: Wed, 12 Jun 2024 15:37:16 +0000 Subject: [PATCH 675/856] Fix test --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 70cd5bf2a62..db560570f67 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1010,7 +1010,7 @@ class IColumn; M(Bool, format_csv_allow_single_quotes, false, "If it is set to true, allow strings in single quotes.", 0) \ M(Bool, format_csv_allow_double_quotes, true, "If it is set to true, allow strings in double quotes.", 0) \ M(Bool, output_format_csv_serialize_tuple_into_separate_columns, true, "If it set to true, then Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost)", 0) \ - M(Bool, input_format_csv_deserialize_separate_columns_into_tuple, true, "if it set to true, then separate columns written in CSV format can be deserialized to Tuple column.", 0) \ + M(Bool, input_format_csv_deserialize_separate_columns_into_tuple, true, "If it set to true, then separate columns written in CSV format can be deserialized to Tuple column.", 0) \ M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ M(Bool, input_format_csv_allow_cr_end_of_line, false, "If it is set true, \\r will be allowed at end of line not followed by \\n", 0) \ M(Bool, input_format_csv_enum_as_number, false, "Treat inserted enum values in CSV formats as enum indices", 0) \ From 74dc6cdb5adc3913180df3aa8e9ae8a0dc101ea8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Jun 2024 19:45:58 +0000 Subject: [PATCH 676/856] Fix tests --- ...csv_best_effort_schema_inference.reference | 78 +++---- .../02969_auto_format_detection.reference | 210 +++++++++--------- 2 files changed, 144 insertions(+), 144 deletions(-) diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference index 777f1155f0c..1c60e40942c 100644 --- a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference @@ -1,107 +1,107 @@ TSV -c1 Nullable(Int64) -c2 Nullable(String) -c3 Array(Nullable(Int64)) -c4 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) +c1 Nullable(Int64) +c2 Nullable(String) +c3 Array(Nullable(Int64)) +c4 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) 42 Some string [1,2,3,4] (1,2,3) 42 abcd [] (4,5,6) -c1 Nullable(String) +c1 Nullable(String) [({\'key\' : 42.42}, [\'String\', \'String2\'], 42.42), ({}, [], -42), ({\'key2\' : NULL}, [NULL], NULL)] [] [({}, [], 0)] [({}, [NULL], NULL)] [({}, [\'String3\'], NULL)] [({\'key3\': NULL}, []), NULL] -c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) +c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) [({'key':42.42},['String','String2'],42.42),({},[],-42),({'key2':NULL},[NULL],NULL)] [] [({},[],0)] [({},[NULL],NULL)] [({},['String3'],NULL)] [({'key3':NULL},[],NULL)] -c1 Nullable(Bool) +c1 Nullable(Bool) true false \N -c1 Array(Nullable(Bool)) +c1 Array(Nullable(Bool)) [true,NULL] [] [NULL] [false] -c1 Nullable(String) +c1 Nullable(String) [] -c1 Nullable(String) +c1 Nullable(String) {} -c1 Nullable(String) +c1 Nullable(String) () -c1 Nullable(String) +c1 Nullable(String) [1, 2, 3 -c1 Nullable(String) +c1 Nullable(String) [(1, 2, 3 4)] -c1 Nullable(String) +c1 Nullable(String) [1, 2, 3 + 4] -c1 Nullable(String) +c1 Nullable(String) (1, 2, -c1 Nullable(String) +c1 Nullable(String) [1, Some trash, 42.2] -c1 Nullable(String) +c1 Nullable(String) [1, \'String\', {\'key\' : 2}] -c1 Nullable(String) +c1 Nullable(String) {\'key\' : 1, [1] : 10} -c1 Nullable(String) +c1 Nullable(String) {}{} -c1 Nullable(String) +c1 Nullable(String) [1, 2, 3 -c1 Nullable(String) +c1 Nullable(String) [abc, def] -c1 Array(Nullable(String)) +c1 Array(Nullable(String)) ['abc','def'] -c1 Nullable(String) +c1 Nullable(String) [\'string] -c1 Nullable(String) +c1 Nullable(String) \'string -c1 Nullable(Float64) +c1 Nullable(Float64) 42.42 -c1 Nullable(String) +c1 Nullable(String) 42.42sometrash -c1 Nullable(String) +c1 Nullable(String) [42.42sometrash, 42.42] CSV -c1 Nullable(String) -c2 Nullable(String) -c3 Array(Nullable(Int64)) -c4 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) +c1 Nullable(String) +c2 Nullable(String) +c3 Array(Nullable(Int64)) +c4 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) 42 Some string [1,2,3,4] [(1,2,3)] 42\\ abcd [] [(4,5,6)] -c1 Nullable(String) +c1 Nullable(String) [({\'key\' : 42.42}, [\'String\', \'String2\'], 42.42), ({}, [], -42), ({\'key2\' : NULL}, [NULL], NULL)] [] [({}, [], 0)] [({}, [NULL], NULL)] [({}, [\'String3\'], NULL)] [({\'key3\': NULL}, []), NULL] -c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) +c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) [({'key':42.42},['String','String2'],42.42),({},[],-42),({'key2':NULL},[NULL],NULL)] [] [({},[],0)] [({},[NULL],NULL)] [({},['String3'],NULL)] [({'key3':NULL},[],NULL)] -c1 Nullable(Bool) +c1 Nullable(Bool) true false \N -c1 Array(Nullable(Bool)) +c1 Array(Nullable(Bool)) [true,NULL] [] [NULL] [false] -c1 Nullable(String) +c1 Nullable(String) (1, 2, 3) -c1 Nullable(String) +c1 Nullable(String) 123.123 -c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) +c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) [(1,2,3)] -c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) +c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) [(1,2,3)] diff --git a/tests/queries/0_stateless/02969_auto_format_detection.reference b/tests/queries/0_stateless/02969_auto_format_detection.reference index 9fcaef54db1..4b86be04996 100644 --- a/tests/queries/0_stateless/02969_auto_format_detection.reference +++ b/tests/queries/0_stateless/02969_auto_format_detection.reference @@ -1,123 +1,123 @@ Parquet -a Nullable(UInt64) -b Nullable(String) -c Array(Nullable(UInt64)) -d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +a Nullable(UInt64) +b Nullable(String) +c Array(Nullable(UInt64)) +d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) ORC -a Nullable(Int64) -b Nullable(String) -c Array(Nullable(Int64)) -d Tuple(\n a Nullable(Int64),\n b Nullable(String)) +a Nullable(Int64) +b Nullable(String) +c Array(Nullable(Int64)) +d Tuple(\n a Nullable(Int64),\n b Nullable(String)) Arrow -a Nullable(UInt64) -b Nullable(String) -c Array(Nullable(UInt64)) -d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +a Nullable(UInt64) +b Nullable(String) +c Array(Nullable(UInt64)) +d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) ArrowStream -a Nullable(UInt64) -b Nullable(String) -c Array(Nullable(UInt64)) -d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +a Nullable(UInt64) +b Nullable(String) +c Array(Nullable(UInt64)) +d Tuple(\n a Nullable(UInt64),\n b Nullable(String)) Avro -a Int64 -b String -c Array(Int64) -d Tuple(\n a Int64,\n b String) +a Int64 +b String +c Array(Int64) +d Tuple(\n a Int64,\n b String) Native -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) BSONEachRow -a Nullable(Int64) -b Nullable(String) -c Array(Nullable(Int64)) -d Tuple(\n a Nullable(Int64),\n b Nullable(String)) +a Nullable(Int64) +b Nullable(String) +c Array(Nullable(Int64)) +d Tuple(\n a Nullable(Int64),\n b Nullable(String)) JSONCompact -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) Values -c1 Nullable(UInt64) -c2 Nullable(String) -c3 Array(Nullable(UInt64)) -c4 Tuple(Nullable(UInt64), Nullable(String)) +c1 Nullable(UInt64) +c2 Nullable(String) +c3 Array(Nullable(UInt64)) +c4 Tuple(Nullable(UInt64), Nullable(String)) TSKV -a Nullable(String) -b Nullable(String) -c Array(Nullable(UInt64)) -d Nullable(String) +a Nullable(String) +b Nullable(String) +c Array(Nullable(UInt64)) +d Nullable(String) JSONObjectEachRow -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) JSONColumns -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) JSONCompactColumns -c1 Nullable(String) -c2 Nullable(String) -c3 Array(Nullable(String)) -c4 Tuple(\n a Nullable(String),\n b Nullable(String)) +c1 Nullable(String) +c2 Nullable(String) +c3 Array(Nullable(String)) +c4 Tuple(\n a Nullable(String),\n b Nullable(String)) JSONCompact -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) JSON -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) TSV -c1 Nullable(UInt64) -c2 Nullable(String) -c3 Array(Nullable(UInt64)) -c4 Tuple(Nullable(UInt64), Nullable(String)) +c1 Nullable(UInt64) +c2 Nullable(String) +c3 Array(Nullable(UInt64)) +c4 Tuple(Nullable(UInt64), Nullable(String)) CSV -c1 Nullable(UInt64) -c2 Nullable(String) -c3 Array(Nullable(UInt64)) -c4 Nullable(UInt64) -c5 Nullable(String) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a UInt64 -b String -c Array(UInt64) -d Tuple(\n a UInt64,\n b String) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) +c1 Nullable(UInt64) +c2 Nullable(String) +c3 Array(Nullable(UInt64)) +c4 Nullable(UInt64) +c5 Nullable(String) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a UInt64 +b String +c Array(UInt64) +d Tuple(\n a UInt64,\n b String) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) 1 -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) -a Nullable(String) -b Nullable(String) -c Array(Nullable(String)) -d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) +a Nullable(String) +b Nullable(String) +c Array(Nullable(String)) +d Tuple(\n a Nullable(String),\n b Nullable(String)) From ad0b396d9d04ddd97d651f272c1b6d2bc0744d9a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 13 Jun 2024 15:14:15 +0000 Subject: [PATCH 677/856] Fix Upgrade Check --- src/Core/SettingsChangesHistory.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index e73527c063d..aa871184b8d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -105,6 +105,9 @@ static const std::map Date: Thu, 30 May 2024 16:51:36 +0800 Subject: [PATCH 678/856] 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 679/856] 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 680/856] 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 681/856] 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 682/856] 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 683/856] 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 684/856] 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 685/856] 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 686/856] 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 687/856] 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 688/856] trigger CI From 7a253aaf0df8d8d5d1d837866bd11607297f0820 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 14 Jun 2024 01:37:39 +0000 Subject: [PATCH 689/856] fix the bug --- ...dictionary_short_circuit_bug_fix.reference | 6 ++++ ...ashed_dictionary_short_circuit_bug_fix.sql | 30 +++++++++++++++++++ 2 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.reference create mode 100644 tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql diff --git a/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.reference b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.reference new file mode 100644 index 00000000000..a2ac115060f --- /dev/null +++ b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.reference @@ -0,0 +1,6 @@ +100 1 1 +300 3 0 +200 2 2 +100 1 1 +300 3 0 +200 2 2 diff --git a/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql new file mode 100644 index 00000000000..6fa5c694cda --- /dev/null +++ b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql @@ -0,0 +1,30 @@ +-- Tags: no-parallel + +CREATE TABLE x ( hash_id UInt64, user_result Decimal(3, 2) ) ENGINE = Memory(); + +CREATE TABLE y ( hash_id UInt64, user_result DECIMAL(18, 6) ) ENGINE = Memory(); + +INSERT INTO x values (100, 1), (200, 2); +INSERT INTO y values (100, 1), (300, 3), (200, 2); + +CREATE DICTIONARY d1 (hash_id UInt64, user_result Decimal(3, 2) ) +PRIMARY KEY hash_id +SOURCE(CLICKHOUSE(QUERY `select * from x`)) +LIFETIME(0) +LAYOUT(HASHED()); + +SELECT hash_id, + dictGetOrDefault(d1, 'user_result', toUInt64(hash_id), toFloat64(user_result)), + dictGet(d1, 'user_result', toUInt64(hash_id)) +FROM y; + +CREATE DICTIONARY d2 (hash_id UInt64, user_result Decimal(3, 2) ) +PRIMARY KEY hash_id +SOURCE(CLICKHOUSE(QUERY `select * from x`)) +LIFETIME(0) +LAYOUT(HASHED_ARRAY()); + +SELECT hash_id, + dictGetOrDefault(d2, 'user_result', toUInt64(hash_id), toFloat64(user_result)), + dictGet(d2, 'user_result', toUInt64(hash_id)) +FROM y; From 08b481af3d8d59088216df5bc182871ef73284ff Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Fri, 14 Jun 2024 09:44:08 +0800 Subject: [PATCH 690/856] trigger CI From d32647a6cd1590fc6ae605fe616b64c238084a84 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 14 Jun 2024 02:15:21 +0000 Subject: [PATCH 691/856] fix --- .../03171_hashed_dictionary_short_circuit_bug_fix.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql index 6fa5c694cda..e1b5531a442 100644 --- a/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql +++ b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql @@ -9,7 +9,7 @@ INSERT INTO y values (100, 1), (300, 3), (200, 2); CREATE DICTIONARY d1 (hash_id UInt64, user_result Decimal(3, 2) ) PRIMARY KEY hash_id -SOURCE(CLICKHOUSE(QUERY `select * from x`)) +SOURCE(CLICKHOUSE(TABLE 'x')) LIFETIME(0) LAYOUT(HASHED()); @@ -20,7 +20,7 @@ FROM y; CREATE DICTIONARY d2 (hash_id UInt64, user_result Decimal(3, 2) ) PRIMARY KEY hash_id -SOURCE(CLICKHOUSE(QUERY `select * from x`)) +SOURCE(CLICKHOUSE(TABLE 'x')) LIFETIME(0) LAYOUT(HASHED_ARRAY()); From b10daabcea66ffb0ac8272bd5d96d6f22c506b05 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 14 Jun 2024 10:14:44 +0300 Subject: [PATCH 692/856] Ping CI From 1944e0ae5f4346dba2ed2aebdad2951855985708 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Fri, 14 Jun 2024 10:44:14 +0300 Subject: [PATCH 693/856] 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 694/856] Restart CI From a6f05df749e8c6720172db2b4e09899a9b5a0498 Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 5 Jun 2024 19:46:23 +0800 Subject: [PATCH 695/856] [feature] Add an asynchronous_metric jemalloc.profile.active to show whether sampling is currently active for the calling thread. This is an activation mechanism in addition to prof.active; both must be active for the calling thread to sample. --- .../system-tables/asynchronous_metrics.md | 4 ++ src/Common/AsynchronousMetrics.cpp | 10 ++++ .../__init__.py | 0 .../asynchronous_metrics_update_period_s.xml | 3 ++ .../test.py | 49 +++++++++++++++++++ 5 files changed, 66 insertions(+) create mode 100644 tests/integration/test_asynchronous_metric_jemalloc_profile_active/__init__.py create mode 100644 tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml create mode 100644 tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index 81725b97e41..762d187917c 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -639,6 +639,10 @@ An internal metric of the low-level memory allocator (jemalloc). See https://jem An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +### jemalloc.prof.active + +An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html + **See Also** - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 4c71b9846c7..d6c8b36f171 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -415,6 +415,15 @@ Value saveAllArenasMetric(AsynchronousMetricValues & values, fmt::format("jemalloc.arenas.all.{}", metric_name)); } +template +static Value saveJemallocProf(AsynchronousMetricValues & values, + const std::string & metric_name) +{ + return saveJemallocMetricImpl(values, + fmt::format("prof.{}", metric_name), + fmt::format("jemalloc.prof.{}", metric_name)); +} + } #endif @@ -607,6 +616,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) saveJemallocMetric(new_values, "background_thread.num_threads"); saveJemallocMetric(new_values, "background_thread.num_runs"); saveJemallocMetric(new_values, "background_thread.run_intervals"); + saveJemallocProf(new_values, "active"); saveAllArenasMetric(new_values, "pactive"); [[maybe_unused]] size_t je_malloc_pdirty = saveAllArenasMetric(new_values, "pdirty"); [[maybe_unused]] size_t je_malloc_pmuzzy = saveAllArenasMetric(new_values, "pmuzzy"); diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/__init__.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml new file mode 100644 index 00000000000..47e88730482 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py new file mode 100644 index 00000000000..245b1fd3bb9 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -0,0 +1,49 @@ +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=["configs/asynchronous_metrics_update_period_s.xml"], + env_variables={"MALLOC_CONF": "background_thread:true,prof:true"}, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +# Tests that the system.asynchronous_metric_log table gets populated. +# asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting +# asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and +# are available for querying during the test. +def test_event_time_microseconds_field(started_cluster): + res_t = node1.query("SYSTEM JEMALLOC ENABLE PROFILE") + res_o = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") + assert ( + res_o== """Row 1: +────── +metric: jemalloc.prof.active +value: 1 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) + node1.query("SYSTEM JEMALLOC DISABLE PROFILE") + time.sleep(5) + res_t = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") + assert ( + res_t== """Row 1: +────── +metric: jemalloc.prof.active +value: 0 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) From 83359af8b75c4d23ff89e71b1f9130027588b735 Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 5 Jun 2024 20:38:18 +0800 Subject: [PATCH 696/856] [update] rm useless res --- .../test_asynchronous_metric_jemalloc_profile_active/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 245b1fd3bb9..fe0ff46cedb 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -26,7 +26,7 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): - res_t = node1.query("SYSTEM JEMALLOC ENABLE PROFILE") + node1.query("SYSTEM JEMALLOC ENABLE PROFILE") res_o = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") assert ( res_o== """Row 1: From 761d8e327c3c27b246aefce15afb7061aeb986e7 Mon Sep 17 00:00:00 2001 From: unashi Date: Thu, 6 Jun 2024 10:20:09 +0800 Subject: [PATCH 697/856] [update] 1. black test.py 2. adjust test.py to cover all situation --- .../test.py | 32 ++++++++++++++++--- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index fe0ff46cedb..80165a28c76 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -26,24 +26,46 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): - node1.query("SYSTEM JEMALLOC ENABLE PROFILE") - res_o = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") + # prof:true -> default open + res_o = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) assert ( - res_o== """Row 1: + res_o + == """Row 1: ────── metric: jemalloc.prof.active value: 1 description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html """ ) + # disable node1.query("SYSTEM JEMALLOC DISABLE PROFILE") time.sleep(5) - res_t = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") + res_t = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) assert ( - res_t== """Row 1: + res_t + == """Row 1: ────── metric: jemalloc.prof.active value: 0 description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) + # enable + node1.query("SYSTEM JEMALLOC ENABLE PROFILE") + time.sleep(5) + res_f = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) + assert ( + res_f + == """Row 1: +────── +metric: jemalloc.prof.active +value: 1 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html """ ) From 13bdcc335f03c5e6a40c285029acc646432227e2 Mon Sep 17 00:00:00 2001 From: unashi Date: Thu, 6 Jun 2024 17:44:20 +0800 Subject: [PATCH 698/856] [fix] remove static from function --- src/Common/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index d6c8b36f171..6309f6079f6 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -416,7 +416,7 @@ Value saveAllArenasMetric(AsynchronousMetricValues & values, } template -static Value saveJemallocProf(AsynchronousMetricValues & values, +Value saveJemallocProf(AsynchronousMetricValues & values, const std::string & metric_name) { return saveJemallocMetricImpl(values, From 17b03c7df9d23e0a6656a9652e73e6ea5fecf251 Mon Sep 17 00:00:00 2001 From: unashi Date: Sat, 8 Jun 2024 00:41:47 +0800 Subject: [PATCH 699/856] [fix] skip sanitizers --- .../test_asynchronous_metric_jemalloc_profile_active/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 80165a28c76..1283becca6e 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -27,6 +27,9 @@ def started_cluster(): # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): # prof:true -> default open + if node1.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + res_o = node1.query( "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" ) From 3c8f3c1930d126295544d46dd88ad912dad74174 Mon Sep 17 00:00:00 2001 From: unashi Date: Sat, 8 Jun 2024 00:53:44 +0800 Subject: [PATCH 700/856] [fix] fix the name of function in test.py --- .../test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 1283becca6e..218b3e2ec6a 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -21,11 +21,11 @@ def started_cluster(): cluster.shutdown() -# Tests that the system.asynchronous_metric_log table gets populated. + # asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting -# asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and +# asynchronous_metric_update_period_s is being set to 1s so that the metrics are populated faster and # are available for querying during the test. -def test_event_time_microseconds_field(started_cluster): +def test_asynchronous_metric_jemalloc_profile_active(started_cluster): # prof:true -> default open if node1.is_built_with_sanitizer(): pytest.skip("Disabled for sanitizers") From f707c0d1ebfc1cd67ba5c5b76f2c055f4ab23a07 Mon Sep 17 00:00:00 2001 From: unashi Date: Sat, 8 Jun 2024 13:52:13 +0800 Subject: [PATCH 701/856] [black] --- .../test_asynchronous_metric_jemalloc_profile_active/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 218b3e2ec6a..627285a2038 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -21,7 +21,6 @@ def started_cluster(): cluster.shutdown() - # asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting # asynchronous_metric_update_period_s is being set to 1s so that the metrics are populated faster and # are available for querying during the test. From 06e11752fde54b4d14ac0634b32646496e5039db Mon Sep 17 00:00:00 2001 From: unashi Date: Sat, 8 Jun 2024 18:19:15 +0800 Subject: [PATCH 702/856] [retry test] --- .../test_asynchronous_metric_jemalloc_profile_active/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 627285a2038..a8f4ab05888 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -25,7 +25,7 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 1s so that the metrics are populated faster and # are available for querying during the test. def test_asynchronous_metric_jemalloc_profile_active(started_cluster): - # prof:true -> default open + # default open if node1.is_built_with_sanitizer(): pytest.skip("Disabled for sanitizers") From 57e84c0344119d1628eed1a5e6f048cae0505b5c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 09:40:11 +0000 Subject: [PATCH 703/856] 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 67c41d0856d7ff653820d376359eb455376d0a5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jun 2024 11:50:52 +0200 Subject: [PATCH 704/856] Add a test just in case --- .../queries/0_stateless/03171_condition_pushdown.reference | 2 ++ tests/queries/0_stateless/03171_condition_pushdown.sql | 6 ++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03171_condition_pushdown.reference create mode 100644 tests/queries/0_stateless/03171_condition_pushdown.sql diff --git a/tests/queries/0_stateless/03171_condition_pushdown.reference b/tests/queries/0_stateless/03171_condition_pushdown.reference new file mode 100644 index 00000000000..c7e6479af3b --- /dev/null +++ b/tests/queries/0_stateless/03171_condition_pushdown.reference @@ -0,0 +1,2 @@ +123 123 +456 456 diff --git a/tests/queries/0_stateless/03171_condition_pushdown.sql b/tests/queries/0_stateless/03171_condition_pushdown.sql new file mode 100644 index 00000000000..9cfe41ce921 --- /dev/null +++ b/tests/queries/0_stateless/03171_condition_pushdown.sql @@ -0,0 +1,6 @@ +-- This query succeeds only if it is correctly optimized. +SET allow_experimental_analyzer = 1; +SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (123, 456) AND t2.number = t1.number ORDER BY ALL; + +-- Still TODO: +-- SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (SELECT 123 UNION ALL SELECT 456) AND t2.number = t1.number ORDER BY ALL; From 6e3199ebea73a8b2768f487171fbe0263d4ce0a0 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 10 Jun 2024 11:18:03 +0200 Subject: [PATCH 705/856] CI: ci_config refactoring --- .github/PULL_REQUEST_TEMPLATE.md | 3 +- .github/workflows/master.yml | 15 +- .github/workflows/pull_request.yml | 19 +- tests/ci/artifacts_helper.py | 4 +- tests/ci/ast_fuzzer_check.py | 5 +- tests/ci/bugfix_validate_check.py | 15 +- tests/ci/build_check.py | 9 +- tests/ci/build_download_helper.py | 10 +- tests/ci/build_report_check.py | 39 +- tests/ci/ci.py | 72 +- tests/ci/ci_cache.py | 71 +- tests/ci/ci_config.py | 2103 +++++------------ tests/ci/ci_definitions.py | 754 ++++++ tests/ci/ci_settings.py | 30 +- tests/ci/commit_status_helper.py | 34 +- tests/ci/digest_helper.py | 11 +- tests/ci/download_binary.py | 5 +- tests/ci/env_helper.py | 3 +- tests/ci/finish_check.py | 12 +- tests/ci/integration_tests_runner.py | 6 +- tests/ci/jepsen_check.py | 4 +- tests/ci/performance_comparison_check.py | 4 +- tests/ci/pr_info.py | 12 +- tests/ci/report.py | 9 +- tests/ci/run_check.py | 4 +- tests/ci/s3_helper.py | 14 +- tests/ci/sqlancer_check.py | 5 +- tests/ci/sqltest.py | 5 +- tests/ci/ssh.py | 6 +- tests/ci/style_check.py | 4 +- tests/ci/sync_pr.py | 4 +- tests/ci/test_ci_cache.py | 40 +- tests/ci/test_ci_config.py | 314 ++- tests/ci/test_ci_options.py | 36 +- utils/check-style/check_py.sh | 10 +- .../check-style/process_style_check_result.py | 2 +- 36 files changed, 1889 insertions(+), 1804 deletions(-) create mode 100644 tests/ci/ci_definitions.py diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 73b2155482e..74a6f95dbb3 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -48,8 +48,7 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Allow: Stateful tests - [ ] Allow: Integration Tests - [ ] Allow: Performance tests -- [ ] Allow: Normal Builds -- [ ] Allow: Special Builds +- [ ] Allow: All Builds - [ ] Allow: All NOT Required Checks - [ ] Allow: batch 1, 2 for multi-batch jobs - [ ] Allow: batch 3, 4, 5, 6 for multi-batch jobs diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index f5c78a6b6a1..837dbba6174 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -115,8 +115,8 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# - # Reports should be run even if Builds_1/2 failed - put them separately in wf (not in Tests_1/2) - Builds_1_Report: + # Reports should run even if Builds_1/2 fail - run them separately, not in Tests_1/2/3 + Builds_Report: # run report check for failed builds to indicate the CI error if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} needs: [RunConfig, Builds_1] @@ -125,15 +125,6 @@ jobs: test_name: ClickHouse build check runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} - Builds_2_Report: - # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse special build check') }} - needs: [RunConfig, Builds_2] - uses: ./.github/workflows/reusable_test.yml - with: - test_name: ClickHouse special build check - runner_type: style-checker-aarch64 - data: ${{ needs.RunConfig.outputs.data }} MarkReleaseReady: if: ${{ !failure() && !cancelled() }} @@ -165,7 +156,7 @@ jobs: FinishCheck: if: ${{ !cancelled() }} - needs: [RunConfig, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] + needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 66ca3381a40..422bcf551ae 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -143,29 +143,20 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# - # Reports should by run even if Builds_1/2 fail, so put them separately in wf (not in Tests_1/2) - Builds_1_Report: + # Reports should run even if Builds_1/2 fail - run them separately (not in Tests_1/2/3) + Builds_Report: # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.StyleCheck.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} + if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} needs: [RunConfig, StyleCheck, Builds_1] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} - Builds_2_Report: - # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.StyleCheck.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse special build check') }} - needs: [RunConfig, StyleCheck, Builds_2] - uses: ./.github/workflows/reusable_test.yml - with: - test_name: ClickHouse special build check - runner_type: style-checker-aarch64 - data: ${{ needs.RunConfig.outputs.data }} CheckReadyForMerge: if: ${{ !cancelled() && needs.StyleCheck.result == 'success' }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code @@ -181,7 +172,7 @@ jobs: # FinishCheck: if: ${{ !cancelled() }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code diff --git a/tests/ci/artifacts_helper.py b/tests/ci/artifacts_helper.py index 37abf0bdefb..503ba2e1ec4 100644 --- a/tests/ci/artifacts_helper.py +++ b/tests/ci/artifacts_helper.py @@ -15,7 +15,7 @@ from github.Commit import Commit from build_download_helper import download_build_with_progress from commit_status_helper import post_commit_status from compress_files import SUFFIX, compress_fast, decompress_fast -from env_helper import CI, RUNNER_TEMP, S3_BUILDS_BUCKET +from env_helper import IS_CI, RUNNER_TEMP, S3_BUILDS_BUCKET from git_helper import SHA_REGEXP from report import FOOTER_HTML_TEMPLATE, HEAD_HTML_TEMPLATE, SUCCESS from s3_helper import S3Helper @@ -131,7 +131,7 @@ class ArtifactsHelper: post_commit_status(commit, SUCCESS, url, "Artifacts for workflow", "Artifacts") def _regenerate_index(self) -> None: - if CI: + if IS_CI: files = self._get_s3_objects() else: files = self._get_local_s3_objects() diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index b88a9476a6d..8bc0f51dfc7 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -6,7 +6,7 @@ import subprocess import sys from pathlib import Path -from build_download_helper import get_build_name_for_check, read_build_urls +from build_download_helper import read_build_urls from clickhouse_helper import CiLogsCredentials from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPORT_PATH, TEMP_PATH @@ -14,6 +14,7 @@ from pr_info import PRInfo from report import FAIL, FAILURE, OK, SUCCESS, JobReport, TestResult from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI IMAGE_NAME = "clickhouse/fuzzer" @@ -64,7 +65,7 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) if not urls: raise ValueError("No build URLs found") diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index d41fdaf05ff..71b18572938 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -7,7 +7,7 @@ import sys from pathlib import Path from typing import List, Sequence, Tuple -from ci_config import JobNames +from ci_config import CI from ci_utils import normalize_string from env_helper import TEMP_PATH from functional_test_check import NO_CHANGES_MSG @@ -92,16 +92,19 @@ def main(): logging.basicConfig(level=logging.INFO) # args = parse_args() stopwatch = Stopwatch() - jobs_to_validate = [JobNames.STATELESS_TEST_RELEASE, JobNames.INTEGRATION_TEST] + jobs_to_validate = [ + CI.JobNames.STATELESS_TEST_RELEASE, + CI.JobNames.INTEGRATION_TEST, + ] functional_job_report_file = Path(TEMP_PATH) / "functional_test_job_report.json" integration_job_report_file = Path(TEMP_PATH) / "integration_test_job_report.json" jobs_report_files = { - JobNames.STATELESS_TEST_RELEASE: functional_job_report_file, - JobNames.INTEGRATION_TEST: integration_job_report_file, + CI.JobNames.STATELESS_TEST_RELEASE: functional_job_report_file, + CI.JobNames.INTEGRATION_TEST: integration_job_report_file, } jobs_scripts = { - JobNames.STATELESS_TEST_RELEASE: "functional_test_check.py", - JobNames.INTEGRATION_TEST: "integration_test_check.py", + CI.JobNames.STATELESS_TEST_RELEASE: "functional_test_check.py", + CI.JobNames.INTEGRATION_TEST: "integration_test_check.py", } for test_job in jobs_to_validate: diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 260b77b0ee5..39f34ed9ccf 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -9,7 +9,7 @@ from pathlib import Path from typing import Tuple import docker_images_helper -from ci_config import CI_CONFIG, BuildConfig +from ci_config import CI from env_helper import REPO_COPY, S3_BUILDS_BUCKET, TEMP_PATH from git_helper import Git from lambda_shared_package.lambda_shared.pr import Labels @@ -27,7 +27,7 @@ IMAGE_NAME = "clickhouse/binary-builder" BUILD_LOG_NAME = "build_log.log" -def _can_export_binaries(build_config: BuildConfig) -> bool: +def _can_export_binaries(build_config: CI.BuildConfig) -> bool: if build_config.package_type != "deb": return False if build_config.sanitizer != "": @@ -38,7 +38,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool: def get_packager_cmd( - build_config: BuildConfig, + build_config: CI.BuildConfig, packager_path: Path, output_path: Path, build_version: str, @@ -147,7 +147,8 @@ def main(): stopwatch = Stopwatch() build_name = args.build_name - build_config = CI_CONFIG.build_config[build_name] + build_config = CI.JOB_CONFIGS[build_name].build_config + assert build_config temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index a641f9f4544..8482abb26e0 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -10,7 +10,7 @@ from typing import Any, Callable, List, Optional, Union import requests -from ci_config import CI_CONFIG +from ci_config import CI try: # A work around for scripts using this downloading module without required deps @@ -122,10 +122,6 @@ def get_gh_api( raise APIException(f"Unable to request data from GH API: {url}") from exc -def get_build_name_for_check(check_name: str) -> str: - return CI_CONFIG.test_configs[check_name].required_build - - def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str]: for root, _, files in os.walk(reports_path): for file in files: @@ -210,7 +206,7 @@ def download_builds_filter( result_path: Path, filter_fn: Callable[[str], bool] = lambda _: True, ) -> None: - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) logger.info("The build report for %s contains the next URLs: %s", build_name, urls) @@ -247,7 +243,7 @@ def download_clickhouse_binary( def get_clickhouse_binary_url( check_name: str, reports_path: Union[Path, str] ) -> Optional[str]: - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) logger.info("The build report for %s contains the next URLs: %s", build_name, urls) for url in urls: diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 1d734fbb3f8..5f24fd8f224 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -1,4 +1,5 @@ #!/usr/bin/env python3 +import argparse import json import logging import os @@ -6,7 +7,6 @@ import sys from pathlib import Path from typing import List -from ci_config import CI_CONFIG, Build from env_helper import ( GITHUB_JOB_URL, GITHUB_REPOSITORY, @@ -14,7 +14,7 @@ from env_helper import ( REPORT_PATH, TEMP_PATH, CI_CONFIG_PATH, - CI, + IS_CI, ) from pr_info import PRInfo from report import ( @@ -27,6 +27,7 @@ from report import ( get_worst_status, ) from stopwatch import Stopwatch +from ci_config import CI # Old way to read the neads_data NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "") @@ -46,16 +47,13 @@ def main(): "\n ".join(p.as_posix() for p in reports_path.rglob("*.json")), ) - build_check_name = sys.argv[1] + build_check_name = CI.JobNames.BUILD_CHECK pr_info = PRInfo() - builds_for_check = CI_CONFIG.get_builds_for_report( - build_check_name, - release=pr_info.is_release, - backport=pr_info.head_ref.startswith("backport/"), - ) - if CI: + args = parse_args() + + if (CI_CONFIG_PATH or IS_CI) and not args.reports: # In CI only specific builds might be manually selected, or some wf does not build all builds. # Filtering @builds_for_check to verify only builds that are present in the current CI workflow with open(CI_CONFIG_PATH, encoding="utf-8") as jfd: @@ -64,8 +62,12 @@ def main(): ci_config["jobs_data"]["jobs_to_skip"] + ci_config["jobs_data"]["jobs_to_do"] ) - builds_for_check = [job for job in builds_for_check if job in all_ci_jobs] - print(f"NOTE: following build reports will be accounted: [{builds_for_check}]") + builds_for_check = [job for job in CI.BuildNames if job in all_ci_jobs] + print(f"NOTE: following build reports will be checked: [{builds_for_check}]") + else: + builds_for_check = parse_args().reports + for job in builds_for_check: + assert job in CI.BuildNames, "Builds must be known build job names" required_builds = len(builds_for_check) missing_builds = 0 @@ -77,8 +79,8 @@ def main(): build_name, pr_info.number, pr_info.head_ref ) if not build_result: - if build_name == Build.FUZZERS: - logging.info("Build [%s] is missing - skip", Build.FUZZERS) + if build_name == CI.BuildNames.FUZZERS: + logging.info("Build [%s] is missing - skip", CI.BuildNames.FUZZERS) continue logging.warning("Build results for %s is missing", build_name) build_result = BuildResult.missing_result("missing") @@ -158,5 +160,16 @@ def main(): sys.exit(1) +def parse_args(): + parser = argparse.ArgumentParser("Generates overall build report") + + parser.add_argument( + "--reports", + nargs="+", + help="List of build reports to check", + ) + return parser.parse_args() + + if __name__ == "__main__": main() diff --git a/tests/ci/ci.py b/tests/ci/ci.py index bb23de142df..cdf1be96b68 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -13,14 +13,7 @@ from typing import Any, Dict, List, Optional import docker_images_helper import upload_result_helper from build_check import get_release_or_pr -from ci_config import ( - CI_CONFIG, - Build, - CILabels, - CIStages, - JobNames, - StatusNames, -) +from ci_config import CI from ci_metadata import CiMetadata from ci_utils import GHActions, normalize_string from clickhouse_helper import ( @@ -41,7 +34,7 @@ from commit_status_helper import ( ) from digest_helper import DockerDigester from env_helper import ( - CI, + IS_CI, GITHUB_JOB_API_URL, GITHUB_REPOSITORY, GITHUB_RUN_ID, @@ -295,7 +288,7 @@ def _mark_success_action( batch: int, ) -> None: ci_cache = CiCache(s3, indata["jobs_data"]["digests"]) - job_config = CI_CONFIG.get_job_config(job) + job_config = CI.get_job_config(job) num_batches = job_config.num_batches # if batch is not provided - set to 0 batch = 0 if batch == -1 else batch @@ -305,7 +298,7 @@ def _mark_success_action( # FIXME: find generic design for propagating and handling job status (e.g. stop using statuses in GH api) # now job ca be build job w/o status data, any other job that exit with 0 with or w/o status data - if CI_CONFIG.is_build_job(job): + if CI.is_build_job(job): # there is no CommitStatus for build jobs # create dummy status relying on JobReport # FIXME: consider creating commit status for build jobs too, to treat everything the same way @@ -425,6 +418,7 @@ def _configure_jobs( pr_info: PRInfo, ci_settings: CiSettings, skip_jobs: bool, + dry_run: bool = False, ) -> CiCache: """ returns CICache instance with configured job's data @@ -436,10 +430,11 @@ def _configure_jobs( # get all jobs if not skip_jobs: - job_configs = CI_CONFIG.get_workflow_jobs_with_configs( + job_configs = CI.get_workflow_jobs_with_configs( is_mq=pr_info.is_merge_queue, is_docs_only=pr_info.has_changes_in_documentation_only(), is_master=pr_info.is_master, + is_pr=pr_info.is_pr, ) else: job_configs = {} @@ -457,7 +452,8 @@ def _configure_jobs( ci_cache = CiCache.calc_digests_and_create( s3, job_configs, - cache_enabled=not ci_settings.no_ci_cache and not skip_jobs and CI, + cache_enabled=not ci_settings.no_ci_cache and not skip_jobs and IS_CI, + dry_run=dry_run, ) ci_cache.update() ci_cache.apply(job_configs, is_release=pr_info.is_release) @@ -475,14 +471,14 @@ def _generate_ci_stage_config(jobs_data: Dict[str, Any]) -> Dict[str, Dict[str, result = {} # type: Dict[str, Any] stages_to_do = [] for job in jobs_data: - stage_type = CI_CONFIG.get_job_ci_stage(job) - if stage_type == CIStages.NA: + stage_type = CI.get_job_ci_stage(job) + if stage_type == CI.WorkflowStages.NA: continue if stage_type not in result: result[stage_type] = [] stages_to_do.append(stage_type) result[stage_type].append( - {"job_name": job, "runner_type": CI_CONFIG.get_runner_type(job)} + {"job_name": job, "runner_type": CI.JOB_CONFIGS[job].runner_type} ) result["stages_to_do"] = stages_to_do return result @@ -529,10 +525,10 @@ def _update_gh_statuses_action(indata: Dict, s3: S3Helper) -> None: if job not in jobs_to_skip and job not in jobs_to_do: # no need to create status for job that are not supposed to be executed continue - if CI_CONFIG.is_build_job(job): + if CI.is_build_job(job): # no GH status for build jobs continue - job_config = CI_CONFIG.get_job_config(job) + job_config = CI.get_job_config(job) if not job_config: # there might be a new job that does not exist on this branch - skip it continue @@ -558,7 +554,7 @@ def _fetch_commit_tokens(message: str, pr_info: PRInfo) -> List[str]: res = [ match for match in matches - if match in CILabels or match.startswith("job_") or match.startswith("batch_") + if match in CI.Tags or match.startswith("job_") or match.startswith("batch_") ] print(f"CI modifiers from commit message: [{res}]") res_2 = [] @@ -567,7 +563,7 @@ def _fetch_commit_tokens(message: str, pr_info: PRInfo) -> List[str]: res_2 = [ match for match in matches - if match in CILabels + if match in CI.Tags or match.startswith("job_") or match.startswith("batch_") ] @@ -643,7 +639,7 @@ def _upload_build_artifacts( print(f"Report file has been uploaded to [{report_url}]") # Upload master head's binaries - static_bin_name = CI_CONFIG.build_config[build_name].static_binary_name + static_bin_name = CI.get_build_config(build_name).static_binary_name if pr_info.is_master and static_bin_name: # Full binary with debug info: s3_path_full = "/".join((pr_info.base_ref, static_bin_name, "clickhouse-full")) @@ -838,15 +834,15 @@ def _add_build_to_version_history( def _run_test(job_name: str, run_command: str) -> int: assert ( - run_command or CI_CONFIG.get_job_config(job_name).run_command + run_command or CI.get_job_config(job_name).run_command ), "Run command must be provided as input argument or be configured in job config" env = os.environ.copy() - timeout = CI_CONFIG.get_job_config(job_name).timeout or None + timeout = CI.get_job_config(job_name).timeout or None if not run_command: run_command = "/".join( - (os.path.dirname(__file__), CI_CONFIG.get_job_config(job_name).run_command) + (os.path.dirname(__file__), CI.get_job_config(job_name).run_command) ) if ".py" in run_command and not run_command.startswith("python"): run_command = "python3 " + run_command @@ -918,7 +914,7 @@ def _set_pending_statuses(pr_info: PRInfo) -> None: state=PENDING, target_url="", description="", - context=StatusNames.SYNC, + context=CI.StatusNames.SYNC, ) except Exception as ex: print(f"ERROR: failed to set GH commit status, ex: {ex}") @@ -952,7 +948,7 @@ def main() -> int: ### CONFIGURE action: start if args.configure: - if CI and pr_info.is_pr: + if IS_CI and pr_info.is_pr: # store meta on s3 (now we need it only for PRs) meta = CiMetadata(s3, pr_info.number, pr_info.head_ref) meta.run_id = int(GITHUB_RUN_ID) @@ -962,7 +958,7 @@ def main() -> int: args.commit_message or None, update_from_api=True ) - if ci_settings.no_merge_commit and CI: + if ci_settings.no_merge_commit and IS_CI: git_runner.run(f"{GIT_PREFIX} checkout {pr_info.sha}") git_ref = git_runner.run(f"{GIT_PREFIX} rev-parse HEAD") @@ -985,18 +981,19 @@ def main() -> int: ) ci_cache.print_status() - if CI and not pr_info.is_merge_queue: + if IS_CI and not pr_info.is_merge_queue: # wait for pending jobs to be finished, await_jobs is a long blocking call ci_cache.await_pending_jobs(pr_info.is_release) if pr_info.is_release: + print("Release/master: CI Cache add pending records for all todo jobs") ci_cache.push_pending_all(pr_info.is_release) # conclude results result["git_ref"] = git_ref result["version"] = version - result["build"] = ci_cache.job_digests[Build.PACKAGE_RELEASE] - result["docs"] = ci_cache.job_digests[JobNames.DOCS_CHECK] + result["build"] = ci_cache.job_digests[CI.BuildNames.PACKAGE_RELEASE] + result["docs"] = ci_cache.job_digests[CI.JobNames.DOCS_CHECK] result["ci_settings"] = ci_settings.as_dict() if not args.skip_jobs: result["stages_data"] = _generate_ci_stage_config(ci_cache.jobs_to_do) @@ -1027,7 +1024,7 @@ def main() -> int: f"Check if rerun for name: [{check_name}], extended name [{check_name_with_group}]" ) previous_status = None - if CI_CONFIG.is_build_job(check_name): + if CI.is_build_job(check_name): # this is a build job - check if a build report is present build_result = ( BuildResult.load_any(check_name, pr_info.number, pr_info.head_ref) @@ -1055,10 +1052,8 @@ def main() -> int: # rerun helper check # FIXME: remove rerun_helper check and rely on ci cache only if check_name not in ( - # we might want to rerun reports' jobs - disable rerun check for them - JobNames.BUILD_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - ): + CI.JobNames.BUILD_CHECK, + ): # we might want to rerun build report job rerun_helper = RerunHelper(commit, check_name_with_group) if rerun_helper.is_already_finished_by_status(): status = rerun_helper.get_finished_status() @@ -1071,7 +1066,7 @@ def main() -> int: # ci cache check if not previous_status and not ci_settings.no_ci_cache: ci_cache = CiCache(s3, indata["jobs_data"]["digests"]).update() - job_config = CI_CONFIG.get_job_config(check_name) + job_config = CI.get_job_config(check_name) if ci_cache.is_successful( check_name, args.batch, @@ -1111,7 +1106,7 @@ def main() -> int: ch_helper = ClickHouseHelper() check_url = "" - if CI_CONFIG.is_build_job(args.job_name): + if CI.is_build_job(args.job_name): assert ( indata ), f"--infile with config must be provided for POST action of a build type job [{args.job_name}]" @@ -1119,8 +1114,7 @@ def main() -> int: # upload binaries only for normal builds in PRs upload_binary = ( not pr_info.is_pr - or args.job_name - not in CI_CONFIG.get_builds_for_report(JobNames.BUILD_CHECK_SPECIAL) + or CI.get_job_ci_stage(args.job_name) == CI.WorkflowStages.BUILDS_1 or CiSettings.create_from_run_config(indata).upload_all ) diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index 56a84272a63..8ee0ae54385 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -5,7 +5,8 @@ from enum import Enum from pathlib import Path from typing import Dict, Optional, Any, Union, Sequence, List, Set -from ci_config import JobNames, Build, CI_CONFIG, JobConfig +from ci_config import CI + from ci_utils import is_hex, GHActions from commit_status_helper import CommitStatusData from env_helper import ( @@ -41,7 +42,7 @@ class CiCache: release - for jobs being executed on the release branch including master branch (not a PR branch) """ - _REQUIRED_DIGESTS = [JobNames.DOCS_CHECK, Build.PACKAGE_RELEASE] + _REQUIRED_DIGESTS = [CI.JobNames.DOCS_CHECK, CI.BuildNames.PACKAGE_RELEASE] _S3_CACHE_PREFIX = "CI_cache_v1" _CACHE_BUILD_REPORT_PREFIX = "build_report" _RECORD_FILE_EXTENSION = ".ci" @@ -80,7 +81,7 @@ class CiCache: @classmethod def is_docs_job(cls, job_name: str) -> bool: - return job_name == JobNames.DOCS_CHECK + return job_name == CI.JobNames.DOCS_CHECK @classmethod def is_srcs_job(cls, job_name: str) -> bool: @@ -105,8 +106,8 @@ class CiCache: ): self.enabled = cache_enabled self.jobs_to_skip = [] # type: List[str] - self.jobs_to_wait = {} # type: Dict[str, JobConfig] - self.jobs_to_do = {} # type: Dict[str, JobConfig] + self.jobs_to_wait = {} # type: Dict[str, CI.JobConfig] + self.jobs_to_do = {} # type: Dict[str, CI.JobConfig] self.s3 = s3 self.job_digests = job_digests self.cache_s3_paths = { @@ -127,9 +128,13 @@ class CiCache: @classmethod def calc_digests_and_create( - cls, s3: S3Helper, job_configs: Dict[str, JobConfig], cache_enabled: bool = True + cls, + s3: S3Helper, + job_configs: Dict[str, CI.JobConfig], + cache_enabled: bool = True, + dry_run: bool = False, ) -> "CiCache": - job_digester = JobDigester() + job_digester = JobDigester(dry_run=dry_run) digests = {} print("::group::Job Digests") @@ -140,9 +145,7 @@ class CiCache: for job in cls._REQUIRED_DIGESTS: if job not in job_configs: - digest = job_digester.get_job_digest( - CI_CONFIG.get_job_config(job).digest - ) + digest = job_digester.get_job_digest(CI.get_job_config(job).digest) digests[job] = digest print( f" job [{job.rjust(50)}] required for CI Cache has digest [{digest}]" @@ -154,10 +157,10 @@ class CiCache: self, job_digests: Dict[str, str], job_type: JobType ) -> str: if job_type == self.JobType.DOCS: - res = job_digests[JobNames.DOCS_CHECK] + res = job_digests[CI.JobNames.DOCS_CHECK] elif job_type == self.JobType.SRCS: - if Build.PACKAGE_RELEASE in job_digests: - res = job_digests[Build.PACKAGE_RELEASE] + if CI.BuildNames.PACKAGE_RELEASE in job_digests: + res = job_digests[CI.BuildNames.PACKAGE_RELEASE] else: assert False, "BUG, no build job in digest' list" else: @@ -648,7 +651,7 @@ class CiCache: report_path = Path(REPORT_PATH) report_path.mkdir(exist_ok=True, parents=True) path = ( - self._get_record_s3_path(Build.PACKAGE_RELEASE) + self._get_record_s3_path(CI.BuildNames.PACKAGE_RELEASE) + self._CACHE_BUILD_REPORT_PREFIX ) if file_prefix: @@ -664,13 +667,14 @@ class CiCache: def upload_build_report(self, build_result: BuildResult) -> str: result_json_path = build_result.write_json(Path(TEMP_PATH)) s3_path = ( - self._get_record_s3_path(Build.PACKAGE_RELEASE) + result_json_path.name + self._get_record_s3_path(CI.BuildNames.PACKAGE_RELEASE) + + result_json_path.name ) return self.s3.upload_file( bucket=S3_BUILDS_BUCKET, file_path=result_json_path, s3_path=s3_path ) - def await_pending_jobs(self, is_release: bool) -> None: + def await_pending_jobs(self, is_release: bool, dry_run: bool = False) -> None: """ await pending jobs to be finished @jobs_with_params - jobs to await. {JOB_NAME: {"batches": [BATCHES...], "num_batches": NUM_BATCHES}} @@ -687,15 +691,9 @@ class CiCache: MAX_JOB_NUM_TO_WAIT = 3 round_cnt = 0 - # FIXME: temporary experiment: lets enable await for PR' workflows awaiting on build' jobs only + # FIXME: temporary experiment: lets enable await for PR' workflows but for a shorter time if not is_release: - MAX_ROUNDS_TO_WAIT = 1 - remove_from_wait = [] - for job in self.jobs_to_wait: - if job not in Build: - remove_from_wait.append(job) - for job in remove_from_wait: - del self.jobs_to_wait[job] + MAX_ROUNDS_TO_WAIT = 3 while ( len(self.jobs_to_wait) > MAX_JOB_NUM_TO_WAIT @@ -713,11 +711,12 @@ class CiCache: start_at = int(time.time()) while expired_sec < TIMEOUT and self.jobs_to_wait: await_finished: Set[str] = set() - time.sleep(poll_interval_sec) + if not dry_run: + time.sleep(poll_interval_sec) self.update() for job_name, job_config in self.jobs_to_wait.items(): num_batches = job_config.num_batches - job_config = CI_CONFIG.get_job_config(job_name) + job_config = CI.get_job_config(job_name) assert job_config.pending_batches assert job_config.batches pending_batches = list(job_config.pending_batches) @@ -741,12 +740,11 @@ class CiCache: f"Job [{job_name}_[{batch}/{num_batches}]] is not pending anymore" ) job_config.batches.remove(batch) - job_config.pending_batches.remove(batch) else: print( f"NOTE: Job [{job_name}:{batch}] finished failed - do not add to ready" ) - job_config.pending_batches.remove(batch) + job_config.pending_batches.remove(batch) if not job_config.pending_batches: await_finished.add(job_name) @@ -754,18 +752,25 @@ class CiCache: for job in await_finished: self.jobs_to_skip.append(job) del self.jobs_to_wait[job] + del self.jobs_to_do[job] - expired_sec = int(time.time()) - start_at - print( - f"...awaiting continues... seconds left [{TIMEOUT - expired_sec}]" - ) + if not dry_run: + expired_sec = int(time.time()) - start_at + print( + f"...awaiting continues... seconds left [{TIMEOUT - expired_sec}]" + ) + else: + # make up for 2 iterations in dry_run + expired_sec += int(TIMEOUT / 2) + 1 GHActions.print_in_group( "Remaining jobs:", [list(self.jobs_to_wait)], ) - def apply(self, job_configs: Dict[str, JobConfig], is_release: bool) -> "CiCache": + def apply( + self, job_configs: Dict[str, CI.JobConfig], is_release: bool + ) -> "CiCache": if not self.enabled: self.jobs_to_do = job_configs return self diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 6ab1eb8bac4..3dc0040663b 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1,749 +1,609 @@ -#!/usr/bin/env python3 - -import logging import random import re from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser -from copy import deepcopy -from dataclasses import dataclass, field -from pathlib import Path -from typing import Callable, Dict, Iterable, List, Literal, Optional, Union +from typing import Dict, Optional, List -from ci_utils import WithIter, normalize_string -from integration_test_images import IMAGES +from ci_utils import normalize_string +from ci_definitions import * -class WorkFlows(metaclass=WithIter): - PULL_REQUEST = "PULL_REQUEST" - MASTER = "MASTER" - BACKPORT = "BACKPORT" - RELEASE = "RELEASE" - SYNC = "SYNC" - - -class CIStages(metaclass=WithIter): - NA = "UNKNOWN" - BUILDS_1 = "Builds_1" - BUILDS_2 = "Builds_2" - TESTS_1 = "Tests_1" - TESTS_2 = "Tests_2" - TESTS_3 = "Tests_3" - - -class Runners(metaclass=WithIter): - BUILDER = "builder" - STYLE_CHECKER = "style-checker" - STYLE_CHECKER_ARM = "style-checker-aarch64" - FUNC_TESTER = "func-tester" - FUNC_TESTER_ARM = "func-tester-aarch64" - STRESS_TESTER = "stress-tester" - FUZZER_UNIT_TESTER = "fuzzer-unit-tester" - - -class CILabels(metaclass=WithIter): - """ - Label names or commit tokens in normalized form - """ - - DO_NOT_TEST_LABEL = "do_not_test" - NO_MERGE_COMMIT = "no_merge_commit" - NO_CI_CACHE = "no_ci_cache" - # to upload all binaries from build jobs - UPLOAD_ALL_ARTIFACTS = "upload_all" - CI_SET_SYNC = "ci_set_sync" - CI_SET_ARM = "ci_set_arm" - CI_SET_REQUIRED = "ci_set_required" - CI_SET_NORMAL_BUILDS = "ci_set_normal_builds" - CI_SET_SPECIAL_BUILDS = "ci_set_special_builds" - CI_SET_NON_REQUIRED = "ci_set_non_required" - CI_SET_OLD_ANALYZER = "ci_set_old_analyzer" - - libFuzzer = "libFuzzer" - - -class Build(metaclass=WithIter): - PACKAGE_RELEASE = "package_release" - PACKAGE_AARCH64 = "package_aarch64" - PACKAGE_ASAN = "package_asan" - PACKAGE_UBSAN = "package_ubsan" - PACKAGE_TSAN = "package_tsan" - PACKAGE_MSAN = "package_msan" - PACKAGE_DEBUG = "package_debug" - PACKAGE_RELEASE_COVERAGE = "package_release_coverage" - BINARY_RELEASE = "binary_release" - BINARY_TIDY = "binary_tidy" - BINARY_DARWIN = "binary_darwin" - BINARY_AARCH64 = "binary_aarch64" - BINARY_AARCH64_V80COMPAT = "binary_aarch64_v80compat" - BINARY_FREEBSD = "binary_freebsd" - BINARY_DARWIN_AARCH64 = "binary_darwin_aarch64" - BINARY_PPC64LE = "binary_ppc64le" - BINARY_AMD64_COMPAT = "binary_amd64_compat" - BINARY_AMD64_MUSL = "binary_amd64_musl" - BINARY_RISCV64 = "binary_riscv64" - BINARY_S390X = "binary_s390x" - BINARY_LOONGARCH64 = "binary_loongarch64" - FUZZERS = "fuzzers" - - -class JobNames(metaclass=WithIter): - STYLE_CHECK = "Style check" - FAST_TEST = "Fast test" - DOCKER_SERVER = "Docker server image" - DOCKER_KEEPER = "Docker keeper image" - INSTALL_TEST_AMD = "Install packages (amd64)" - INSTALL_TEST_ARM = "Install packages (arm64)" - - STATELESS_TEST_DEBUG = "Stateless tests (debug)" - STATELESS_TEST_RELEASE = "Stateless tests (release)" - STATELESS_TEST_RELEASE_COVERAGE = "Stateless tests (coverage)" - STATELESS_TEST_AARCH64 = "Stateless tests (aarch64)" - STATELESS_TEST_ASAN = "Stateless tests (asan)" - STATELESS_TEST_TSAN = "Stateless tests (tsan)" - STATELESS_TEST_MSAN = "Stateless tests (msan)" - STATELESS_TEST_UBSAN = "Stateless tests (ubsan)" - STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE = ( - "Stateless tests (release, old analyzer, s3, DatabaseReplicated)" - ) - # merged into STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: - # STATELESS_TEST_OLD_ANALYZER_RELEASE = "Stateless tests (release, analyzer)" - # STATELESS_TEST_DB_REPL_RELEASE = "Stateless tests (release, DatabaseReplicated)" - # STATELESS_TEST_S3_RELEASE = "Stateless tests (release, s3 storage)" - STATELESS_TEST_S3_DEBUG = "Stateless tests (debug, s3 storage)" - STATELESS_TEST_S3_TSAN = "Stateless tests (tsan, s3 storage)" - STATELESS_TEST_AZURE_ASAN = "Stateless tests (azure, asan)" - STATELESS_TEST_FLAKY_ASAN = "Stateless tests flaky check (asan)" - - STATEFUL_TEST_DEBUG = "Stateful tests (debug)" - STATEFUL_TEST_RELEASE = "Stateful tests (release)" - STATEFUL_TEST_RELEASE_COVERAGE = "Stateful tests (coverage)" - STATEFUL_TEST_AARCH64 = "Stateful tests (aarch64)" - STATEFUL_TEST_ASAN = "Stateful tests (asan)" - STATEFUL_TEST_TSAN = "Stateful tests (tsan)" - STATEFUL_TEST_MSAN = "Stateful tests (msan)" - STATEFUL_TEST_UBSAN = "Stateful tests (ubsan)" - STATEFUL_TEST_PARALLEL_REPL_RELEASE = "Stateful tests (release, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_DEBUG = "Stateful tests (debug, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_ASAN = "Stateful tests (asan, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_MSAN = "Stateful tests (msan, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_UBSAN = "Stateful tests (ubsan, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_TSAN = "Stateful tests (tsan, ParallelReplicas)" - - STRESS_TEST_ASAN = "Stress test (asan)" - STRESS_TEST_TSAN = "Stress test (tsan)" - STRESS_TEST_UBSAN = "Stress test (ubsan)" - STRESS_TEST_MSAN = "Stress test (msan)" - STRESS_TEST_DEBUG = "Stress test (debug)" - STRESS_TEST_AZURE_TSAN = "Stress test (azure, tsan)" - STRESS_TEST_AZURE_MSAN = "Stress test (azure, msan)" - - INTEGRATION_TEST = "Integration tests (release)" - INTEGRATION_TEST_ASAN = "Integration tests (asan)" - INTEGRATION_TEST_ASAN_OLD_ANALYZER = "Integration tests (asan, old analyzer)" - INTEGRATION_TEST_TSAN = "Integration tests (tsan)" - INTEGRATION_TEST_ARM = "Integration tests (aarch64)" - INTEGRATION_TEST_FLAKY = "Integration tests flaky check (asan)" - - UPGRADE_TEST_DEBUG = "Upgrade check (debug)" - UPGRADE_TEST_ASAN = "Upgrade check (asan)" - UPGRADE_TEST_TSAN = "Upgrade check (tsan)" - UPGRADE_TEST_MSAN = "Upgrade check (msan)" - - UNIT_TEST = "Unit tests (release)" - UNIT_TEST_ASAN = "Unit tests (asan)" - UNIT_TEST_MSAN = "Unit tests (msan)" - UNIT_TEST_TSAN = "Unit tests (tsan)" - UNIT_TEST_UBSAN = "Unit tests (ubsan)" - - AST_FUZZER_TEST_DEBUG = "AST fuzzer (debug)" - AST_FUZZER_TEST_ASAN = "AST fuzzer (asan)" - AST_FUZZER_TEST_MSAN = "AST fuzzer (msan)" - AST_FUZZER_TEST_TSAN = "AST fuzzer (tsan)" - AST_FUZZER_TEST_UBSAN = "AST fuzzer (ubsan)" - - JEPSEN_KEEPER = "ClickHouse Keeper Jepsen" - JEPSEN_SERVER = "ClickHouse Server Jepsen" - - PERFORMANCE_TEST_AMD64 = "Performance Comparison" - PERFORMANCE_TEST_ARM64 = "Performance Comparison Aarch64" - - SQL_LOGIC_TEST = "Sqllogic test (release)" - - SQLANCER = "SQLancer (release)" - SQLANCER_DEBUG = "SQLancer (debug)" - SQLTEST = "SQLTest" - - COMPATIBILITY_TEST = "Compatibility check (amd64)" - COMPATIBILITY_TEST_ARM = "Compatibility check (aarch64)" - - CLICKBENCH_TEST = "ClickBench (amd64)" - CLICKBENCH_TEST_ARM = "ClickBench (aarch64)" - - LIBFUZZER_TEST = "libFuzzer tests" - - BUILD_CHECK = "ClickHouse build check" - BUILD_CHECK_SPECIAL = "ClickHouse special build check" - - DOCS_CHECK = "Docs check" - BUGFIX_VALIDATE = "Bugfix validation" - - -class StatusNames(metaclass=WithIter): - "Class with statuses that aren't related to particular jobs" - CI = "CI running" - MERGEABLE = "Mergeable Check" - SYNC = "A Sync" - - -# dynamically update JobName with Build jobs -for attr_name in dir(Build): - if not attr_name.startswith("__") and not callable(getattr(Build, attr_name)): - setattr(JobNames, attr_name, getattr(Build, attr_name)) - - -@dataclass -class DigestConfig: - # all files, dirs to include into digest, glob supported - include_paths: List[Union[str, Path]] = field(default_factory=list) - # file suffixes to exclude from digest - exclude_files: List[str] = field(default_factory=list) - # directories to exclude from digest - exclude_dirs: List[Union[str, Path]] = field(default_factory=list) - # docker names to include into digest - docker: List[str] = field(default_factory=list) - # git submodules digest - git_submodules: bool = False - - -@dataclass -class LabelConfig: - """ - configures different CI scenarios per GH label - """ - - run_jobs: Iterable[str] = frozenset() - - -@dataclass -class JobConfig: - """ - contains config parameters for job execution in CI workflow - """ - - # configures digest calculation for the job - digest: DigestConfig = field(default_factory=DigestConfig) - # will be triggered for the job if omitted in CI workflow yml - run_command: str = "" - # job timeout, seconds - timeout: Optional[int] = None - # sets number of batches for a multi-batch job - num_batches: int = 1 - # label that enables job in CI, if set digest isn't used - run_by_label: str = "" - # to run always regardless of the job digest or/and label - run_always: bool = False - # if the job needs to be run on the release branch, including master (building packages, docker server). - # NOTE: Subsequent runs on the same branch with the similar digest are still considered skip-able. - required_on_release_branch: bool = False - # job is for pr workflow only - pr_only: bool = False - # job is for release/master branches only - release_only: bool = False - # to randomly pick and run one job among jobs in the same @random_bucket (PR branches only). - random_bucket: str = "" - # Do not set it. A list of batches to run. It will be set in runtime in accordance with ci cache and ci settings - batches: Optional[List[int]] = None - # Do not set it. A list of batches to await. It will be set in runtime in accordance with ci cache and ci settings - pending_batches: Optional[List[int]] = None - - -builds_job_config = JobConfig( - required_on_release_branch=True, - digest=DigestConfig( - include_paths=[ - "./src", - "./contrib/*-cmake", - "./contrib/consistent-hashing", - "./contrib/murmurhash", - "./contrib/libfarmhash", - "./contrib/pdqsort", - "./contrib/cityhash102", - "./contrib/sparse-checkout", - "./contrib/libmetrohash", - "./contrib/update-submodules.sh", - "./contrib/CMakeLists.txt", - "./CMakeLists.txt", - "./PreLoad.cmake", - "./cmake", - "./base", - "./programs", - "./packages", - "./docker/packager/packager", - "./rust", - "./tests/ci/version_helper.py", - # FIXME: This is a WA to rebuild the CH and recreate the Performance.tar.zst artifact - # when there are changes in performance test scripts. - # Due to the current design of the perf test we need to rebuild CH when the performance test changes, - # otherwise the changes will not be visible in the PerformanceTest job in CI - "./tests/performance", - ], - exclude_files=[".md"], - docker=["clickhouse/binary-builder"], - git_submodules=True, - ), - run_command="build_check.py $BUILD_NAME", -) -fuzzer_build_job_config = deepcopy(builds_job_config) -fuzzer_build_job_config.run_by_label = CILabels.libFuzzer - - -@dataclass -class BuildConfig: - name: str - compiler: str - package_type: Literal["deb", "binary", "fuzzers"] - additional_pkgs: bool = False - debug_build: bool = False - coverage: bool = False - sanitizer: str = "" - tidy: bool = False - # sparse_checkout is needed only to test the option itself. - # No particular sense to use it in every build, since it slows down the job. - sparse_checkout: bool = False - comment: str = "" - static_binary_name: str = "" - job_config: JobConfig = field(default_factory=lambda: deepcopy(builds_job_config)) - - def export_env(self, export: bool = False) -> str: - def process(field_name: str, field: Union[bool, str]) -> str: - if isinstance(field, bool): - field = str(field).lower() - elif not isinstance(field, str): - field = "" - if export: - return f"export BUILD_{field_name.upper()}={repr(field)}" - return f"BUILD_{field_name.upper()}={field}" - - return "\n".join(process(k, v) for k, v in self.__dict__.items()) - - -@dataclass -class BuildReportConfig: - builds: List[str] - job_config: JobConfig = field( - default_factory=lambda: JobConfig( - run_command='build_report_check.py "$CHECK_NAME"', - digest=DigestConfig( - include_paths=[ - "./tests/ci/build_report_check.py", - "./tests/ci/upload_result_helper.py", - ], - ), - ) - ) - - -@dataclass -class TestConfig: - required_build: str - job_config: JobConfig = field(default_factory=JobConfig) - - -BuildConfigs = Dict[str, BuildConfig] -BuildsReportConfig = Dict[str, BuildReportConfig] -TestConfigs = Dict[str, TestConfig] -LabelConfigs = Dict[str, LabelConfig] - -# common digests configs -compatibility_check_digest = DigestConfig( - include_paths=["./tests/ci/compatibility_check.py"], - docker=["clickhouse/test-old-ubuntu", "clickhouse/test-old-centos"], -) -install_check_digest = DigestConfig( - include_paths=["./tests/ci/install_check.py"], - docker=["clickhouse/install-deb-test", "clickhouse/install-rpm-test"], -) -stateless_check_digest = DigestConfig( - include_paths=[ - "./tests/ci/functional_test_check.py", - "./tests/queries/0_stateless/", - "./tests/clickhouse-test", - "./tests/config", - "./tests/*.txt", - ], - exclude_files=[".md"], - docker=["clickhouse/stateless-test"], -) -stateful_check_digest = DigestConfig( - include_paths=[ - "./tests/ci/functional_test_check.py", - "./tests/queries/1_stateful/", - "./tests/clickhouse-test", - "./tests/config", - "./tests/*.txt", - ], - exclude_files=[".md"], - docker=["clickhouse/stateful-test"], -) - -stress_check_digest = DigestConfig( - include_paths=[ - "./tests/queries/0_stateless/", - "./tests/queries/1_stateful/", - "./tests/clickhouse-test", - "./tests/config", - "./tests/*.txt", - ], - exclude_files=[".md"], - docker=["clickhouse/stress-test"], -) -# FIXME: which tests are upgrade? just python? -upgrade_check_digest = DigestConfig( - include_paths=["./tests/ci/upgrade_check.py"], - exclude_files=[".md"], - docker=["clickhouse/upgrade-check"], -) -integration_check_digest = DigestConfig( - include_paths=[ - "./tests/ci/integration_test_check.py", - "./tests/ci/integration_tests_runner.py", - "./tests/integration/", - ], - exclude_files=[".md"], - docker=IMAGES.copy(), -) - -ast_fuzzer_check_digest = DigestConfig( - # include_paths=["./tests/ci/ast_fuzzer_check.py"], - # exclude_files=[".md"], - # docker=["clickhouse/fuzzer"], -) -unit_check_digest = DigestConfig( - include_paths=["./tests/ci/unit_tests_check.py"], - exclude_files=[".md"], - docker=["clickhouse/unit-test"], -) -perf_check_digest = DigestConfig( - include_paths=[ - "./tests/ci/performance_comparison_check.py", - "./tests/performance/", - ], - exclude_files=[".md"], - docker=["clickhouse/performance-comparison"], -) -sqllancer_check_digest = DigestConfig( - # include_paths=["./tests/ci/sqlancer_check.py"], - # exclude_files=[".md"], - # docker=["clickhouse/sqlancer-test"], -) -sqllogic_check_digest = DigestConfig( - include_paths=["./tests/ci/sqllogic_test.py"], - exclude_files=[".md"], - docker=["clickhouse/sqllogic-test"], -) -sqltest_check_digest = DigestConfig( - include_paths=["./tests/ci/sqltest.py"], - exclude_files=[".md"], - docker=["clickhouse/sqltest"], -) -bugfix_validate_check = DigestConfig( - include_paths=[ - "./tests/queries/0_stateless/", - "./tests/ci/integration_test_check.py", - "./tests/ci/functional_test_check.py", - "./tests/ci/bugfix_validate_check.py", - ], - exclude_files=[".md"], - docker=IMAGES.copy() - + [ - "clickhouse/stateless-test", - ], -) -# common test params -docker_server_job_config = JobConfig( - required_on_release_branch=True, - run_command='docker_server.py --check-name "$CHECK_NAME" --release-type head --allow-build-reuse', - digest=DigestConfig( - include_paths=[ - "tests/ci/docker_server.py", - "./docker/server", - ] - ), -) -compatibility_test_common_params = { - "digest": compatibility_check_digest, - "run_command": "compatibility_check.py", -} -stateless_test_common_params = { - "digest": stateless_check_digest, - "run_command": 'functional_test_check.py "$CHECK_NAME"', - "timeout": 10800, -} -stateful_test_common_params = { - "digest": stateful_check_digest, - "run_command": 'functional_test_check.py "$CHECK_NAME"', - "timeout": 3600, -} -stress_test_common_params = { - "digest": stress_check_digest, - "run_command": "stress_check.py", - "timeout": 9000, -} -upgrade_test_common_params = { - "digest": upgrade_check_digest, - "run_command": "upgrade_check.py", -} -astfuzzer_test_common_params = { - "digest": ast_fuzzer_check_digest, - "run_command": "ast_fuzzer_check.py", - "run_always": True, -} -integration_test_common_params = { - "digest": integration_check_digest, - "run_command": 'integration_test_check.py "$CHECK_NAME"', -} -unit_test_common_params = { - "digest": unit_check_digest, - "run_command": "unit_tests_check.py", -} -perf_test_common_params = { - "digest": perf_check_digest, - "run_command": "performance_comparison_check.py", -} -sqllancer_test_common_params = JobConfig( - digest=sqllancer_check_digest, - run_command="sqlancer_check.py", - release_only=True, - run_always=True, -) -sqllogic_test_params = JobConfig( - digest=sqllogic_check_digest, - run_command="sqllogic_test.py", - timeout=10800, - release_only=True, -) -sql_test_params = JobConfig( - digest=sqltest_check_digest, - run_command="sqltest.py", - timeout=10800, - release_only=True, -) -clickbench_test_params = { - "digest": DigestConfig( - include_paths=[ - "tests/ci/clickbench.py", - ], - docker=["clickhouse/clickbench"], - ), - "run_command": 'clickbench.py "$CHECK_NAME"', - "timeout": 900, -} -install_test_params = JobConfig( - digest=install_check_digest, - run_command='install_check.py "$CHECK_NAME"', - timeout=900, -) - - -@dataclass -class CIConfig: +class CI: """ Contains configs for all jobs in the CI pipeline each config item in the below dicts should be an instance of JobConfig class or inherited from it """ - build_config: BuildConfigs - builds_report_config: BuildsReportConfig - test_configs: TestConfigs - other_jobs_configs: TestConfigs - label_configs: LabelConfigs + # reimport types to CI class so that they visible as CI.* and mypy is happy + # pylint:disable=useless-import-alias,reimported,import-outside-toplevel + from ci_definitions import BuildConfig as BuildConfig + from ci_definitions import DigestConfig as DigestConfig + from ci_definitions import JobConfig as JobConfig + from ci_definitions import CheckDescription as CheckDescription + from ci_definitions import Tags as Tags + from ci_definitions import JobNames as JobNames + from ci_definitions import BuildNames as BuildNames + from ci_definitions import StatusNames as StatusNames + from ci_definitions import CHECK_DESCRIPTIONS as CHECK_DESCRIPTIONS + from ci_definitions import MQ_JOBS as MQ_JOBS + from ci_definitions import WorkflowStages as WorkflowStages + from ci_definitions import Runners as Runners # Jobs that run for doc related updates _DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] - # Jobs that run in Merge Queue if it's enabled - _MQ_JOBS = [ - JobNames.STYLE_CHECK, - JobNames.FAST_TEST, - Build.BINARY_RELEASE, - JobNames.UNIT_TEST, - ] + TAG_CONFIGS = { + Tags.DO_NOT_TEST_LABEL: LabelConfig(run_jobs=[JobNames.STYLE_CHECK]), + Tags.CI_SET_ARM: LabelConfig( + run_jobs=[ + JobNames.STYLE_CHECK, + BuildNames.PACKAGE_AARCH64, + JobNames.INTEGRATION_TEST_ARM, + ] + ), + Tags.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), + Tags.CI_SET_BUILDS: LabelConfig( + run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK] + + [build for build in BuildNames if build != BuildNames.FUZZERS] + ), + Tags.CI_SET_NON_REQUIRED: LabelConfig( + run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] + ), + Tags.CI_SET_OLD_ANALYZER: LabelConfig( + run_jobs=[ + JobNames.STYLE_CHECK, + JobNames.FAST_TEST, + BuildNames.PACKAGE_RELEASE, + BuildNames.PACKAGE_ASAN, + JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE, + JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER, + ] + ), + Tags.CI_SET_SYNC: LabelConfig( + run_jobs=[ + BuildNames.PACKAGE_ASAN, + JobNames.STYLE_CHECK, + JobNames.BUILD_CHECK, + JobNames.UNIT_TEST_ASAN, + JobNames.STATEFUL_TEST_ASAN, + ] + ), + } - def get_label_config(self, label_name: str) -> Optional[LabelConfig]: - for label, config in self.label_configs.items(): + JOB_CONFIGS: Dict[str, JobConfig] = { + BuildNames.PACKAGE_RELEASE: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_RELEASE, + compiler="clang-18", + package_type="deb", + static_binary_name="amd64", + additional_pkgs=True, + ) + ), + BuildNames.PACKAGE_AARCH64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_AARCH64, + compiler="clang-18-aarch64", + package_type="deb", + static_binary_name="aarch64", + additional_pkgs=True, + ) + ), + BuildNames.PACKAGE_ASAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_ASAN, + compiler="clang-18", + sanitizer="address", + package_type="deb", + ), + ), + BuildNames.PACKAGE_UBSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_UBSAN, + compiler="clang-18", + sanitizer="undefined", + package_type="deb", + ), + ), + BuildNames.PACKAGE_TSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_TSAN, + compiler="clang-18", + sanitizer="thread", + package_type="deb", + ), + ), + BuildNames.PACKAGE_MSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_MSAN, + compiler="clang-18", + sanitizer="memory", + package_type="deb", + ), + ), + BuildNames.PACKAGE_DEBUG: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_DEBUG, + compiler="clang-18", + debug_build=True, + package_type="deb", + sparse_checkout=True, # Check that it works with at least one build, see also update-submodules.sh + ), + ), + BuildNames.PACKAGE_RELEASE_COVERAGE: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_RELEASE_COVERAGE, + compiler="clang-18", + coverage=True, + package_type="deb", + ), + ), + BuildNames.BINARY_RELEASE: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_RELEASE, + compiler="clang-18", + package_type="binary", + ), + ), + BuildNames.BINARY_TIDY: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_TIDY, + compiler="clang-18", + debug_build=True, + package_type="binary", + static_binary_name="debug-amd64", + tidy=True, + comment="clang-tidy is used for static analysis", + ), + ), + BuildNames.BINARY_DARWIN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_DARWIN, + compiler="clang-18-darwin", + package_type="binary", + static_binary_name="macos", + ), + ), + BuildNames.BINARY_AARCH64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_AARCH64, + compiler="clang-18-aarch64", + package_type="binary", + ), + ), + BuildNames.BINARY_AARCH64_V80COMPAT: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_AARCH64_V80COMPAT, + compiler="clang-18-aarch64-v80compat", + package_type="binary", + static_binary_name="aarch64v80compat", + comment="For ARMv8.1 and older", + ), + ), + BuildNames.BINARY_FREEBSD: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_FREEBSD, + compiler="clang-18-freebsd", + package_type="binary", + static_binary_name="freebsd", + ), + ), + BuildNames.BINARY_DARWIN_AARCH64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_DARWIN_AARCH64, + compiler="clang-18-darwin-aarch64", + package_type="binary", + static_binary_name="macos-aarch64", + ), + ), + BuildNames.BINARY_PPC64LE: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_PPC64LE, + compiler="clang-18-ppc64le", + package_type="binary", + static_binary_name="powerpc64le", + ), + ), + BuildNames.BINARY_AMD64_COMPAT: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_AMD64_COMPAT, + compiler="clang-18-amd64-compat", + package_type="binary", + static_binary_name="amd64compat", + comment="SSE2-only build", + ), + ), + BuildNames.BINARY_AMD64_MUSL: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_AMD64_MUSL, + compiler="clang-18-amd64-musl", + package_type="binary", + static_binary_name="amd64musl", + comment="Build with Musl", + ), + ), + BuildNames.BINARY_RISCV64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_RISCV64, + compiler="clang-18-riscv64", + package_type="binary", + static_binary_name="riscv64", + ), + ), + BuildNames.BINARY_S390X: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_S390X, + compiler="clang-18-s390x", + package_type="binary", + static_binary_name="s390x", + ), + ), + BuildNames.BINARY_LOONGARCH64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_LOONGARCH64, + compiler="clang-18-loongarch64", + package_type="binary", + static_binary_name="loongarch64", + ), + ), + BuildNames.FUZZERS: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.FUZZERS, + compiler="clang-18", + package_type="fuzzers", + ), + run_by_label=Tags.libFuzzer, + ), + JobNames.BUILD_CHECK: CommonJobConfigs.BUILD_REPORT.with_properties(), + JobNames.INSTALL_TEST_AMD: CommonJobConfigs.INSTALL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] + ), + JobNames.INSTALL_TEST_ARM: CommonJobConfigs.INSTALL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64] + ), + JobNames.STATEFUL_TEST_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN] + ), + JobNames.STATEFUL_TEST_TSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN] + ), + JobNames.STATEFUL_TEST_MSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN] + ), + JobNames.STATEFUL_TEST_UBSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN] + ), + JobNames.STATEFUL_TEST_DEBUG: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG] + ), + JobNames.STATEFUL_TEST_RELEASE: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] + ), + JobNames.STATEFUL_TEST_RELEASE_COVERAGE: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE] + ), + JobNames.STATEFUL_TEST_AARCH64: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64] + ), + JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] + ), + JobNames.STATEFUL_TEST_PARALLEL_REPL_DEBUG: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG] + ), + JobNames.STATEFUL_TEST_PARALLEL_REPL_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + random_bucket="parrepl_with_sanitizer", + ), + JobNames.STATEFUL_TEST_PARALLEL_REPL_MSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + random_bucket="parrepl_with_sanitizer", + ), + JobNames.STATEFUL_TEST_PARALLEL_REPL_UBSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], + random_bucket="parrepl_with_sanitizer", + ), + JobNames.STATEFUL_TEST_PARALLEL_REPL_TSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + random_bucket="parrepl_with_sanitizer", + ), + JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4 + ), + JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], num_batches=5 + ), + JobNames.STATELESS_TEST_MSAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], num_batches=6 + ), + JobNames.STATELESS_TEST_UBSAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=2 + ), + JobNames.STATELESS_TEST_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=5 + ), + JobNames.STATELESS_TEST_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + ), + JobNames.STATELESS_TEST_RELEASE_COVERAGE: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE], num_batches=6 + ), + JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], + ), + JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 + ), + JobNames.STATELESS_TEST_S3_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=6 + ), + JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4, release_only=True + ), + JobNames.STATELESS_TEST_S3_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + num_batches=5, + ), + JobNames.STRESS_TEST_DEBUG: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], + ), + JobNames.STRESS_TEST_TSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + ), + JobNames.STRESS_TEST_ASAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + random_bucket="stress_with_sanitizer", + ), + JobNames.STRESS_TEST_UBSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], + random_bucket="stress_with_sanitizer", + ), + JobNames.STRESS_TEST_MSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + random_bucket="stress_with_sanitizer", + ), + JobNames.STRESS_TEST_AZURE_TSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], release_only=True + ), + JobNames.STRESS_TEST_AZURE_MSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], release_only=True + ), + JobNames.UPGRADE_TEST_ASAN: CommonJobConfigs.UPGRADE_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + random_bucket="upgrade_with_sanitizer", + pr_only=True, + ), + JobNames.UPGRADE_TEST_TSAN: CommonJobConfigs.UPGRADE_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + random_bucket="upgrade_with_sanitizer", + pr_only=True, + ), + JobNames.UPGRADE_TEST_MSAN: CommonJobConfigs.UPGRADE_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + random_bucket="upgrade_with_sanitizer", + pr_only=True, + ), + JobNames.UPGRADE_TEST_DEBUG: CommonJobConfigs.UPGRADE_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], pr_only=True + ), + JobNames.INTEGRATION_TEST_ASAN: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], release_only=True, num_batches=4 + ), + JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=6 + ), + JobNames.INTEGRATION_TEST_TSAN: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], num_batches=6 + ), + JobNames.INTEGRATION_TEST_ARM: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=6 + ), + JobNames.INTEGRATION_TEST: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + num_batches=4, + release_only=True, + ), + JobNames.INTEGRATION_TEST_FLAKY: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True + ), + JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + required_on_release_branch=True, + ), + JobNames.COMPATIBILITY_TEST_ARM: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], + required_on_release_branch=True, + ), + JobNames.UNIT_TEST: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.BINARY_RELEASE], + ), + JobNames.UNIT_TEST_ASAN: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + ), + JobNames.UNIT_TEST_MSAN: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + ), + JobNames.UNIT_TEST_TSAN: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + ), + JobNames.UNIT_TEST_UBSAN: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], + ), + JobNames.AST_FUZZER_TEST_DEBUG: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], + ), + JobNames.AST_FUZZER_TEST_ASAN: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + ), + JobNames.AST_FUZZER_TEST_MSAN: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + ), + JobNames.AST_FUZZER_TEST_TSAN: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + ), + JobNames.AST_FUZZER_TEST_UBSAN: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], + ), + JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, timeout=3600 + ), + JobNames.JEPSEN_KEEPER: JobConfig( + required_builds=[BuildNames.BINARY_RELEASE], + run_by_label="jepsen-test", + run_command="jepsen_check.py keeper", + runner_type=Runners.STYLE_CHECKER_ARM, + ), + JobNames.JEPSEN_SERVER: JobConfig( + required_builds=[BuildNames.BINARY_RELEASE], + run_by_label="jepsen-test", + run_command="jepsen_check.py server", + runner_type=Runners.STYLE_CHECKER_ARM, + ), + JobNames.PERFORMANCE_TEST_AMD64: CommonJobConfigs.PERF_TESTS.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 + ), + JobNames.PERFORMANCE_TEST_ARM64: CommonJobConfigs.PERF_TESTS.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], + num_batches=4, + run_by_label="pr-performance", + ), + JobNames.SQLANCER: CommonJobConfigs.SQLLANCER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + ), + JobNames.SQLANCER_DEBUG: CommonJobConfigs.SQLLANCER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], + ), + JobNames.SQL_LOGIC_TEST: CommonJobConfigs.SQLLOGIC_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + ), + JobNames.SQLTEST: CommonJobConfigs.SQL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + ), + JobNames.CLICKBENCH_TEST: CommonJobConfigs.SQL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + ), + JobNames.CLICKBENCH_TEST_ARM: CommonJobConfigs.SQL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], + ), + JobNames.LIBFUZZER_TEST: JobConfig( + required_builds=[BuildNames.FUZZERS], + run_by_label=Tags.libFuzzer, + timeout=10800, + run_command='libfuzzer_test_check.py "$CHECK_NAME"', + runner_type=Runners.STYLE_CHECKER, + ), + JobNames.DOCKER_SERVER: CommonJobConfigs.DOCKER_SERVER.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] + ), + JobNames.DOCKER_KEEPER: CommonJobConfigs.DOCKER_SERVER.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] + ), + JobNames.DOCS_CHECK: JobConfig( + digest=DigestConfig( + include_paths=["**/*.md", "./docs", "tests/ci/docs_check.py"], + docker=["clickhouse/docs-builder"], + ), + run_command="docs_check.py", + runner_type=Runners.FUNC_TESTER, + ), + JobNames.FAST_TEST: JobConfig( + pr_only=True, + digest=DigestConfig( + include_paths=["./tests/queries/0_stateless/"], + exclude_files=[".md"], + docker=["clickhouse/fasttest"], + ), + timeout=2400, + runner_type=Runners.BUILDER, + ), + JobNames.STYLE_CHECK: JobConfig( + run_always=True, + runner_type=Runners.STYLE_CHECKER_ARM, + ), + JobNames.BUGFIX_VALIDATE: JobConfig( + run_by_label="pr-bugfix", + run_command="bugfix_validate_check.py", + timeout=900, + runner_type=Runners.STYLE_CHECKER, + ), + } + + @classmethod + def get_tag_config(cls, label_name: str) -> Optional[LabelConfig]: + for label, config in cls.TAG_CONFIGS.items(): if normalize_string(label_name) == normalize_string(label): return config return None - def get_job_ci_stage(self, job_name: str) -> str: + @classmethod + def get_job_ci_stage(cls, job_name: str) -> str: if job_name in [ JobNames.STYLE_CHECK, JobNames.FAST_TEST, + JobNames.JEPSEN_SERVER, JobNames.JEPSEN_KEEPER, JobNames.BUILD_CHECK, - JobNames.BUILD_CHECK_SPECIAL, ]: - # FIXME: we can't currently handle Jepsen in the Stage as it's job has concurrency directive - # BUILD_CHECK and BUILD_CHECK_SPECIAL runs not in stage because we need them even if Builds stage failed - return CIStages.NA + return WorkflowStages.NA + stage_type = None - if self.is_build_job(job_name): - stage_type = CIStages.BUILDS_1 - if job_name in CI_CONFIG.get_builds_for_report( - JobNames.BUILD_CHECK_SPECIAL - ): - # special builds go to Build_2 stage to not delay Builds_1/Test_1 - stage_type = CIStages.BUILDS_2 - elif self.is_docs_job(job_name): - stage_type = CIStages.TESTS_1 - elif self.is_test_job(job_name): - if job_name in CI_CONFIG.test_configs: - required_build = CI_CONFIG.test_configs[job_name].required_build - assert required_build - if required_build in CI_CONFIG.get_builds_for_report( - JobNames.BUILD_CHECK - ): - stage_type = CIStages.TESTS_1 - else: - stage_type = CIStages.TESTS_2 + if cls.is_build_job(job_name): + for _job, config in cls.JOB_CONFIGS.items(): + if config.required_builds and job_name in config.required_builds: + stage_type = WorkflowStages.BUILDS_1 + break else: - stage_type = CIStages.TESTS_1 - if job_name not in REQUIRED_CHECKS: - stage_type = CIStages.TESTS_3 + stage_type = WorkflowStages.BUILDS_2 + elif cls.is_docs_job(job_name): + stage_type = WorkflowStages.TESTS_1 + elif cls.is_test_job(job_name): + if job_name in CI.JOB_CONFIGS: + if job_name in REQUIRED_CHECKS: + stage_type = WorkflowStages.TESTS_1 + else: + stage_type = WorkflowStages.TESTS_3 assert stage_type, f"BUG [{job_name}]" return stage_type - def get_job_config(self, check_name: str) -> JobConfig: - res = None - for config in ( - self.build_config, - self.builds_report_config, - self.test_configs, - self.other_jobs_configs, - ): - if check_name in config: # type: ignore - res = config[check_name].job_config # type: ignore - break - return res # type: ignore + @classmethod + def get_job_config(cls, check_name: str) -> JobConfig: + return cls.JOB_CONFIGS[check_name] - def get_runner_type(self, check_name: str) -> str: - result = None - if self.is_build_job(check_name) or check_name == JobNames.FAST_TEST: - result = Runners.BUILDER - elif any( - words in check_name.lower() - for words in [ - "install packages", - "compatibility check", - "docker", - "build check", - "jepsen", - "style check", - ] - ): - result = Runners.STYLE_CHECKER - elif check_name == JobNames.DOCS_CHECK: - # docs job is demanding - result = Runners.FUNC_TESTER_ARM - elif any( - words in check_name.lower() - for words in [ - "stateless", - "stateful", - "clickbench", - "sqllogic test", - "libfuzzer", - "bugfix validation", - ] - ): - result = Runners.FUNC_TESTER - elif any( - words in check_name.lower() - for words in ["stress", "upgrade", "integration", "performance comparison"] - ): - result = Runners.STRESS_TESTER - elif any( - words in check_name.lower() - for words in ["ast fuzzer", "unit tests", "sqlancer", "sqltest"] - ): - result = Runners.FUZZER_UNIT_TESTER + @classmethod + def get_required_build_name(cls, check_name: str) -> str: + assert check_name in cls.JOB_CONFIGS + required_builds = cls.JOB_CONFIGS[check_name].required_builds + assert required_builds and len(required_builds) == 1 + return required_builds[0] - assert result, f"BUG, no runner for [{check_name}]" - - if ( - "aarch" in check_name.lower() or "arm64" in check_name.lower() - ) and "aarch" not in result: - if result == Runners.STRESS_TESTER: - # FIXME: no arm stress tester group atm - result = Runners.FUNC_TESTER_ARM - elif result == Runners.BUILDER: - # crosscompile - no arm required - pass - else: - # switch to aarch64 runner - result += "-aarch64" - - return result - - def get_job_parents(self, check_name: str) -> List[str]: - res = [] - check_name = normalize_string(check_name) - for config in ( - self.build_config, - self.test_configs, - self.other_jobs_configs, - ): - for job_name in config: # type: ignore - if check_name == normalize_string(job_name): - if isinstance(config[job_name], TestConfig): # type: ignore - if config[job_name].required_build: # type: ignore - res.append(config[job_name].required_build) # type: ignore - return res - - def get_digest_config(self, check_name: str) -> DigestConfig: - res = None - for config in ( - self.other_jobs_configs, - self.build_config, - self.builds_report_config, - self.test_configs, - ): - if check_name in config: # type: ignore - res = config[check_name].job_config.digest # type: ignore - assert ( - res - ), f"Invalid check_name or CI_CONFIG outdated, config not found for [{check_name}]" - return res # type: ignore + @classmethod + def get_job_parents(cls, check_name: str) -> List[str]: + return cls.JOB_CONFIGS[check_name].required_builds or [] + @classmethod def get_workflow_jobs_with_configs( - self, is_mq: bool, is_docs_only: bool, is_master: bool + cls, is_mq: bool, is_docs_only: bool, is_master: bool, is_pr: bool ) -> Dict[str, JobConfig]: """ get a list of all jobs for a workflow with configs """ jobs = [] if is_mq: - jobs = self._MQ_JOBS + jobs = MQ_JOBS elif is_docs_only: - jobs = self._DOCS_CHECK_JOBS + jobs = cls._DOCS_CHECK_JOBS else: - for config in ( - self.other_jobs_configs, - self.build_config, - self.builds_report_config, - self.test_configs, - ): - jobs += list(config) # type:ignore + # add all jobs + jobs = list(cls.JOB_CONFIGS) if is_master: - for job in self._MQ_JOBS: + for job in MQ_JOBS: jobs.remove(job) randomization_bucket_jobs = {} # type: Dict[str, Dict[str, JobConfig]] res = {} # type: Dict[str, JobConfig] for job in jobs: - job_config = self.get_job_config(job) + job_config = cls.JOB_CONFIGS[job] - if job_config.random_bucket: + if job_config.random_bucket and is_pr: if job_config.random_bucket not in randomization_bucket_jobs: randomization_bucket_jobs[job_config.random_bucket] = {} randomization_bucket_jobs[job_config.random_bucket][job] = job_config @@ -759,41 +619,20 @@ class CIConfig: return res - def get_builds_for_report( - self, report_name: str, release: bool = False, backport: bool = False - ) -> List[str]: - # hack to modify build list for release and bp wf - assert not (release and backport), "Invalid input" - if backport and report_name == JobNames.BUILD_CHECK: - return [ - Build.PACKAGE_RELEASE, - Build.PACKAGE_AARCH64, - Build.PACKAGE_ASAN, - Build.PACKAGE_TSAN, - Build.PACKAGE_DEBUG, - ] - if (release or backport) and report_name == JobNames.BUILD_CHECK_SPECIAL: - return [ - Build.BINARY_DARWIN, - Build.BINARY_DARWIN_AARCH64, - ] - - return self.builds_report_config[report_name].builds - @classmethod def is_build_job(cls, job: str) -> bool: - return job in Build + return job in cls.BuildNames @classmethod def is_test_job(cls, job: str) -> bool: - return not cls.is_build_job(job) and job != JobNames.STYLE_CHECK + return not cls.is_build_job(job) and job != cls.JobNames.STYLE_CHECK @classmethod def is_docs_job(cls, job: str) -> bool: return job == JobNames.DOCS_CHECK - @staticmethod - def is_required(check_name: str) -> bool: + @classmethod + def is_required(cls, check_name: str) -> bool: """Checks if a check_name is in REQUIRED_CHECKS, including batched jobs""" _BATCH_REGEXP = re.compile(r"\s+\[[0-9/]+\]$") if check_name in REQUIRED_CHECKS: @@ -802,810 +641,15 @@ class CIConfig: return check_name[: batch.start()] in REQUIRED_CHECKS return False - def validate(self) -> None: - errors = [] - for name, build_config in self.build_config.items(): - build_in_reports = False - for _, report_config in self.builds_report_config.items(): - if name in report_config.builds: - build_in_reports = True - break - # All build configs must belong to build_report_config - if not build_in_reports: - logging.error("Build name %s does not belong to build reports", name) - errors.append(f"Build name {name} does not belong to build reports") - # The name should be the same as build_config.name - if not build_config.name == name: - logging.error( - "Build name '%s' does not match the config 'name' value '%s'", - name, - build_config.name, - ) - errors.append( - f"Build name {name} does not match 'name' value '{build_config.name}'" - ) - # All build_report_config values should be in build_config.keys() - for build_report_name, build_report_config in self.builds_report_config.items(): - build_names = build_report_config.builds - missed_names = [ - name for name in build_names if name not in self.build_config.keys() - ] - if missed_names: - logging.error( - "The following names of the build report '%s' " - "are missed in build_config: %s", - build_report_name, - missed_names, - ) - errors.append( - f"The following names of the build report '{build_report_name}' " - f"are missed in build_config: {missed_names}", - ) - # And finally, all tests' requirements must be in the builds - for test_name, test_config in self.test_configs.items(): - if test_config.required_build not in self.build_config.keys(): - logging.error( - "The requirement '%s' for '%s' is not found in builds", - test_config, - test_name, - ) - errors.append( - f"The requirement '{test_config}' for " - f"'{test_name}' is not found in builds" - ) - if ( - test_config.required_build - and test_config.required_build - not in self.builds_report_config[JobNames.BUILD_CHECK].builds - ): - errors.append( - f"Test job' required build must be from [{JobNames.BUILD_CHECK}] list" - ) - - if errors: - raise KeyError("config contains errors", errors) + @classmethod + def get_build_config(cls, build_name: str) -> BuildConfig: + assert build_name in cls.JOB_CONFIGS, f"Invalid build name [{build_name}]" + res = cls.JOB_CONFIGS[build_name].build_config + assert res, f"not a build [{build_name}] or invalid JobConfig" + return res -# checks required by Mergeable Check -REQUIRED_CHECKS = [ - "PR Check", - StatusNames.SYNC, - JobNames.BUILD_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - JobNames.DOCS_CHECK, - JobNames.FAST_TEST, - JobNames.STATEFUL_TEST_RELEASE, - JobNames.STATELESS_TEST_RELEASE, - JobNames.STATELESS_TEST_ASAN, - JobNames.STATELESS_TEST_FLAKY_ASAN, - JobNames.STATEFUL_TEST_ASAN, - JobNames.STYLE_CHECK, - JobNames.UNIT_TEST_ASAN, - JobNames.UNIT_TEST_MSAN, - JobNames.UNIT_TEST, - JobNames.UNIT_TEST_TSAN, - JobNames.UNIT_TEST_UBSAN, - JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER, - JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE, -] - -CI_CONFIG = CIConfig( - label_configs={ - CILabels.DO_NOT_TEST_LABEL: LabelConfig(run_jobs=[JobNames.STYLE_CHECK]), - CILabels.CI_SET_ARM: LabelConfig( - run_jobs=[ - JobNames.STYLE_CHECK, - Build.PACKAGE_AARCH64, - JobNames.INTEGRATION_TEST_ARM, - ] - ), - CILabels.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), - CILabels.CI_SET_NORMAL_BUILDS: LabelConfig( - run_jobs=[ - JobNames.STYLE_CHECK, - JobNames.BUILD_CHECK, - Build.PACKAGE_RELEASE, - Build.PACKAGE_AARCH64, - Build.PACKAGE_ASAN, - Build.PACKAGE_UBSAN, - Build.PACKAGE_TSAN, - Build.PACKAGE_MSAN, - Build.PACKAGE_DEBUG, - Build.BINARY_RELEASE, - Build.PACKAGE_RELEASE_COVERAGE, - Build.FUZZERS, - ] - ), - CILabels.CI_SET_SPECIAL_BUILDS: LabelConfig( - run_jobs=[ - JobNames.STYLE_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - Build.BINARY_TIDY, - Build.BINARY_DARWIN, - Build.BINARY_AARCH64, - Build.BINARY_AARCH64_V80COMPAT, - Build.BINARY_FREEBSD, - Build.BINARY_DARWIN_AARCH64, - Build.BINARY_PPC64LE, - Build.BINARY_RISCV64, - Build.BINARY_S390X, - Build.BINARY_LOONGARCH64, - Build.BINARY_AMD64_COMPAT, - Build.BINARY_AMD64_MUSL, - ] - ), - CILabels.CI_SET_NON_REQUIRED: LabelConfig( - run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] - ), - CILabels.CI_SET_OLD_ANALYZER: LabelConfig( - run_jobs=[ - JobNames.STYLE_CHECK, - JobNames.FAST_TEST, - Build.PACKAGE_RELEASE, - Build.PACKAGE_ASAN, - JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE, - JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER, - ] - ), - CILabels.CI_SET_SYNC: LabelConfig( - run_jobs=[ - Build.PACKAGE_ASAN, - JobNames.STYLE_CHECK, - JobNames.BUILD_CHECK, - JobNames.UNIT_TEST_ASAN, - JobNames.STATEFUL_TEST_ASAN, - ] - ), - }, - build_config={ - Build.PACKAGE_RELEASE: BuildConfig( - name=Build.PACKAGE_RELEASE, - compiler="clang-18", - package_type="deb", - static_binary_name="amd64", - additional_pkgs=True, - ), - Build.PACKAGE_AARCH64: BuildConfig( - name=Build.PACKAGE_AARCH64, - compiler="clang-18-aarch64", - package_type="deb", - static_binary_name="aarch64", - additional_pkgs=True, - ), - Build.PACKAGE_ASAN: BuildConfig( - name=Build.PACKAGE_ASAN, - compiler="clang-18", - sanitizer="address", - package_type="deb", - ), - Build.PACKAGE_UBSAN: BuildConfig( - name=Build.PACKAGE_UBSAN, - compiler="clang-18", - sanitizer="undefined", - package_type="deb", - ), - Build.PACKAGE_TSAN: BuildConfig( - name=Build.PACKAGE_TSAN, - compiler="clang-18", - sanitizer="thread", - package_type="deb", - ), - Build.PACKAGE_MSAN: BuildConfig( - name=Build.PACKAGE_MSAN, - compiler="clang-18", - sanitizer="memory", - package_type="deb", - ), - Build.PACKAGE_DEBUG: BuildConfig( - name=Build.PACKAGE_DEBUG, - compiler="clang-18", - debug_build=True, - package_type="deb", - sparse_checkout=True, # Check that it works with at least one build, see also update-submodules.sh - ), - Build.PACKAGE_RELEASE_COVERAGE: BuildConfig( - name=Build.PACKAGE_RELEASE_COVERAGE, - compiler="clang-18", - coverage=True, - package_type="deb", - ), - Build.BINARY_RELEASE: BuildConfig( - name=Build.BINARY_RELEASE, - compiler="clang-18", - package_type="binary", - ), - Build.BINARY_TIDY: BuildConfig( - name=Build.BINARY_TIDY, - compiler="clang-18", - debug_build=True, - package_type="binary", - static_binary_name="debug-amd64", - tidy=True, - comment="clang-tidy is used for static analysis", - ), - Build.BINARY_DARWIN: BuildConfig( - name=Build.BINARY_DARWIN, - compiler="clang-18-darwin", - package_type="binary", - static_binary_name="macos", - ), - Build.BINARY_AARCH64: BuildConfig( - name=Build.BINARY_AARCH64, - compiler="clang-18-aarch64", - package_type="binary", - ), - Build.BINARY_AARCH64_V80COMPAT: BuildConfig( - name=Build.BINARY_AARCH64_V80COMPAT, - compiler="clang-18-aarch64-v80compat", - package_type="binary", - static_binary_name="aarch64v80compat", - comment="For ARMv8.1 and older", - ), - Build.BINARY_FREEBSD: BuildConfig( - name=Build.BINARY_FREEBSD, - compiler="clang-18-freebsd", - package_type="binary", - static_binary_name="freebsd", - ), - Build.BINARY_DARWIN_AARCH64: BuildConfig( - name=Build.BINARY_DARWIN_AARCH64, - compiler="clang-18-darwin-aarch64", - package_type="binary", - static_binary_name="macos-aarch64", - ), - Build.BINARY_PPC64LE: BuildConfig( - name=Build.BINARY_PPC64LE, - compiler="clang-18-ppc64le", - package_type="binary", - static_binary_name="powerpc64le", - ), - Build.BINARY_AMD64_COMPAT: BuildConfig( - name=Build.BINARY_AMD64_COMPAT, - compiler="clang-18-amd64-compat", - package_type="binary", - static_binary_name="amd64compat", - comment="SSE2-only build", - ), - Build.BINARY_AMD64_MUSL: BuildConfig( - name=Build.BINARY_AMD64_MUSL, - compiler="clang-18-amd64-musl", - package_type="binary", - static_binary_name="amd64musl", - comment="Build with Musl", - ), - Build.BINARY_RISCV64: BuildConfig( - name=Build.BINARY_RISCV64, - compiler="clang-18-riscv64", - package_type="binary", - static_binary_name="riscv64", - ), - Build.BINARY_S390X: BuildConfig( - name=Build.BINARY_S390X, - compiler="clang-18-s390x", - package_type="binary", - static_binary_name="s390x", - ), - Build.BINARY_LOONGARCH64: BuildConfig( - name=Build.BINARY_LOONGARCH64, - compiler="clang-18-loongarch64", - package_type="binary", - static_binary_name="loongarch64", - ), - Build.FUZZERS: BuildConfig( - name=Build.FUZZERS, - compiler="clang-18", - package_type="fuzzers", - job_config=fuzzer_build_job_config, - ), - }, - builds_report_config={ - JobNames.BUILD_CHECK: BuildReportConfig( - builds=[ - Build.PACKAGE_RELEASE, - Build.PACKAGE_AARCH64, - Build.PACKAGE_ASAN, - Build.PACKAGE_UBSAN, - Build.PACKAGE_TSAN, - Build.PACKAGE_MSAN, - Build.PACKAGE_DEBUG, - Build.BINARY_RELEASE, - Build.PACKAGE_RELEASE_COVERAGE, - Build.FUZZERS, - ] - ), - JobNames.BUILD_CHECK_SPECIAL: BuildReportConfig( - builds=[ - Build.BINARY_TIDY, - Build.BINARY_DARWIN, - Build.BINARY_AARCH64, - Build.BINARY_AARCH64_V80COMPAT, - Build.BINARY_FREEBSD, - Build.BINARY_DARWIN_AARCH64, - Build.BINARY_PPC64LE, - Build.BINARY_RISCV64, - Build.BINARY_S390X, - Build.BINARY_LOONGARCH64, - Build.BINARY_AMD64_COMPAT, - Build.BINARY_AMD64_MUSL, - ] - ), - }, - other_jobs_configs={ - JobNames.DOCKER_SERVER: TestConfig("", job_config=docker_server_job_config), - JobNames.DOCKER_KEEPER: TestConfig("", job_config=docker_server_job_config), - JobNames.DOCS_CHECK: TestConfig( - "", - job_config=JobConfig( - digest=DigestConfig( - include_paths=["**/*.md", "./docs", "tests/ci/docs_check.py"], - docker=["clickhouse/docs-builder"], - ), - run_command="docs_check.py", - ), - ), - JobNames.FAST_TEST: TestConfig( - "", - job_config=JobConfig( - pr_only=True, - digest=DigestConfig( - include_paths=["./tests/queries/0_stateless/"], - exclude_files=[".md"], - docker=["clickhouse/fasttest"], - ), - timeout=2400, - ), - ), - JobNames.STYLE_CHECK: TestConfig( - "", - job_config=JobConfig( - run_always=True, - ), - ), - JobNames.BUGFIX_VALIDATE: TestConfig( - "", - # we run this check by label - no digest required - job_config=JobConfig( - run_by_label="pr-bugfix", - run_command="bugfix_validate_check.py", - timeout=900, - ), - ), - }, - test_configs={ - JobNames.INSTALL_TEST_AMD: TestConfig( - Build.PACKAGE_RELEASE, job_config=install_test_params - ), - JobNames.INSTALL_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, job_config=install_test_params - ), - JobNames.STATEFUL_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_RELEASE: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_RELEASE_COVERAGE: TestConfig( - Build.PACKAGE_RELEASE_COVERAGE, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_AARCH64: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - # Stateful tests for parallel replicas - JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(random_bucket="parrepl_with_sanitizer", **stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(random_bucket="parrepl_with_sanitizer", **stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(random_bucket="parrepl_with_sanitizer", **stateful_test_common_params) # type: ignore - ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(random_bucket="parrepl_with_sanitizer", **stateful_test_common_params) # type: ignore - ), - # End stateful tests for parallel replicas - JobNames.STATELESS_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **stateless_test_common_params), # type: ignore - ), - JobNames.STATELESS_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, - job_config=JobConfig(num_batches=5, **stateless_test_common_params), # type: ignore - ), - JobNames.STATELESS_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, - job_config=JobConfig(num_batches=6, **stateless_test_common_params), # type: ignore - ), - JobNames.STATELESS_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, - job_config=JobConfig(num_batches=2, **stateless_test_common_params), # type: ignore - ), - JobNames.STATELESS_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, - job_config=JobConfig(num_batches=5, **stateless_test_common_params), # type: ignore - ), - JobNames.STATELESS_TEST_RELEASE: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**stateless_test_common_params) # type: ignore - ), - JobNames.STATELESS_TEST_RELEASE_COVERAGE: TestConfig( - Build.PACKAGE_RELEASE_COVERAGE, - job_config=JobConfig(num_batches=6, **stateless_test_common_params), # type: ignore - ), - JobNames.STATELESS_TEST_AARCH64: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(**stateless_test_common_params) # type: ignore - ), - JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: TestConfig( - Build.PACKAGE_RELEASE, - job_config=JobConfig(num_batches=4, **stateless_test_common_params), # type: ignore - ), - JobNames.STATELESS_TEST_S3_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, - job_config=JobConfig(num_batches=6, **stateless_test_common_params), # type: ignore - ), - JobNames.STATELESS_TEST_AZURE_ASAN: TestConfig( - Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **stateless_test_common_params, release_only=True), # type: ignore - ), - JobNames.STATELESS_TEST_S3_TSAN: TestConfig( - Build.PACKAGE_TSAN, - job_config=JobConfig(num_batches=5, **stateless_test_common_params), # type: ignore - ), - JobNames.STRESS_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**stress_test_common_params) # type: ignore - ), - JobNames.STRESS_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params) # type: ignore - ), - JobNames.STRESS_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(random_bucket="stress_with_sanitizer", **stress_test_common_params) # type: ignore - ), - JobNames.STRESS_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(random_bucket="stress_with_sanitizer", **stress_test_common_params) # type: ignore - ), - JobNames.STRESS_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(random_bucket="stress_with_sanitizer", **stress_test_common_params) # type: ignore - ), - JobNames.UPGRADE_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore - ), - JobNames.STRESS_TEST_AZURE_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params, release_only=True) # type: ignore - ), - JobNames.STRESS_TEST_AZURE_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(**stress_test_common_params, release_only=True) # type: ignore - ), - JobNames.UPGRADE_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore - ), - JobNames.UPGRADE_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore - ), - JobNames.UPGRADE_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(pr_only=True, **upgrade_test_common_params) # type: ignore - ), - JobNames.INTEGRATION_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **integration_test_common_params, release_only=True), # type: ignore - ), - JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER: TestConfig( - Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=6, **integration_test_common_params), # type: ignore - ), - JobNames.INTEGRATION_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, - job_config=JobConfig(num_batches=6, **integration_test_common_params), # type: ignore - ), - JobNames.INTEGRATION_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, - job_config=JobConfig(num_batches=6, **integration_test_common_params), # type: ignore - ), - JobNames.INTEGRATION_TEST: TestConfig( - Build.PACKAGE_RELEASE, - job_config=JobConfig(num_batches=4, **integration_test_common_params, release_only=True), # type: ignore - ), - JobNames.INTEGRATION_TEST_FLAKY: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, **integration_test_common_params) # type: ignore - ), - JobNames.COMPATIBILITY_TEST: TestConfig( - Build.PACKAGE_RELEASE, - job_config=JobConfig( - required_on_release_branch=True, **compatibility_test_common_params # type: ignore - ), - ), - JobNames.COMPATIBILITY_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, - job_config=JobConfig( - required_on_release_branch=True, **compatibility_test_common_params # type: ignore - ), - ), - JobNames.UNIT_TEST: TestConfig( - Build.BINARY_RELEASE, job_config=JobConfig(**unit_test_common_params) # type: ignore - ), - JobNames.UNIT_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(**unit_test_common_params) # type: ignore - ), - JobNames.UNIT_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(**unit_test_common_params) # type: ignore - ), - JobNames.UNIT_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**unit_test_common_params) # type: ignore - ), - JobNames.UNIT_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(**unit_test_common_params) # type: ignore - ), - JobNames.AST_FUZZER_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore - ), - JobNames.AST_FUZZER_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore - ), - JobNames.AST_FUZZER_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore - ), - JobNames.AST_FUZZER_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore - ), - JobNames.AST_FUZZER_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore - ), - JobNames.STATELESS_TEST_FLAKY_ASAN: TestConfig( - # replace to non-default - Build.PACKAGE_ASAN, - job_config=JobConfig(pr_only=True, **{**stateless_test_common_params, "timeout": 3600}), # type: ignore - ), - JobNames.JEPSEN_KEEPER: TestConfig( - Build.BINARY_RELEASE, - job_config=JobConfig( - run_by_label="jepsen-test", run_command="jepsen_check.py keeper" - ), - ), - JobNames.JEPSEN_SERVER: TestConfig( - Build.BINARY_RELEASE, - job_config=JobConfig( - run_by_label="jepsen-test", run_command="jepsen_check.py server" - ), - ), - JobNames.PERFORMANCE_TEST_AMD64: TestConfig( - Build.PACKAGE_RELEASE, - job_config=JobConfig(num_batches=4, **perf_test_common_params), # type: ignore - ), - JobNames.PERFORMANCE_TEST_ARM64: TestConfig( - Build.PACKAGE_AARCH64, - job_config=JobConfig(num_batches=4, run_by_label="pr-performance", **perf_test_common_params), # type: ignore - ), - JobNames.SQLANCER: TestConfig( - Build.PACKAGE_RELEASE, job_config=sqllancer_test_common_params - ), - JobNames.SQLANCER_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=sqllancer_test_common_params - ), - JobNames.SQL_LOGIC_TEST: TestConfig( - Build.PACKAGE_RELEASE, job_config=sqllogic_test_params - ), - JobNames.SQLTEST: TestConfig(Build.PACKAGE_RELEASE, job_config=sql_test_params), - JobNames.CLICKBENCH_TEST: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**clickbench_test_params) # type: ignore - ), - JobNames.CLICKBENCH_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(**clickbench_test_params) # type: ignore - ), - JobNames.LIBFUZZER_TEST: TestConfig( - Build.FUZZERS, - job_config=JobConfig( - run_by_label=CILabels.libFuzzer, - timeout=10800, - run_command='libfuzzer_test_check.py "$CHECK_NAME"', - ), - ), # type: ignore - }, -) -CI_CONFIG.validate() - - -@dataclass -class CheckDescription: - name: str - description: str # the check descriptions, will be put into the status table - match_func: Callable[[str], bool] # the function to check vs the commit status - - def __hash__(self) -> int: - return hash(self.name + self.description) - - -CHECK_DESCRIPTIONS = [ - CheckDescription( - "PR Check", - "Checks correctness of the PR's body", - lambda x: x == "PR Check", - ), - CheckDescription( - StatusNames.SYNC, - "If it fails, ask a maintainer for help", - lambda x: x == StatusNames.SYNC, - ), - CheckDescription( - "AST fuzzer", - "Runs randomly generated queries to catch program errors. " - "The build type is optionally given in parenthesis. " - "If it fails, ask a maintainer for help", - lambda x: x.startswith("AST fuzzer"), - ), - CheckDescription( - JobNames.BUGFIX_VALIDATE, - "Checks that either a new test (functional or integration) or there " - "some changed tests that fail with the binary built on master branch", - lambda x: x == JobNames.BUGFIX_VALIDATE, - ), - CheckDescription( - "CI running", - "A meta-check that indicates the running CI. Normally, it's in success or " - "pending state. The failed status indicates some problems with the PR", - lambda x: x == "CI running", - ), - CheckDescription( - "ClickHouse build check", - "Builds ClickHouse in various configurations for use in further steps. " - "You have to fix the builds that fail. Build logs often has enough " - "information to fix the error, but you might have to reproduce the failure " - "locally. The cmake options can be found in the build log, grepping for " - 'cmake. Use these options and follow the general build process', - lambda x: x.startswith("ClickHouse") and x.endswith("build check"), - ), - CheckDescription( - "Compatibility check", - "Checks that clickhouse binary runs on distributions with old libc " - "versions. If it fails, ask a maintainer for help", - lambda x: x.startswith("Compatibility check"), - ), - CheckDescription( - JobNames.DOCKER_SERVER, - "The check to build and optionally push the mentioned image to docker hub", - lambda x: x.startswith("Docker server"), - ), - CheckDescription( - JobNames.DOCKER_KEEPER, - "The check to build and optionally push the mentioned image to docker hub", - lambda x: x.startswith("Docker keeper"), - ), - CheckDescription( - JobNames.DOCS_CHECK, - "Builds and tests the documentation", - lambda x: x == JobNames.DOCS_CHECK, - ), - CheckDescription( - JobNames.FAST_TEST, - "Normally this is the first check that is ran for a PR. It builds ClickHouse " - 'and runs most of stateless functional tests, ' - "omitting some. If it fails, further checks are not started until it is fixed. " - "Look at the report to see which tests fail, then reproduce the failure " - 'locally as described here', - lambda x: x == JobNames.FAST_TEST, - ), - CheckDescription( - "Flaky tests", - "Checks if new added or modified tests are flaky by running them repeatedly, " - "in parallel, with more randomization. Functional tests are run 100 times " - "with address sanitizer, and additional randomization of thread scheduling. " - "Integration tests are run up to 10 times. If at least once a new test has " - "failed, or was too long, this check will be red. We don't allow flaky tests, " - 'read the doc', - lambda x: "tests flaky check" in x, - ), - CheckDescription( - "Install packages", - "Checks that the built packages are installable in a clear environment", - lambda x: x.startswith("Install packages ("), - ), - CheckDescription( - "Integration tests", - "The integration tests report. In parenthesis the package type is given, " - "and in square brackets are the optional part/total tests", - lambda x: x.startswith("Integration tests ("), - ), - CheckDescription( - StatusNames.MERGEABLE, - "Checks if all other necessary checks are successful", - lambda x: x == StatusNames.MERGEABLE, - ), - CheckDescription( - "Performance Comparison", - "Measure changes in query performance. The performance test report is " - 'described in detail here. ' - "In square brackets are the optional part/total tests", - lambda x: x.startswith("Performance Comparison"), - ), - CheckDescription( - "Push to Dockerhub", - "The check for building and pushing the CI related docker images to docker hub", - lambda x: x.startswith("Push") and "to Dockerhub" in x, - ), - CheckDescription( - "Sqllogic", - "Run clickhouse on the " - 'sqllogic ' - "test set against sqlite and checks that all statements are passed", - lambda x: x.startswith("Sqllogic test"), - ), - CheckDescription( - "SQLancer", - "Fuzzing tests that detect logical bugs with " - 'SQLancer tool', - lambda x: x.startswith("SQLancer"), - ), - CheckDescription( - "Stateful tests", - "Runs stateful functional tests for ClickHouse binaries built in various " - "configurations -- release, debug, with sanitizers, etc", - lambda x: x.startswith("Stateful tests ("), - ), - CheckDescription( - "Stateless tests", - "Runs stateless functional tests for ClickHouse binaries built in various " - "configurations -- release, debug, with sanitizers, etc", - lambda x: x.startswith("Stateless tests ("), - ), - CheckDescription( - "Stress test", - "Runs stateless functional tests concurrently from several clients to detect " - "concurrency-related errors", - lambda x: x.startswith("Stress test ("), - ), - CheckDescription( - JobNames.STYLE_CHECK, - "Runs a set of checks to keep the code style clean. If some of tests failed, " - "see the related log from the report", - lambda x: x == JobNames.STYLE_CHECK, - ), - CheckDescription( - "Unit tests", - "Runs the unit tests for different release types", - lambda x: x.startswith("Unit tests ("), - ), - CheckDescription( - "Upgrade check", - "Runs stress tests on server version from last release and then tries to " - "upgrade it to the version from the PR. It checks if the new server can " - "successfully startup without any errors, crashes or sanitizer asserts", - lambda x: x.startswith("Upgrade check ("), - ), - CheckDescription( - "ClickBench", - "Runs [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table", - lambda x: x.startswith("ClickBench"), - ), - CheckDescription( - "Fallback for unknown", - "There's no description for the check yet, please add it to " - "tests/ci/ci_config.py:CHECK_DESCRIPTIONS", - lambda x: True, - ), -] - - -def main() -> None: +if __name__ == "__main__": parser = ArgumentParser( formatter_class=ArgumentDefaultsHelpFormatter, description="The script provides build config for GITHUB_ENV or shell export", @@ -1617,10 +661,9 @@ def main() -> None: help="if set, the ENV parameters are provided for shell export", ) args = parser.parse_args() - build_config = CI_CONFIG.build_config.get(args.build_name) - if build_config: - print(build_config.export_env(args.export)) - - -if __name__ == "__main__": - main() + assert ( + args.build_name in CI.JOB_CONFIGS + ), f"Build name [{args.build_name}] is not valid" + build_config = CI.JOB_CONFIGS[args.build_name].build_config + assert build_config, "--export must not be used for non-build jobs" + print(build_config.export_env(args.export)) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py new file mode 100644 index 00000000000..e427d48505e --- /dev/null +++ b/tests/ci/ci_definitions.py @@ -0,0 +1,754 @@ +import copy +from dataclasses import dataclass, field +from pathlib import Path +from typing import Callable, List, Union, Iterable, Optional, Literal, Any + +from ci_utils import WithIter +from integration_test_images import IMAGES + + +class WorkflowStages(metaclass=WithIter): + """ + Stages of GitHUb actions workflow + """ + + # for jobs that do not belong to any stage, e.g. Build Report Check + NA = "UNKNOWN" + # normal builds (builds that required for further testing) + BUILDS_1 = "Builds_1" + # special builds + BUILDS_2 = "Builds_2" + # all tests required for merge + TESTS_1 = "Tests_1" + # not used atm + TESTS_2 = "Tests_2" + # all tests not required for merge + TESTS_3 = "Tests_3" + + +class Runners(metaclass=WithIter): + """ + GitHub runner's labels + """ + + BUILDER = "builder" + STYLE_CHECKER = "style-checker" + STYLE_CHECKER_ARM = "style-checker-aarch64" + FUNC_TESTER = "func-tester" + FUNC_TESTER_ARM = "func-tester-aarch64" + STRESS_TESTER = "stress-tester" + FUZZER_UNIT_TESTER = "fuzzer-unit-tester" + + +class Tags(metaclass=WithIter): + """ + CI Customization tags (set via PR body or some of them in GH labels, e.g. libFuzzer) + """ + + DO_NOT_TEST_LABEL = "do_not_test" + NO_MERGE_COMMIT = "no_merge_commit" + NO_CI_CACHE = "no_ci_cache" + # to upload all binaries from build jobs + UPLOAD_ALL_ARTIFACTS = "upload_all" + CI_SET_SYNC = "ci_set_sync" + CI_SET_ARM = "ci_set_arm" + CI_SET_REQUIRED = "ci_set_required" + CI_SET_BUILDS = "ci_set_builds" + CI_SET_NON_REQUIRED = "ci_set_non_required" + CI_SET_OLD_ANALYZER = "ci_set_old_analyzer" + + libFuzzer = "libFuzzer" + + +class BuildNames(metaclass=WithIter): + """ + Build' job names + """ + + PACKAGE_RELEASE = "package_release" + PACKAGE_AARCH64 = "package_aarch64" + PACKAGE_ASAN = "package_asan" + PACKAGE_UBSAN = "package_ubsan" + PACKAGE_TSAN = "package_tsan" + PACKAGE_MSAN = "package_msan" + PACKAGE_DEBUG = "package_debug" + PACKAGE_RELEASE_COVERAGE = "package_release_coverage" + BINARY_RELEASE = "binary_release" + BINARY_TIDY = "binary_tidy" + BINARY_DARWIN = "binary_darwin" + BINARY_AARCH64 = "binary_aarch64" + BINARY_AARCH64_V80COMPAT = "binary_aarch64_v80compat" + BINARY_FREEBSD = "binary_freebsd" + BINARY_DARWIN_AARCH64 = "binary_darwin_aarch64" + BINARY_PPC64LE = "binary_ppc64le" + BINARY_AMD64_COMPAT = "binary_amd64_compat" + BINARY_AMD64_MUSL = "binary_amd64_musl" + BINARY_RISCV64 = "binary_riscv64" + BINARY_S390X = "binary_s390x" + BINARY_LOONGARCH64 = "binary_loongarch64" + FUZZERS = "fuzzers" + + +class JobNames(metaclass=WithIter): + """ + All CI non-build jobs (Build jobs are concatenated to this list via python hack) + """ + + STYLE_CHECK = "Style check" + FAST_TEST = "Fast test" + DOCKER_SERVER = "Docker server image" + DOCKER_KEEPER = "Docker keeper image" + INSTALL_TEST_AMD = "Install packages (amd64)" + INSTALL_TEST_ARM = "Install packages (aarch64)" + + STATELESS_TEST_DEBUG = "Stateless tests (debug)" + STATELESS_TEST_RELEASE = "Stateless tests (release)" + STATELESS_TEST_RELEASE_COVERAGE = "Stateless tests (coverage)" + STATELESS_TEST_AARCH64 = "Stateless tests (aarch64)" + STATELESS_TEST_ASAN = "Stateless tests (asan)" + STATELESS_TEST_TSAN = "Stateless tests (tsan)" + STATELESS_TEST_MSAN = "Stateless tests (msan)" + STATELESS_TEST_UBSAN = "Stateless tests (ubsan)" + STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE = ( + "Stateless tests (release, old analyzer, s3, DatabaseReplicated)" + ) + STATELESS_TEST_S3_DEBUG = "Stateless tests (debug, s3 storage)" + STATELESS_TEST_S3_TSAN = "Stateless tests (tsan, s3 storage)" + STATELESS_TEST_AZURE_ASAN = "Stateless tests (azure, asan)" + STATELESS_TEST_FLAKY_ASAN = "Stateless tests flaky check (asan)" + + STATEFUL_TEST_DEBUG = "Stateful tests (debug)" + STATEFUL_TEST_RELEASE = "Stateful tests (release)" + STATEFUL_TEST_RELEASE_COVERAGE = "Stateful tests (coverage)" + STATEFUL_TEST_AARCH64 = "Stateful tests (aarch64)" + STATEFUL_TEST_ASAN = "Stateful tests (asan)" + STATEFUL_TEST_TSAN = "Stateful tests (tsan)" + STATEFUL_TEST_MSAN = "Stateful tests (msan)" + STATEFUL_TEST_UBSAN = "Stateful tests (ubsan)" + STATEFUL_TEST_PARALLEL_REPL_RELEASE = "Stateful tests (release, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_DEBUG = "Stateful tests (debug, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_ASAN = "Stateful tests (asan, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_MSAN = "Stateful tests (msan, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_UBSAN = "Stateful tests (ubsan, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_TSAN = "Stateful tests (tsan, ParallelReplicas)" + + STRESS_TEST_ASAN = "Stress test (asan)" + STRESS_TEST_TSAN = "Stress test (tsan)" + STRESS_TEST_UBSAN = "Stress test (ubsan)" + STRESS_TEST_MSAN = "Stress test (msan)" + STRESS_TEST_DEBUG = "Stress test (debug)" + STRESS_TEST_AZURE_TSAN = "Stress test (azure, tsan)" + STRESS_TEST_AZURE_MSAN = "Stress test (azure, msan)" + + INTEGRATION_TEST = "Integration tests (release)" + INTEGRATION_TEST_ASAN = "Integration tests (asan)" + INTEGRATION_TEST_ASAN_OLD_ANALYZER = "Integration tests (asan, old analyzer)" + INTEGRATION_TEST_TSAN = "Integration tests (tsan)" + INTEGRATION_TEST_ARM = "Integration tests (aarch64)" + INTEGRATION_TEST_FLAKY = "Integration tests flaky check (asan)" + + UPGRADE_TEST_DEBUG = "Upgrade check (debug)" + UPGRADE_TEST_ASAN = "Upgrade check (asan)" + UPGRADE_TEST_TSAN = "Upgrade check (tsan)" + UPGRADE_TEST_MSAN = "Upgrade check (msan)" + + UNIT_TEST = "Unit tests (release)" + UNIT_TEST_ASAN = "Unit tests (asan)" + UNIT_TEST_MSAN = "Unit tests (msan)" + UNIT_TEST_TSAN = "Unit tests (tsan)" + UNIT_TEST_UBSAN = "Unit tests (ubsan)" + + AST_FUZZER_TEST_DEBUG = "AST fuzzer (debug)" + AST_FUZZER_TEST_ASAN = "AST fuzzer (asan)" + AST_FUZZER_TEST_MSAN = "AST fuzzer (msan)" + AST_FUZZER_TEST_TSAN = "AST fuzzer (tsan)" + AST_FUZZER_TEST_UBSAN = "AST fuzzer (ubsan)" + + JEPSEN_KEEPER = "ClickHouse Keeper Jepsen" + JEPSEN_SERVER = "ClickHouse Server Jepsen" + + PERFORMANCE_TEST_AMD64 = "Performance Comparison" + PERFORMANCE_TEST_ARM64 = "Performance Comparison Aarch64" + + SQL_LOGIC_TEST = "Sqllogic test (release)" + + SQLANCER = "SQLancer (release)" + SQLANCER_DEBUG = "SQLancer (debug)" + SQLTEST = "SQLTest" + + COMPATIBILITY_TEST = "Compatibility check (amd64)" + COMPATIBILITY_TEST_ARM = "Compatibility check (aarch64)" + + CLICKBENCH_TEST = "ClickBench (amd64)" + CLICKBENCH_TEST_ARM = "ClickBench (aarch64)" + + LIBFUZZER_TEST = "libFuzzer tests" + + BUILD_CHECK = "ClickHouse build check" + # BUILD_CHECK_SPECIAL = "ClickHouse special build check" + + DOCS_CHECK = "Docs check" + BUGFIX_VALIDATE = "Bugfix validation" + + +# hack to concatenate Build and non-build jobs under JobNames class +for attr_name in dir(BuildNames): + if not attr_name.startswith("__") and not callable(getattr(BuildNames, attr_name)): + setattr(JobNames, attr_name, getattr(BuildNames, attr_name)) + + +class StatusNames(metaclass=WithIter): + """ + Class with statuses that aren't related to particular jobs + """ + + # overall CI report + CI = "CI running" + # mergeable status + MERGEABLE = "Mergeable Check" + # status of a sync pr + SYNC = "A Sync" + # PR formatting check status + PR_CHECK = "PR Check" + + +@dataclass +class DigestConfig: + # all files, dirs to include into digest, glob supported + include_paths: List[Union[str, Path]] = field(default_factory=list) + # file suffixes to exclude from digest + exclude_files: List[str] = field(default_factory=list) + # directories to exclude from digest + exclude_dirs: List[Union[str, Path]] = field(default_factory=list) + # docker names to include into digest + docker: List[str] = field(default_factory=list) + # git submodules digest + git_submodules: bool = False + + +@dataclass +class LabelConfig: + """ + configures different CI scenarios per CI Tag/GH label + """ + + run_jobs: Iterable[str] = frozenset() + + +@dataclass +class BuildConfig: + name: str + compiler: str + package_type: Literal["deb", "binary", "fuzzers"] + additional_pkgs: bool = False + debug_build: bool = False + coverage: bool = False + sanitizer: str = "" + tidy: bool = False + # sparse_checkout is needed only to test the option itself. + # No particular sense to use it in every build, since it slows down the job. + sparse_checkout: bool = False + comment: str = "" + static_binary_name: str = "" + + def export_env(self, export: bool = False) -> str: + def process(field_name: str, field: Union[bool, str]) -> str: + if isinstance(field, bool): + field = str(field).lower() + elif not isinstance(field, str): + field = "" + if export: + return f"export BUILD_{field_name.upper()}={repr(field)}" + return f"BUILD_{field_name.upper()}={field}" + + return "\n".join(process(k, v) for k, v in self.__dict__.items()) + + +@dataclass +class JobConfig: + """ + contains config parameters for job execution in CI workflow + """ + + # GH Runner type (tag from @Runners) + runner_type: str + # builds required for the job (applicable for test jobs) + required_builds: Optional[List[str]] = None + # build config for the build job (applicable for builds) + build_config: Optional[BuildConfig] = None + # configures digest calculation for the job + digest: DigestConfig = field(default_factory=DigestConfig) + # will be triggered for the job if omitted in CI workflow yml + run_command: str = "" + # job timeout, seconds + timeout: Optional[int] = None + # sets number of batches for a multi-batch job + num_batches: int = 1 + # label that enables job in CI, if set digest isn't used + run_by_label: str = "" + # to run always regardless of the job digest or/and label + run_always: bool = False + # if the job needs to be run on the release branch, including master (building packages, docker server). + # NOTE: Subsequent runs on the same branch with the similar digest are still considered skip-able. + required_on_release_branch: bool = False + # job is for pr workflow only + pr_only: bool = False + # job is for release/master branches only + release_only: bool = False + # to randomly pick and run one job among jobs in the same @random_bucket (PR branches only). + random_bucket: str = "" + # Do not set it. A list of batches to run. It will be set in runtime in accordance with ci cache and ci settings + batches: Optional[List[int]] = None + # Do not set it. A list of batches to await. It will be set in runtime in accordance with ci cache and ci settings + pending_batches: Optional[List[int]] = None + + def with_properties(self, **kwargs: Any) -> "JobConfig": + res = copy.deepcopy(self) + for k, v in kwargs.items(): + assert hasattr(self, k), f"Setting invalid attribute [{k}]" + setattr(res, k, v) + return res + + def get_required_build(self) -> str: + assert self.required_builds + return self.required_builds[0] + + +class CommonJobConfigs: + """ + Common job configs + """ + + BUILD_REPORT = JobConfig( + run_command="build_report_check.py", + digest=DigestConfig( + include_paths=[ + "./tests/ci/build_report_check.py", + "./tests/ci/upload_result_helper.py", + ], + ), + runner_type=Runners.STYLE_CHECKER_ARM, + ) + COMPATIBILITY_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/compatibility_check.py"], + docker=["clickhouse/test-old-ubuntu", "clickhouse/test-old-centos"], + ), + run_command="compatibility_check.py", + runner_type=Runners.STYLE_CHECKER_ARM, + ) + INSTALL_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/install_check.py"], + docker=["clickhouse/install-deb-test", "clickhouse/install-rpm-test"], + ), + run_command='install_check.py "$CHECK_NAME"', + runner_type=Runners.STYLE_CHECKER_ARM, + timeout=900, + ) + STATELESS_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/ci/functional_test_check.py", + "./tests/queries/0_stateless/", + "./tests/clickhouse-test", + "./tests/config", + "./tests/*.txt", + ], + exclude_files=[".md"], + docker=["clickhouse/stateless-test"], + ), + run_command='functional_test_check.py "$CHECK_NAME"', + runner_type=Runners.FUNC_TESTER, + timeout=10800, + ) + STATEFUL_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/ci/functional_test_check.py", + "./tests/queries/1_stateful/", + "./tests/clickhouse-test", + "./tests/config", + "./tests/*.txt", + ], + exclude_files=[".md"], + docker=["clickhouse/stateful-test"], + ), + run_command='functional_test_check.py "$CHECK_NAME"', + runner_type=Runners.FUNC_TESTER, + timeout=3600, + ) + STRESS_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/queries/0_stateless/", + "./tests/queries/1_stateful/", + "./tests/clickhouse-test", + "./tests/config", + "./tests/*.txt", + ], + exclude_files=[".md"], + docker=["clickhouse/stress-test"], + ), + run_command="stress_check.py", + runner_type=Runners.STRESS_TESTER, + timeout=9000, + ) + UPGRADE_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/upgrade_check.py"], + exclude_files=[".md"], + docker=["clickhouse/upgrade-check"], + ), + run_command="upgrade_check.py", + runner_type=Runners.STRESS_TESTER, + ) + INTEGRATION_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/ci/integration_test_check.py", + "./tests/ci/integration_tests_runner.py", + "./tests/integration/", + ], + exclude_files=[".md"], + docker=IMAGES.copy(), + ), + run_command='integration_test_check.py "$CHECK_NAME"', + runner_type=Runners.STRESS_TESTER, + ) + ASTFUZZER_TEST = JobConfig( + digest=DigestConfig(), + run_command="ast_fuzzer_check.py", + run_always=True, + runner_type=Runners.FUZZER_UNIT_TESTER, + ) + UNIT_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/unit_tests_check.py"], + exclude_files=[".md"], + docker=["clickhouse/unit-test"], + ), + run_command="unit_tests_check.py", + runner_type=Runners.FUZZER_UNIT_TESTER, + ) + PERF_TESTS = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/ci/performance_comparison_check.py", + "./tests/performance/", + ], + exclude_files=[".md"], + docker=["clickhouse/performance-comparison"], + ), + run_command="performance_comparison_check.py", + runner_type=Runners.STRESS_TESTER, + ) + SQLLANCER_TEST = JobConfig( + digest=DigestConfig(), + run_command="sqlancer_check.py", + release_only=True, + run_always=True, + runner_type=Runners.FUZZER_UNIT_TESTER, + ) + SQLLOGIC_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/sqllogic_test.py"], + exclude_files=[".md"], + docker=["clickhouse/sqllogic-test"], + ), + run_command="sqllogic_test.py", + timeout=10800, + release_only=True, + runner_type=Runners.STYLE_CHECKER_ARM, + ) + SQL_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/sqltest.py"], + exclude_files=[".md"], + docker=["clickhouse/sqltest"], + ), + run_command="sqltest.py", + timeout=10800, + release_only=True, + runner_type=Runners.FUZZER_UNIT_TESTER, + ) + BUGFIX_TEST = JobConfig( + digest=DigestConfig(), + run_command="bugfix_validate_check.py", + timeout=900, + runner_type=Runners.FUNC_TESTER, + ) + DOCKER_SERVER = JobConfig( + required_on_release_branch=True, + run_command='docker_server.py --check-name "$CHECK_NAME" --release-type head --allow-build-reuse', + digest=DigestConfig( + include_paths=[ + "tests/ci/docker_server.py", + "./docker/server", + ] + ), + runner_type=Runners.STYLE_CHECKER, + ) + CLICKBENCH_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "tests/ci/clickbench.py", + ], + docker=["clickhouse/clickbench"], + ), + run_command='clickbench.py "$CHECK_NAME"', + timeout=900, + runner_type=Runners.FUNC_TESTER, + ) + BUILD = JobConfig( + required_on_release_branch=True, + digest=DigestConfig( + include_paths=[ + "./src", + "./contrib/*-cmake", + "./contrib/consistent-hashing", + "./contrib/murmurhash", + "./contrib/libfarmhash", + "./contrib/pdqsort", + "./contrib/cityhash102", + "./contrib/sparse-checkout", + "./contrib/libmetrohash", + "./contrib/update-submodules.sh", + "./contrib/CMakeLists.txt", + "./CMakeLists.txt", + "./PreLoad.cmake", + "./cmake", + "./base", + "./programs", + "./packages", + "./docker/packager/packager", + "./rust", + "./tests/ci/version_helper.py", + # FIXME: This is a WA to rebuild the CH and recreate the Performance.tar.zst artifact + # when there are changes in performance test scripts. + # Due to the current design of the perf test we need to rebuild CH when the performance test changes, + # otherwise the changes will not be visible in the PerformanceTest job in CI + "./tests/performance", + ], + exclude_files=[".md"], + docker=["clickhouse/binary-builder"], + git_submodules=True, + ), + run_command="build_check.py $BUILD_NAME", + runner_type=Runners.BUILDER, + ) + + +REQUIRED_CHECKS = [ + StatusNames.PR_CHECK, + StatusNames.SYNC, + JobNames.BUILD_CHECK, + JobNames.DOCS_CHECK, + JobNames.FAST_TEST, + JobNames.STATEFUL_TEST_RELEASE, + JobNames.STATELESS_TEST_RELEASE, + JobNames.STATELESS_TEST_ASAN, + JobNames.STATELESS_TEST_FLAKY_ASAN, + JobNames.STATEFUL_TEST_ASAN, + JobNames.STYLE_CHECK, + JobNames.UNIT_TEST_ASAN, + JobNames.UNIT_TEST_MSAN, + JobNames.UNIT_TEST, + JobNames.UNIT_TEST_TSAN, + JobNames.UNIT_TEST_UBSAN, + JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER, + JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE, +] + +# Jobs that run in Merge Queue if it's enabled +MQ_JOBS = [ + JobNames.STYLE_CHECK, + JobNames.FAST_TEST, + BuildNames.BINARY_RELEASE, + JobNames.UNIT_TEST, +] + + +@dataclass +class CheckDescription: + name: str + description: str # the check descriptions, will be put into the status table + match_func: Callable[[str], bool] # the function to check vs the commit status + + def __hash__(self) -> int: + return hash(self.name + self.description) + + +CHECK_DESCRIPTIONS = [ + CheckDescription( + StatusNames.PR_CHECK, + "Checks correctness of the PR's body", + lambda x: x == "PR Check", + ), + CheckDescription( + StatusNames.SYNC, + "If it fails, ask a maintainer for help", + lambda x: x == StatusNames.SYNC, + ), + CheckDescription( + "AST fuzzer", + "Runs randomly generated queries to catch program errors. " + "The build type is optionally given in parenthesis. " + "If it fails, ask a maintainer for help", + lambda x: x.startswith("AST fuzzer"), + ), + CheckDescription( + JobNames.BUGFIX_VALIDATE, + "Checks that either a new test (functional or integration) or there " + "some changed tests that fail with the binary built on master branch", + lambda x: x == JobNames.BUGFIX_VALIDATE, + ), + CheckDescription( + StatusNames.CI, + "A meta-check that indicates the running CI. Normally, it's in success or " + "pending state. The failed status indicates some problems with the PR", + lambda x: x == "CI running", + ), + CheckDescription( + "ClickHouse build check", + "Builds ClickHouse in various configurations for use in further steps. " + "You have to fix the builds that fail. Build logs often has enough " + "information to fix the error, but you might have to reproduce the failure " + "locally. The cmake options can be found in the build log, grepping for " + 'cmake. Use these options and follow the general build process', + lambda x: x.startswith("ClickHouse") and x.endswith("build check"), + ), + CheckDescription( + "Compatibility check", + "Checks that clickhouse binary runs on distributions with old libc " + "versions. If it fails, ask a maintainer for help", + lambda x: x.startswith("Compatibility check"), + ), + CheckDescription( + JobNames.DOCKER_SERVER, + "The check to build and optionally push the mentioned image to docker hub", + lambda x: x.startswith("Docker server"), + ), + CheckDescription( + JobNames.DOCKER_KEEPER, + "The check to build and optionally push the mentioned image to docker hub", + lambda x: x.startswith("Docker keeper"), + ), + CheckDescription( + JobNames.DOCS_CHECK, + "Builds and tests the documentation", + lambda x: x == JobNames.DOCS_CHECK, + ), + CheckDescription( + JobNames.FAST_TEST, + "Normally this is the first check that is ran for a PR. It builds ClickHouse " + 'and runs most of stateless functional tests, ' + "omitting some. If it fails, further checks are not started until it is fixed. " + "Look at the report to see which tests fail, then reproduce the failure " + 'locally as described here', + lambda x: x == JobNames.FAST_TEST, + ), + CheckDescription( + "Flaky tests", + "Checks if new added or modified tests are flaky by running them repeatedly, " + "in parallel, with more randomization. Functional tests are run 100 times " + "with address sanitizer, and additional randomization of thread scheduling. " + "Integration tests are run up to 10 times. If at least once a new test has " + "failed, or was too long, this check will be red. We don't allow flaky tests, " + 'read the doc', + lambda x: "tests flaky check" in x, + ), + CheckDescription( + "Install packages", + "Checks that the built packages are installable in a clear environment", + lambda x: x.startswith("Install packages ("), + ), + CheckDescription( + "Integration tests", + "The integration tests report. In parenthesis the package type is given, " + "and in square brackets are the optional part/total tests", + lambda x: x.startswith("Integration tests ("), + ), + CheckDescription( + StatusNames.MERGEABLE, + "Checks if all other necessary checks are successful", + lambda x: x == StatusNames.MERGEABLE, + ), + CheckDescription( + "Performance Comparison", + "Measure changes in query performance. The performance test report is " + 'described in detail here. ' + "In square brackets are the optional part/total tests", + lambda x: x.startswith("Performance Comparison"), + ), + CheckDescription( + "Push to Dockerhub", + "The check for building and pushing the CI related docker images to docker hub", + lambda x: x.startswith("Push") and "to Dockerhub" in x, + ), + CheckDescription( + "Sqllogic", + "Run clickhouse on the " + 'sqllogic ' + "test set against sqlite and checks that all statements are passed", + lambda x: x.startswith("Sqllogic test"), + ), + CheckDescription( + "SQLancer", + "Fuzzing tests that detect logical bugs with " + 'SQLancer tool', + lambda x: x.startswith("SQLancer"), + ), + CheckDescription( + "Stateful tests", + "Runs stateful functional tests for ClickHouse binaries built in various " + "configurations -- release, debug, with sanitizers, etc", + lambda x: x.startswith("Stateful tests ("), + ), + CheckDescription( + "Stateless tests", + "Runs stateless functional tests for ClickHouse binaries built in various " + "configurations -- release, debug, with sanitizers, etc", + lambda x: x.startswith("Stateless tests ("), + ), + CheckDescription( + "Stress test", + "Runs stateless functional tests concurrently from several clients to detect " + "concurrency-related errors", + lambda x: x.startswith("Stress test ("), + ), + CheckDescription( + JobNames.STYLE_CHECK, + "Runs a set of checks to keep the code style clean. If some of tests failed, " + "see the related log from the report", + lambda x: x == JobNames.STYLE_CHECK, + ), + CheckDescription( + "Unit tests", + "Runs the unit tests for different release types", + lambda x: x.startswith("Unit tests ("), + ), + CheckDescription( + "Upgrade check", + "Runs stress tests on server version from last release and then tries to " + "upgrade it to the version from the PR. It checks if the new server can " + "successfully startup without any errors, crashes or sanitizer asserts", + lambda x: x.startswith("Upgrade check ("), + ), + CheckDescription( + "ClickBench", + "Runs [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table", + lambda x: x.startswith("ClickBench"), + ), + CheckDescription( + "Fallback for unknown", + "There's no description for the check yet, please add it to " + "tests/ci/ci_config.py:CHECK_DESCRIPTIONS", + lambda x: True, + ), +] diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index f25344c7701..a36fcf953ae 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -3,7 +3,7 @@ from dataclasses import dataclass, asdict from typing import Optional, List, Dict, Any, Iterable from ci_utils import normalize_string -from ci_config import CILabels, CI_CONFIG, JobConfig, JobNames +from ci_config import CI from git_helper import Runner as GitRunner, GIT_PREFIX from pr_info import PRInfo @@ -80,7 +80,7 @@ class CiSettings: if not res.ci_jobs: res.ci_jobs = [] res.ci_jobs.append(match.removeprefix("job_")) - elif match.startswith("ci_set_") and match in CILabels: + elif match.startswith("ci_set_") and match in CI.Tags: if not res.ci_sets: res.ci_sets = [] res.ci_sets.append(match) @@ -97,15 +97,15 @@ class CiSettings: res.exclude_keywords += [ normalize_string(keyword) for keyword in keywords ] - elif match == CILabels.NO_CI_CACHE: + elif match == CI.Tags.NO_CI_CACHE: res.no_ci_cache = True print("NOTE: CI Cache will be disabled") - elif match == CILabels.UPLOAD_ALL_ARTIFACTS: + elif match == CI.Tags.UPLOAD_ALL_ARTIFACTS: res.upload_all = True print("NOTE: All binary artifacts will be uploaded") - elif match == CILabels.DO_NOT_TEST_LABEL: + elif match == CI.Tags.DO_NOT_TEST_LABEL: res.do_not_test = True - elif match == CILabels.NO_MERGE_COMMIT: + elif match == CI.Tags.NO_MERGE_COMMIT: res.no_merge_commit = True print("NOTE: Merge Commit will be disabled") elif match.startswith("batch_"): @@ -131,18 +131,18 @@ class CiSettings: def _check_if_selected( self, job: str, - job_config: JobConfig, + job_config: CI.JobConfig, is_release: bool, is_pr: bool, is_mq: bool, labels: Iterable[str], ) -> bool: # type: ignore #too-many-return-statements if self.do_not_test: - label_config = CI_CONFIG.get_label_config(CILabels.DO_NOT_TEST_LABEL) - assert label_config, f"Unknown tag [{CILabels.DO_NOT_TEST_LABEL}]" + label_config = CI.get_tag_config(CI.Tags.DO_NOT_TEST_LABEL) + assert label_config, f"Unknown tag [{CI.Tags.DO_NOT_TEST_LABEL}]" if job in label_config.run_jobs: print( - f"Job [{job}] present in CI set [{CILabels.DO_NOT_TEST_LABEL}] - pass" + f"Job [{job}] present in CI set [{CI.Tags.DO_NOT_TEST_LABEL}] - pass" ) return True return False @@ -164,7 +164,7 @@ class CiSettings: to_deny = False if self.include_keywords: - if job == JobNames.STYLE_CHECK: + if job == CI.JobNames.STYLE_CHECK: # never exclude Style Check by include keywords return True for keyword in self.include_keywords: @@ -175,7 +175,7 @@ class CiSettings: if self.ci_sets: for tag in self.ci_sets: - label_config = CI_CONFIG.get_label_config(tag) + label_config = CI.get_tag_config(tag) assert label_config, f"Unknown tag [{tag}]" if job in label_config.run_jobs: print(f"Job [{job}] present in CI set [{tag}] - pass") @@ -197,12 +197,12 @@ class CiSettings: def apply( self, - job_configs: Dict[str, JobConfig], + job_configs: Dict[str, CI.JobConfig], is_release: bool, is_pr: bool, is_mq: bool, labels: Iterable[str], - ) -> Dict[str, JobConfig]: + ) -> Dict[str, CI.JobConfig]: """ Apply CI settings from pr body """ @@ -220,7 +220,7 @@ class CiSettings: add_parents = [] for job in list(res): - parent_jobs = CI_CONFIG.get_job_parents(job) + parent_jobs = CI.get_job_parents(job) for parent_job in parent_jobs: if parent_job not in res: add_parents.append(parent_job) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index a0d6495452f..96cf700ed77 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -17,7 +17,7 @@ from github.GithubObject import NotSet from github.IssueComment import IssueComment from github.Repository import Repository -from ci_config import CHECK_DESCRIPTIONS, CheckDescription, StatusNames, CIConfig +from ci_config import CI from env_helper import GITHUB_REPOSITORY, GITHUB_UPSTREAM_REPOSITORY, TEMP_PATH from lambda_shared_package.lambda_shared.pr import Labels from pr_info import PRInfo @@ -160,7 +160,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: if not statuses: return - if not [status for status in statuses if status.context == StatusNames.CI]: + if not [status for status in statuses if status.context == CI.StatusNames.CI]: # This is the case, when some statuses already exist for the check, # but not the StatusNames.CI. We should create it as pending. # W/o pr_info to avoid recursion, and yes, one extra create_ci_report @@ -169,7 +169,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: PENDING, create_ci_report(pr_info, statuses), "The report for running CI", - StatusNames.CI, + CI.StatusNames.CI, ) # We update the report in generate_status_comment function, so do it each @@ -212,20 +212,20 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: f"\n" ) # group checks by the name to get the worst one per each - grouped_statuses = {} # type: Dict[CheckDescription, CommitStatuses] + grouped_statuses = {} # type: Dict[CI.CheckDescription, CommitStatuses] for status in statuses: cd = None - for c in CHECK_DESCRIPTIONS: + for c in CI.CHECK_DESCRIPTIONS: if c.match_func(status.context): cd = c break - if cd is None or cd == CHECK_DESCRIPTIONS[-1]: + if cd is None or cd == CI.CHECK_DESCRIPTIONS[-1]: # This is the case for either non-found description or a fallback - cd = CheckDescription( + cd = CI.CheckDescription( status.context, - CHECK_DESCRIPTIONS[-1].description, - CHECK_DESCRIPTIONS[-1].match_func, + CI.CHECK_DESCRIPTIONS[-1].description, + CI.CHECK_DESCRIPTIONS[-1].match_func, ) if cd in grouped_statuses: @@ -301,7 +301,7 @@ def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str: ) ) return upload_results( - S3Helper(), pr_info.number, pr_info.sha, test_results, [], StatusNames.CI + S3Helper(), pr_info.number, pr_info.sha, test_results, [], CI.StatusNames.CI ) @@ -435,7 +435,7 @@ def set_mergeable_check( state, report_url, format_description(description), - StatusNames.MERGEABLE, + CI.StatusNames.MERGEABLE, ) @@ -443,7 +443,7 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> "check if the check_name in REQUIRED_CHECKS and then trigger update" not_run = ( pr_info.labels.intersection({Labels.SKIP_MERGEABLE_CHECK, Labels.RELEASE}) - or not CIConfig.is_required(check_name) + or not CI.is_required(check_name) or pr_info.release_pr or pr_info.number == 0 ) @@ -465,13 +465,11 @@ def trigger_mergeable_check( workflow_failed: bool = False, ) -> StatusType: """calculate and update StatusNames.MERGEABLE""" - required_checks = [ - status for status in statuses if CIConfig.is_required(status.context) - ] + required_checks = [status for status in statuses if CI.is_required(status.context)] mergeable_status = None for status in statuses: - if status.context == StatusNames.MERGEABLE: + if status.context == CI.StatusNames.MERGEABLE: mergeable_status = status break @@ -548,7 +546,7 @@ def update_upstream_sync_status( "Using commit %s to post the %s status `%s`: [%s]", last_synced_upstream_commit.sha, sync_status, - StatusNames.SYNC, + CI.StatusNames.SYNC, "", ) post_commit_status( @@ -556,7 +554,7 @@ def update_upstream_sync_status( sync_status, "", "", - StatusNames.SYNC, + CI.StatusNames.SYNC, ) trigger_mergeable_check( last_synced_upstream_commit, diff --git a/tests/ci/digest_helper.py b/tests/ci/digest_helper.py index 8d6ec127f6e..4dcfb03c04f 100644 --- a/tests/ci/digest_helper.py +++ b/tests/ci/digest_helper.py @@ -9,10 +9,10 @@ from typing import TYPE_CHECKING, Dict, Iterable, List, Optional, Union from sys import modules from docker_images_helper import get_images_info -from ci_config import DigestConfig from git_helper import Runner from env_helper import ROOT_DIR from ci_utils import cd +from ci_config import CI DOCKER_DIGEST_LEN = 12 JOB_DIGEST_LEN = 10 @@ -139,20 +139,21 @@ class DockerDigester: class JobDigester: - def __init__(self): + def __init__(self, dry_run: bool = False): self.dd = DockerDigester() self.cache: Dict[str, str] = {} + self.dry_run = dry_run @staticmethod - def _get_config_hash(digest_config: DigestConfig) -> str: + def _get_config_hash(digest_config: CI.DigestConfig) -> str: data_dict = asdict(digest_config) hash_obj = md5() hash_obj.update(str(data_dict).encode()) hash_string = hash_obj.hexdigest() return hash_string - def get_job_digest(self, digest_config: DigestConfig) -> str: - if not digest_config.include_paths: + def get_job_digest(self, digest_config: CI.DigestConfig) -> str: + if not digest_config.include_paths or self.dry_run: # job is not for digest return "f" * JOB_DIGEST_LEN diff --git a/tests/ci/download_binary.py b/tests/ci/download_binary.py index 79db1e57d62..b0b5659ca83 100755 --- a/tests/ci/download_binary.py +++ b/tests/ci/download_binary.py @@ -8,7 +8,7 @@ import logging from pathlib import Path from build_download_helper import download_build_with_progress -from ci_config import CI_CONFIG +from ci_config import CI from env_helper import RUNNER_TEMP, S3_ARTIFACT_DOWNLOAD_TEMPLATE from git_helper import Git, commit from version_helper import get_version_from_repo, version_arg @@ -59,7 +59,8 @@ def main(): temp_path.mkdir(parents=True, exist_ok=True) for build in args.build_names: # check if it's in CI_CONFIG - config = CI_CONFIG.build_config[build] + config = CI.JOB_CONFIGS[build].build_config + assert config if args.rename and config.static_binary_name: path = temp_path / f"clickhouse-{config.static_binary_name}" else: diff --git a/tests/ci/env_helper.py b/tests/ci/env_helper.py index 36732bd7c9f..5217e4035da 100644 --- a/tests/ci/env_helper.py +++ b/tests/ci/env_helper.py @@ -9,8 +9,9 @@ from build_download_helper import APIException, get_gh_api module_dir = p.abspath(p.dirname(__file__)) git_root = p.abspath(p.join(module_dir, "..", "..")) + ROOT_DIR = git_root -CI = bool(os.getenv("CI")) +IS_CI = bool(os.getenv("CI")) TEMP_PATH = os.getenv("TEMP_PATH", p.abspath(p.join(module_dir, "./tmp"))) REPORT_PATH = f"{TEMP_PATH}/reports" # FIXME: latest should not be used in CI, set temporary for transition to "docker with digest as a tag" diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 904b565ad86..bd83f875790 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -4,7 +4,7 @@ import logging from github import Github -from ci_config import StatusNames +from ci_config import CI from commit_status_helper import ( get_commit, get_commit_filtered_statuses, @@ -71,7 +71,7 @@ def main(): can_set_green_mergeable_status=True, ) - ci_running_statuses = [s for s in statuses if s.context == StatusNames.CI] + ci_running_statuses = [s for s in statuses if s.context == CI.StatusNames.CI] if not ci_running_statuses: return # Take the latest status @@ -81,7 +81,11 @@ def main(): has_pending = False error_cnt = 0 for status in statuses: - if status.context in (StatusNames.MERGEABLE, StatusNames.CI, StatusNames.SYNC): + if status.context in ( + CI.StatusNames.MERGEABLE, + CI.StatusNames.CI, + CI.StatusNames.SYNC, + ): # do not account these statuses continue if status.state == PENDING: @@ -108,7 +112,7 @@ def main(): ci_state, ci_status.target_url, description, - StatusNames.CI, + CI.StatusNames.CI, pr_info, dump_to_file=True, ) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 4abaeac30b7..87f721cfde7 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -18,7 +18,7 @@ from collections import defaultdict from itertools import chain from typing import Any, Dict -from env_helper import CI +from env_helper import IS_CI from integration_test_images import IMAGES MAX_RETRY = 1 @@ -1004,7 +1004,7 @@ def run(): logging.info("Running tests") - if CI: + if IS_CI: # Avoid overlaps with previous runs logging.info("Clearing dmesg before run") subprocess.check_call("sudo -E dmesg --clear", shell=True) @@ -1012,7 +1012,7 @@ def run(): state, description, test_results, _ = runner.run_impl(repo_path, build_path) logging.info("Tests finished") - if CI: + if IS_CI: # Dump dmesg (to capture possible OOMs) logging.info("Dumping dmesg") subprocess.check_call("sudo -E dmesg -T", shell=True) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 1e61fd9fab7..f91a3f080c0 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -13,7 +13,6 @@ import requests from build_download_helper import ( download_build_with_progress, - get_build_name_for_check, read_build_urls, ) from compress_files import compress_fast @@ -25,6 +24,7 @@ from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults from ssh import SSHKey from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI JEPSEN_GROUP_NAME = "jepsen_group" @@ -224,7 +224,7 @@ def main(): head = requests.head(build_url, timeout=60) assert head.status_code == 200, f"Clickhouse binary not found: {build_url}" else: - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, REPORT_PATH) build_url = None for url in urls: diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 0c779b515bd..b3488ac0af2 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -12,7 +12,7 @@ from pathlib import Path from github import Github from build_download_helper import download_builds_filter -from ci_config import CI_CONFIG +from ci_config import CI from clickhouse_helper import get_instance_id, get_instance_type from commit_status_helper import get_commit from docker_images_helper import get_docker_image, pull_image @@ -83,7 +83,7 @@ def main(): assert ( check_name ), "Check name must be provided as an input arg or in CHECK_NAME env" - required_build = CI_CONFIG.test_configs[check_name].required_build + required_build = CI.JOB_CONFIGS[check_name].get_required_build() with open(GITHUB_EVENT_PATH, "r", encoding="utf-8") as event_file: event = json.load(event_file) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index dda5b30f1e3..fb25a29cc57 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -316,7 +316,9 @@ class PRInfo: @property def is_master(self) -> bool: - return self.number == 0 and self.head_ref == "master" + return ( + self.number == 0 and self.head_ref == "master" and not self.is_merge_queue + ) @property def is_release(self) -> bool: @@ -324,7 +326,10 @@ class PRInfo: @property def is_pr(self): - return self.event_type == EventType.PULL_REQUEST + if self.event_type == EventType.PULL_REQUEST: + assert self.number + return True + return False @property def is_scheduled(self) -> bool: @@ -353,9 +358,6 @@ class PRInfo: if self.changed_files_requested: return - if not getattr(self, "diff_urls", False): - raise TypeError("The event does not have diff URLs") - for diff_url in self.diff_urls: response = get_gh_api( diff_url, diff --git a/tests/ci/report.py b/tests/ci/report.py index ee58efdba52..bdaa2e15130 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -21,7 +21,7 @@ from typing import ( ) from build_download_helper import get_gh_api -from ci_config import CI_CONFIG, BuildConfig +from ci_config import CI from ci_utils import normalize_string from env_helper import REPORT_PATH, TEMP_PATH @@ -412,6 +412,7 @@ class BuildResult: ref_report = None master_report = None any_report = None + Path(REPORT_PATH).mkdir(parents=True, exist_ok=True) for file in Path(REPORT_PATH).iterdir(): if f"{build_name}.json" in file.name: any_report = file @@ -448,8 +449,10 @@ class BuildResult: return json.dumps(asdict(self), indent=2) @property - def build_config(self) -> Optional[BuildConfig]: - return CI_CONFIG.build_config.get(self.build_name, None) + def build_config(self) -> Optional[CI.BuildConfig]: + if self.build_name not in CI.JOB_CONFIGS: + return None + return CI.JOB_CONFIGS[self.build_name].build_config @property def comment(self) -> str: diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 131cbeef786..00942352dde 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -5,7 +5,6 @@ from typing import Tuple from github import Github -from ci_config import StatusNames from commit_status_helper import ( create_ci_report, format_description, @@ -24,6 +23,7 @@ from lambda_shared_package.lambda_shared.pr import ( ) from pr_info import PRInfo from report import FAILURE, PENDING, SUCCESS, StatusType +from ci_config import CI TRUSTED_ORG_IDS = { 54801242, # clickhouse @@ -208,7 +208,7 @@ def main(): PENDING, ci_report_url, description, - StatusNames.CI, + CI.StatusNames.CI, pr_info, ) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index bff53f00ad3..da0ccb2b74d 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -11,7 +11,7 @@ import boto3 # type: ignore import botocore # type: ignore from compress_files import compress_file_fast from env_helper import ( - CI, + IS_CI, RUNNER_TEMP, S3_BUILDS_BUCKET, S3_DOWNLOAD, @@ -111,13 +111,13 @@ class S3Helper: self.client.delete_object(Bucket=bucket_name, Key=s3_path) def upload_test_report_to_s3(self, file_path: Path, s3_path: str) -> str: - if CI: + if IS_CI: return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path) return S3Helper.copy_file_to_local(S3_TEST_REPORTS_BUCKET, file_path, s3_path) def upload_build_file_to_s3(self, file_path: Path, s3_path: str) -> str: - if CI: + if IS_CI: return self._upload_file_to_s3(S3_BUILDS_BUCKET, file_path, s3_path) return S3Helper.copy_file_to_local(S3_BUILDS_BUCKET, file_path, s3_path) @@ -255,7 +255,7 @@ class S3Helper: if full_fs_path.is_symlink(): if upload_symlinks: - if CI: + if IS_CI: return self._upload_file_to_s3( bucket_name, full_fs_path, @@ -266,7 +266,7 @@ class S3Helper: ) return [] - if CI: + if IS_CI: return self._upload_file_to_s3( bucket_name, full_fs_path, full_s3_path + "/" + file_path.name ) @@ -331,7 +331,7 @@ class S3Helper: return result def url_if_exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> str: - if not CI: + if not IS_CI: local_path = self.local_path(bucket, key) if local_path.exists(): return local_path.as_uri() @@ -345,7 +345,7 @@ class S3Helper: @staticmethod def get_url(bucket: str, key: str) -> str: - if CI: + if IS_CI: return S3Helper.s3_url(bucket, key) return S3Helper.local_path(bucket, key).as_uri() diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index 9d33c480598..a68db8b9791 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -6,12 +6,13 @@ import subprocess import sys from pathlib import Path -from build_download_helper import get_build_name_for_check, read_build_urls +from build_download_helper import read_build_urls from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPORT_PATH, TEMP_PATH from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI IMAGE_NAME = "clickhouse/sqlancer-test" @@ -43,7 +44,7 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) if not urls: raise ValueError("No build URLs found") diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index c8c2adbbd56..8e6ca6ff87f 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -6,12 +6,13 @@ import subprocess import sys from pathlib import Path -from build_download_helper import get_build_name_for_check, read_build_urls +from build_download_helper import read_build_urls from docker_images_helper import get_docker_image, pull_image from env_helper import REPORT_PATH, TEMP_PATH from pr_info import PRInfo from report import SUCCESS, JobReport, TestResult from stopwatch import Stopwatch +from ci_config import CI IMAGE_NAME = "clickhouse/sqltest" @@ -49,7 +50,7 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) print(build_name) urls = read_build_urls(build_name, reports_path) if not urls: diff --git a/tests/ci/ssh.py b/tests/ci/ssh.py index 321826fcf44..89d90d724d2 100644 --- a/tests/ci/ssh.py +++ b/tests/ci/ssh.py @@ -37,9 +37,9 @@ class SSHAgent: ssh_options = ( "," + os.environ["SSH_OPTIONS"] if os.environ.get("SSH_OPTIONS") else "" ) - os.environ[ - "SSH_OPTIONS" - ] = f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" + os.environ["SSH_OPTIONS"] = ( + f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" + ) def add(self, key): key_pub = self._key_pub(key) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9906d87a8c0..35e48246be9 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -13,7 +13,7 @@ from typing import List, Tuple, Union import magic from docker_images_helper import get_docker_image, pull_image -from env_helper import CI, REPO_COPY, TEMP_PATH +from env_helper import IS_CI, REPO_COPY, TEMP_PATH from git_helper import GIT_PREFIX, git_runner from pr_info import PRInfo from report import ERROR, FAILURE, SUCCESS, JobReport, TestResults, read_test_results @@ -152,7 +152,7 @@ def main(): run_cpp_check = True run_shell_check = True run_python_check = True - if CI and pr_info.number > 0: + if IS_CI and pr_info.number > 0: pr_info.fetch_changed_files() run_cpp_check = any( not (is_python(file) or is_shell(file)) for file in pr_info.changed_files diff --git a/tests/ci/sync_pr.py b/tests/ci/sync_pr.py index f1073603e8d..8251ccbaf38 100644 --- a/tests/ci/sync_pr.py +++ b/tests/ci/sync_pr.py @@ -5,12 +5,12 @@ import argparse import sys -from ci_config import StatusNames from commit_status_helper import get_commit, post_commit_status from get_robot_token import get_best_robot_token from github_helper import GitHub from pr_info import PRInfo from report import SUCCESS +from ci_config import CI def parse_args() -> argparse.Namespace: @@ -75,7 +75,7 @@ def set_sync_status(gh, pr_info, sync_pr): if sync_pr.mergeable_state == "clean": print(f"Sync PR [{sync_pr.number}] is clean") post_commit_status( - get_commit(gh, pr_info.sha), SUCCESS, "", "", StatusNames.SYNC + get_commit(gh, pr_info.sha), SUCCESS, "", "", CI.StatusNames.SYNC ) else: print( diff --git a/tests/ci/test_ci_cache.py b/tests/ci/test_ci_cache.py index b1be0709803..81d649b246b 100644 --- a/tests/ci/test_ci_cache.py +++ b/tests/ci/test_ci_cache.py @@ -5,12 +5,12 @@ from pathlib import Path import shutil from typing import Dict, Set import unittest -from ci_config import Build, JobNames from s3_helper import S3Helper from ci_cache import CiCache from digest_helper import JOB_DIGEST_LEN from commit_status_helper import CommitStatusData from env_helper import S3_BUILDS_BUCKET, TEMP_PATH +from ci_config import CI def _create_mock_digest_1(string): @@ -21,8 +21,8 @@ def _create_mock_digest_2(string): return md5((string + "+nonce").encode("utf-8")).hexdigest()[:JOB_DIGEST_LEN] -DIGESTS = {job: _create_mock_digest_1(job) for job in JobNames} -DIGESTS2 = {job: _create_mock_digest_2(job) for job in JobNames} +DIGESTS = {job: _create_mock_digest_1(job) for job in CI.JobNames} +DIGESTS2 = {job: _create_mock_digest_2(job) for job in CI.JobNames} # pylint:disable=protected-access @@ -84,8 +84,10 @@ class TestCiCache(unittest.TestCase): NUM_BATCHES = 10 DOCS_JOBS_NUM = 1 - assert len(set(job for job in JobNames)) == len(list(job for job in JobNames)) - NONDOCS_JOBS_NUM = len(set(job for job in JobNames)) - DOCS_JOBS_NUM + assert len(set(job for job in CI.JobNames)) == len( + list(job for job in CI.JobNames) + ) + NONDOCS_JOBS_NUM = len(set(job for job in CI.JobNames)) - DOCS_JOBS_NUM PR_NUM = 123456 status = CommitStatusData( @@ -97,13 +99,13 @@ class TestCiCache(unittest.TestCase): ) ### add some pending statuses for two batches, non-release branch - for job in JobNames: + for job in CI.JobNames: ci_cache.push_pending(job, [0, 1, 2], NUM_BATCHES, release_branch=False) ci_cache_2.push_pending(job, [0, 1, 2], NUM_BATCHES, release_branch=False) ### add success status for 0 batch, non-release branch batch = 0 - for job in JobNames: + for job in CI.JobNames: ci_cache.push_successful( job, batch, NUM_BATCHES, status, release_branch=False ) @@ -113,21 +115,17 @@ class TestCiCache(unittest.TestCase): ### add failed status for 2 batch, non-release branch batch = 2 - for job in JobNames: + for job in CI.JobNames: ci_cache.push_failed(job, batch, NUM_BATCHES, status, release_branch=False) ci_cache_2.push_failed( job, batch, NUM_BATCHES, status, release_branch=False ) ### check all expected directories were created on s3 mock - expected_build_path_1 = f"{CiCache.JobType.SRCS.value}-{_create_mock_digest_1(Build.PACKAGE_RELEASE)}" - expected_docs_path_1 = ( - f"{CiCache.JobType.DOCS.value}-{_create_mock_digest_1(JobNames.DOCS_CHECK)}" - ) - expected_build_path_2 = f"{CiCache.JobType.SRCS.value}-{_create_mock_digest_2(Build.PACKAGE_RELEASE)}" - expected_docs_path_2 = ( - f"{CiCache.JobType.DOCS.value}-{_create_mock_digest_2(JobNames.DOCS_CHECK)}" - ) + expected_build_path_1 = f"{CiCache.JobType.SRCS.value}-{_create_mock_digest_1(CI.BuildNames.PACKAGE_RELEASE)}" + expected_docs_path_1 = f"{CiCache.JobType.DOCS.value}-{_create_mock_digest_1(CI.JobNames.DOCS_CHECK)}" + expected_build_path_2 = f"{CiCache.JobType.SRCS.value}-{_create_mock_digest_2(CI.BuildNames.PACKAGE_RELEASE)}" + expected_docs_path_2 = f"{CiCache.JobType.DOCS.value}-{_create_mock_digest_2(CI.JobNames.DOCS_CHECK)}" self.assertCountEqual( list(s3_mock.files_on_s3_paths.keys()), [ @@ -174,7 +172,7 @@ class TestCiCache(unittest.TestCase): ) ### check statuses for all jobs in cache - for job in JobNames: + for job in CI.JobNames: self.assertEqual( ci_cache.is_successful(job, 0, NUM_BATCHES, release_branch=False), True ) @@ -212,7 +210,7 @@ class TestCiCache(unittest.TestCase): assert status2 is None ### add some more pending statuses for two batches and for a release branch - for job in JobNames: + for job in CI.JobNames: ci_cache.push_pending( job, batches=[0, 1], num_batches=NUM_BATCHES, release_branch=True ) @@ -226,7 +224,7 @@ class TestCiCache(unittest.TestCase): sha="deadbeaf2", pr_num=PR_NUM, ) - for job in JobNames: + for job in CI.JobNames: ci_cache.push_successful(job, 0, NUM_BATCHES, status, release_branch=True) ### check number of cache files is as expected @@ -249,7 +247,7 @@ class TestCiCache(unittest.TestCase): ) ### check statuses - for job in JobNames: + for job in CI.JobNames: self.assertEqual(ci_cache.is_successful(job, 0, NUM_BATCHES, False), True) self.assertEqual(ci_cache.is_successful(job, 0, NUM_BATCHES, True), True) self.assertEqual(ci_cache.is_successful(job, 1, NUM_BATCHES, False), False) @@ -273,7 +271,7 @@ class TestCiCache(unittest.TestCase): ### create new cache object and verify the same checks ci_cache = CiCache(s3_mock, DIGESTS) - for job in JobNames: + for job in CI.JobNames: self.assertEqual(ci_cache.is_successful(job, 0, NUM_BATCHES, False), True) self.assertEqual(ci_cache.is_successful(job, 0, NUM_BATCHES, True), True) self.assertEqual(ci_cache.is_successful(job, 1, NUM_BATCHES, False), False) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index badbc4c5dcf..76ab3b82fa5 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -1,30 +1,306 @@ #!/usr/bin/env python3 import unittest -from ci_config import CIStages, JobNames, CI_CONFIG, Runners +from ci_config import CI +import ci as CIPY +from ci_settings import CiSettings +from pr_info import PRInfo, EventType +from s3_helper import S3Helper +from ci_cache import CiCache +from ci_utils import normalize_string + + +_TEST_EVENT_JSON = {"dummy": "dummy"} + +# pylint:disable=protected-access class TestCIConfig(unittest.TestCase): def test_runner_config(self): """check runner is provided w/o exception""" - for job in JobNames: - runner = CI_CONFIG.get_runner_type(job) - self.assertIn(runner, Runners) + for job in CI.JobNames: + self.assertIn(CI.JOB_CONFIGS[job].runner_type, CI.Runners) def test_job_stage_config(self): - """check runner is provided w/o exception""" - for job in JobNames: - stage = CI_CONFIG.get_job_ci_stage(job) - if job in [ - JobNames.STYLE_CHECK, - JobNames.FAST_TEST, - JobNames.JEPSEN_KEEPER, - JobNames.BUILD_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - ]: - assert ( - stage == CIStages.NA - ), "These jobs are not in CI stages, must be NA" + """ + check runner is provided w/o exception + """ + # check stages + for job in CI.JobNames: + if job in CI.BuildNames: + self.assertTrue( + CI.get_job_ci_stage(job) + in (CI.WorkflowStages.BUILDS_1, CI.WorkflowStages.BUILDS_2) + ) else: - assert stage != CIStages.NA, f"stage not found for [{job}]" - self.assertIn(stage, CIStages) + if job in ( + CI.JobNames.STYLE_CHECK, + CI.JobNames.FAST_TEST, + CI.JobNames.JEPSEN_SERVER, + CI.JobNames.JEPSEN_KEEPER, + CI.JobNames.BUILD_CHECK, + ): + self.assertEqual( + CI.get_job_ci_stage(job), + CI.WorkflowStages.NA, + msg=f"Stage for [{job}] is not correct", + ) + else: + self.assertTrue( + CI.get_job_ci_stage(job) + in (CI.WorkflowStages.TESTS_1, CI.WorkflowStages.TESTS_3), + msg=f"Stage for [{job}] is not correct", + ) + + def test_build_jobs_configs(self): + """ + check build jobs have non-None build_config attribute + check test jobs have None build_config attribute + """ + for job in CI.JobNames: + if job in CI.BuildNames: + self.assertTrue( + isinstance(CI.JOB_CONFIGS[job].build_config, CI.BuildConfig) + ) + else: + self.assertTrue(CI.JOB_CONFIGS[job].build_config is None) + + def test_ci_py_for_pull_request(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + settings.ci_sets = [CI.Tags.CI_SET_BUILDS] + settings.include_keywords = [ + "package", + "integration", + "upgrade", + "clickHouse_build_check", + "stateless", + ] + settings.exclude_keywords = ["asan", "aarch64"] + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + # make it pull request info + pr_info.event_type = EventType.PULL_REQUEST + pr_info.number = 12345 + assert pr_info.is_pr and not pr_info.is_release and not pr_info.is_master + assert not pr_info.is_merge_queue + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [] + for set_ in settings.ci_sets: + tag_config = CI.get_tag_config(set_) + assert tag_config + set_jobs = tag_config.run_jobs + for job in set_jobs: + if any(k in normalize_string(job) for k in settings.exclude_keywords): + continue + expected_jobs_to_do.append(job) + for job, config in CI.JOB_CONFIGS.items(): + if not any( + keyword in normalize_string(job) + for keyword in settings.include_keywords + ): + continue + if any( + keyword in normalize_string(job) + for keyword in settings.exclude_keywords + ): + continue + if config.random_bucket: + continue + if job not in expected_jobs_to_do: + expected_jobs_to_do.append(job) + + random_buckets = [] + for job, config in ci_cache.jobs_to_do.items(): + if config.random_bucket: + self.assertTrue( + config.random_bucket not in random_buckets, + "Only one job must be picked up from each random bucket", + ) + random_buckets.append(config.random_bucket) + actual_jobs_to_do.remove(job) + + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + + def test_ci_py_for_pull_request_no_settings(self): + """ + checks ci.py job configuration in PR with empty ci_settings + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + # make it pull request info + pr_info.event_type = EventType.PULL_REQUEST + pr_info.number = 12345 + assert pr_info.is_pr and not pr_info.is_release and not pr_info.is_master + assert not pr_info.is_merge_queue + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [] + for job, config in CI.JOB_CONFIGS.items(): + if config.random_bucket: + continue + if config.release_only: + continue + if config.run_by_label: + continue + expected_jobs_to_do.append(job) + + random_buckets = [] + for job, config in ci_cache.jobs_to_do.items(): + if config.random_bucket: + self.assertTrue( + config.random_bucket not in random_buckets, + "Only one job must be picked up from each random bucket", + ) + random_buckets.append(config.random_bucket) + actual_jobs_to_do.remove(job) + + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + + def test_ci_py_for_master(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + pr_info.event_type = EventType.PUSH + assert pr_info.number == 0 and pr_info.is_release and not pr_info.is_merge_queue + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [] + for job, config in CI.JOB_CONFIGS.items(): + if config.pr_only: + continue + if config.run_by_label: + continue + if job in CI.MQ_JOBS: + continue + expected_jobs_to_do.append(job) + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + + def test_ci_py_for_merge_queue(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + # make it merge_queue + pr_info.event_type = EventType.MERGE_QUEUE + assert ( + pr_info.number == 0 + and pr_info.is_merge_queue + and not pr_info.is_release + and not pr_info.is_master + and not pr_info.is_pr + ) + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [ + "Style check", + "Fast test", + "binary_release", + "Unit tests (release)", + ] + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + + def test_ci_py_await(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + pr_info.event_type = EventType.PUSH + pr_info.number = 0 + assert pr_info.is_release and not pr_info.is_merge_queue + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + self.assertTrue(not ci_cache.jobs_to_skip, "Must be no jobs in skip list") + all_jobs_in_wf = list(ci_cache.jobs_to_do) + assert not ci_cache.jobs_to_wait + ci_cache.await_pending_jobs(is_release=pr_info.is_release, dry_run=True) + assert not ci_cache.jobs_to_skip + assert not ci_cache.jobs_to_wait + + # pretend there are pending jobs that we neet to wait + ci_cache.jobs_to_wait = dict(ci_cache.jobs_to_do) + for job, config in ci_cache.jobs_to_wait.items(): + assert not config.pending_batches + assert config.batches + config.pending_batches = list(config.batches) + for job, config in ci_cache.jobs_to_wait.items(): + for batch in range(config.num_batches): + record = CiCache.Record( + record_type=CiCache.RecordType.PENDING, + job_name=job, + job_digest=ci_cache.job_digests[job], + batch=batch, + num_batches=config.num_batches, + release_branch=True, + ) + for record_t_, records_ in ci_cache.records.items(): + if record_t_.value == CiCache.RecordType.PENDING.value: + records_[record.to_str_key()] = record + + def _test_await_for_batch( + ci_cache: CiCache, record_type: CiCache.RecordType, batch: int + ) -> None: + assert ci_cache.jobs_to_wait + for job_, config_ in ci_cache.jobs_to_wait.items(): + record = CiCache.Record( + record_type=record_type, + job_name=job_, + job_digest=ci_cache.job_digests[job_], + batch=batch, + num_batches=config_.num_batches, + release_branch=True, + ) + for record_t_, records_ in ci_cache.records.items(): + if record_t_.value == record_type.value: + records_[record.to_str_key()] = record + # await + ci_cache.await_pending_jobs(is_release=pr_info.is_release, dry_run=True) + for _, config_ in ci_cache.jobs_to_wait.items(): + assert config_.pending_batches + if ( + record_type != CiCache.RecordType.PENDING + and batch < config_.num_batches + ): + assert batch not in config_.pending_batches + else: + assert batch in config_.pending_batches + + _test_await_for_batch(ci_cache, CiCache.RecordType.SUCCESSFUL, 0) + # check all one-batch jobs are in jobs_to_skip + for job in all_jobs_in_wf: + config = CI.JOB_CONFIGS[job] + if config.num_batches == 1: + self.assertTrue(job in ci_cache.jobs_to_skip) + self.assertTrue(job not in ci_cache.jobs_to_do) + else: + self.assertTrue(job not in ci_cache.jobs_to_skip) + self.assertTrue(job in ci_cache.jobs_to_do) + + _test_await_for_batch(ci_cache, CiCache.RecordType.FAILED, 1) + _test_await_for_batch(ci_cache, CiCache.RecordType.SUCCESSFUL, 2) + + self.assertTrue(len(ci_cache.jobs_to_skip) > 0) + self.assertTrue(len(ci_cache.jobs_to_do) > 0) + self.assertCountEqual( + list(ci_cache.jobs_to_do) + ci_cache.jobs_to_skip, all_jobs_in_wf + ) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 60888932803..42a1460691a 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -4,7 +4,7 @@ import unittest from ci_settings import CiSettings -from ci_config import JobConfig +from ci_config import CI _TEST_BODY_1 = """ #### Run only: @@ -166,15 +166,14 @@ class TestCIOptions(unittest.TestCase): ["tsan", "foobar", "aarch64", "analyzer", "s3_storage", "coverage"], ) - jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST} - jobs_configs[ - "fuzzers" - ].run_by_label = ( + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST + } + jobs_configs["fuzzers"].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs[ - "Integration tests (asan)" - ].release_only = ( + jobs_configs["Integration tests (asan)"].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -210,7 +209,10 @@ class TestCIOptions(unittest.TestCase): ) def test_options_applied_2(self): - jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST_2} + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST_2 + } jobs_configs["Style check"].release_only = True jobs_configs["Fast test"].pr_only = True jobs_configs["fuzzers"].run_by_label = "TEST_LABEL" @@ -252,7 +254,10 @@ class TestCIOptions(unittest.TestCase): def test_options_applied_3(self): ci_settings = CiSettings() ci_settings.include_keywords = ["Style"] - jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST_2} + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST_2 + } jobs_configs["Style check"].release_only = True jobs_configs["Fast test"].pr_only = True # no settings are set @@ -296,10 +301,13 @@ class TestCIOptions(unittest.TestCase): ) self.assertCountEqual(ci_options.include_keywords, ["analyzer"]) self.assertIsNone(ci_options.exclude_keywords) - jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST} - jobs_configs[ - "fuzzers" - ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST + } + jobs_configs["fuzzers"].run_by_label = ( + "TEST_LABEL" # check "fuzzers" does not appears in the result + ) jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 2e645d2f19a..98fdbf8bd7e 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -4,11 +4,11 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_ou start_total=`date +%s` -start=`date +%s` -echo "Check " | ts -./check-black -n |& tee /test_output/black_output.txt -runtime=$((`date +%s`-start)) -echo "Check python formatting with black. Done. $runtime seconds." +#start=`date +%s` +#echo "Check " | ts +#./check-black -n |& tee /test_output/black_output.txt +#runtime=$((`date +%s`-start)) +#echo "Check python formatting with black. Done. $runtime seconds." start=`date +%s` ./check-pylint -n |& tee /test_output/pylint_output.txt diff --git a/utils/check-style/process_style_check_result.py b/utils/check-style/process_style_check_result.py index 2c349114a59..460b5f841ab 100755 --- a/utils/check-style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -17,7 +17,7 @@ def process_result(result_folder): "shellcheck", "style", "pylint", - "black", + # "black", "flake8", "mypy", "typos", From 9d416da6010db16d0cde24d0c3fba1fe4dab97a1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 10:23:26 +0000 Subject: [PATCH 706/856] 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 707/856] 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 0b8035a6cd8a8d1aa8a3a6a22e8b0f7db48e0a53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Jun 2024 13:35:12 +0200 Subject: [PATCH 708/856] Test requires libraries --- tests/queries/0_stateless/03172_bcrypt_validation.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03172_bcrypt_validation.sql b/tests/queries/0_stateless/03172_bcrypt_validation.sql index 2c34a7d60d1..37dd0c9bb5d 100644 --- a/tests/queries/0_stateless/03172_bcrypt_validation.sql +++ b/tests/queries/0_stateless/03172_bcrypt_validation.sql @@ -1,2 +1,3 @@ +-- Tags: no-fasttest DROP USER IF EXISTS 03172_user_invalid_bcrypt_hash; CREATE USER 03172_user_invalid_bcrypt_hash IDENTIFIED WITH bcrypt_hash BY '012345678901234567890123456789012345678901234567890123456789'; -- { serverError BAD_ARGUMENTS } From aacb85495b826d26c84a17b28c3a479124aaf319 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 14 Jun 2024 13:58:04 +0200 Subject: [PATCH 709/856] Update tests/integration/test_manipulate_statistics/test.py --- tests/integration/test_manipulate_statistics/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 9485b611c01..a602cce63df 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -157,7 +157,6 @@ def test_replicated_table_ddl(started_cluster): ) node2.query("insert into test_stat values(1,2,3), (2,3,4)") - # check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True) node1.query( From 515f911033f2c911318da1fa6d6ab2c1842a676d Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 14 Jun 2024 15:28:55 +0300 Subject: [PATCH 710/856] Add handling on commit --- src/Coordination/KeeperStorage.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 5920e098470..9251624346b 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -885,6 +885,10 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) session_and_auth[operation.session_id].emplace_back(std::move(operation.auth_id)); return Coordination::Error::ZOK; } + else if constexpr (std::same_as) + { + return Coordination::Error::ZOK; + } else { // shouldn't be called in any process functions From b43240068600c806679a061c2359c728e62dd22f Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 13:10:04 +0000 Subject: [PATCH 711/856] 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 712/856] 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 713/856] 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 5f3535309be83cebab9fae967d6b4ab739aca9dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 14 Jun 2024 15:38:09 +0200 Subject: [PATCH 714/856] 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 715/856] 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 716/856] 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 717/856] 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 36a575e124a3fe940cfcecb565270784b0f89ee4 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 14 Jun 2024 15:55:02 +0200 Subject: [PATCH 718/856] Fix float page --- docs/en/sql-reference/data-types/float.md | 32 +++++++++++++++-------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 23131d5b4fe..7c232a286ea 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -7,33 +7,43 @@ sidebar_label: Float32, Float64 # Float32, Float64 :::note -If you need accurate calculations, in particular if you work with financial or business data requiring a high precision you should consider using Decimal instead. Floats might lead to inaccurate results as illustrated below: +If you need accurate calculations, in particular if you work with financial or business data requiring a high precision, you should consider using [Decimal](../data-types/decimal.md) instead. -``` +[Floating Point Numbers](https://en.wikipedia.org/wiki/IEEE_754) might lead to inaccurate results as illustrated below: + +```sql CREATE TABLE IF NOT EXISTS float_vs_decimal ( my_float Float64, my_decimal Decimal64(3) -)Engine=MergeTree ORDER BY tuple() - -INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; # Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal +) +Engine=MergeTree +ORDER BY tuple(); +# Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal +INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; +``` +``` SELECT sum(my_float), sum(my_decimal) FROM float_vs_decimal; -> 500279.56300000014 500279.563 + +┌──────sum(my_float)─┬─sum(my_decimal)─┐ +│ 499693.60500000004 │ 499693.605 │ +└────────────────────┴─────────────────┘ SELECT sumKahan(my_float), sumKahan(my_decimal) FROM float_vs_decimal; -> 500279.563 500279.563 + +┌─sumKahan(my_float)─┬─sumKahan(my_decimal)─┐ +│ 499693.605 │ 499693.605 │ +└────────────────────┴──────────────────────┘ ``` ::: -[Floating point numbers](https://en.wikipedia.org/wiki/IEEE_754). - -Types are equivalent to types of C: +The equivalent types in ClickHouse and in C are given below: - `Float32` — `float`. - `Float64` — `double`. -Aliases: +Float types in ClickHouse have the following aliases: - `Float32` — `FLOAT`, `REAL`, `SINGLE`. - `Float64` — `DOUBLE`, `DOUBLE PRECISION`. From ad103b9db8dfeec41862ccb18acfe908b5b0d14b Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 14 Jun 2024 16:06:40 +0200 Subject: [PATCH 719/856] Update float.md --- docs/en/sql-reference/data-types/float.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 7c232a286ea..3c789076c1e 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -20,7 +20,7 @@ CREATE TABLE IF NOT EXISTS float_vs_decimal Engine=MergeTree ORDER BY tuple(); -# Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal +# Generate 1 000 000 random numbers with 2 decimal places and store them as a float and as a decimal INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; ``` ``` From e0128556f273873d9b027624e01fd86cec4ff653 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 14 Jun 2024 16:30:27 +0200 Subject: [PATCH 720/856] Disable long test in coverage run --- ...581_share_big_sets_between_multiple_mutations_tasks_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql index ff8b9c71e92..741d0177971 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql @@ -1,4 +1,4 @@ --- Tags: long, no-debug, no-tsan, no-asan, no-ubsan, no-msan, no-parallel +-- Tags: long, no-debug, no-tsan, no-asan, no-ubsan, no-msan, no-parallel, no-sanitize-coverage -- no-parallel because the sets use a lot of memory, which may interfere with other tests From d421d39b735b29b348d21a607b8de61ecfaf3994 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 14 Jun 2024 15:33:31 +0000 Subject: [PATCH 721/856] Update version_date.tsv and changelogs after v23.8.15.35-lts --- docs/changelogs/v23.8.15.35-lts.md | 40 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 41 insertions(+) create mode 100644 docs/changelogs/v23.8.15.35-lts.md diff --git a/docs/changelogs/v23.8.15.35-lts.md b/docs/changelogs/v23.8.15.35-lts.md new file mode 100644 index 00000000000..bab5c507fe8 --- /dev/null +++ b/docs/changelogs/v23.8.15.35-lts.md @@ -0,0 +1,40 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.15.35-lts (060ff8e813a) FIXME as compared to v23.8.14.6-lts (967e51c1d6b) + +#### Build/Testing/Packaging Improvement +* Backported in [#63621](https://github.com/ClickHouse/ClickHouse/issues/63621): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#65153](https://github.com/ClickHouse/ClickHouse/issues/65153): 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 [#64422](https://github.com/ClickHouse/ClickHouse/issues/64422): Fixes [#59989](https://github.com/ClickHouse/ClickHouse/issues/59989): runs init scripts when force-enabled or when no database exists, rather than the inverse. [#59991](https://github.com/ClickHouse/ClickHouse/pull/59991) ([jktng](https://github.com/jktng)). +* Backported in [#64016](https://github.com/ClickHouse/ClickHouse/issues/64016): Fix "Invalid storage definition in metadata file" for parameterized views. [#60708](https://github.com/ClickHouse/ClickHouse/pull/60708) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#63456](https://github.com/ClickHouse/ClickHouse/issues/63456): Fix the issue where the function `addDays` (and similar functions) reports an error when the first parameter is `DateTime64`. [#61561](https://github.com/ClickHouse/ClickHouse/pull/61561) ([Shuai li](https://github.com/loneylee)). +* Backported in [#63289](https://github.com/ClickHouse/ClickHouse/issues/63289): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63512](https://github.com/ClickHouse/ClickHouse/issues/63512): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)). +* Backported in [#63902](https://github.com/ClickHouse/ClickHouse/issues/63902): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#64104](https://github.com/ClickHouse/ClickHouse/issues/64104): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64265](https://github.com/ClickHouse/ClickHouse/issues/64265): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64867](https://github.com/ClickHouse/ClickHouse/issues/64867): 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)). + +#### NO CL CATEGORY + +* Backported in [#63704](https://github.com/ClickHouse/ClickHouse/issues/63704):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Installation test has wrong check_state'. [#63994](https://github.com/ClickHouse/ClickHouse/pull/63994) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#63343](https://github.com/ClickHouse/ClickHouse/issues/63343): The commit url has different pattern. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63965](https://github.com/ClickHouse/ClickHouse/issues/63965): fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#64043](https://github.com/ClickHouse/ClickHouse/issues/64043): Do not create new release in release branch automatically. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Pin requests to fix the integration tests. [#65183](https://github.com/ClickHouse/ClickHouse/pull/65183) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + 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 5a577915aba1f927124d340d0b27da39a9db5bad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 15:43:43 +0000 Subject: [PATCH 722/856] 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 ed60188050926d03042f659a6a448a4e7d634ca8 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 14 Jun 2024 17:53:31 +0200 Subject: [PATCH 723/856] mute test test_query_is_canceled_with_inf_retries --- tests/integration/test_checking_s3_blobs_paranoid/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 476f7c61b28..b3b8bf86800 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -512,6 +512,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( ), error +@pytest.mark.skip(reason="test is flaky, waiting ClickHouse/issues/64451") def test_query_is_canceled_with_inf_retries(cluster, broken_s3): node = cluster.instances["node_with_inf_s3_retries"] From 592d704bf96ed941bb0c8a6713967cf0c173c386 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 14 Jun 2024 18:42:37 +0200 Subject: [PATCH 724/856] Build Report check remove pending status --- tests/ci/build_report_check.py | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 5f24fd8f224..434cf83cfb1 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -24,7 +24,7 @@ from report import ( BuildResult, JobReport, create_build_html_report, - get_worst_status, + get_worst_status, FAILURE, ) from stopwatch import Stopwatch from ci_config import CI @@ -134,17 +134,16 @@ def main(): # Check if there are no builds at all, do not override bad status if summary_status == SUCCESS: if missing_builds: - summary_status = PENDING + summary_status = FAILURE elif ok_groups == 0: summary_status = ERROR - addition = "" - if missing_builds: - addition = ( - f" ({required_builds - missing_builds} of {required_builds} builds are OK)" - ) + description = "" - description = f"{ok_groups}/{total_groups} artifact groups are OK{addition}" + if missing_builds: + description = f"{missing_builds} of {required_builds} builds are missing." + + description += f" {ok_groups}/{total_groups} artifact groups are OK" JobReport( description=description, From eb2af8202224587ff2c7d078639d947009e5f676 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 14 Jun 2024 18:43:59 +0200 Subject: [PATCH 725/856] reenable black --- utils/check-style/check_py.sh | 10 +++++----- utils/check-style/process_style_check_result.py | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 98fdbf8bd7e..2e645d2f19a 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -4,11 +4,11 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_ou start_total=`date +%s` -#start=`date +%s` -#echo "Check " | ts -#./check-black -n |& tee /test_output/black_output.txt -#runtime=$((`date +%s`-start)) -#echo "Check python formatting with black. Done. $runtime seconds." +start=`date +%s` +echo "Check " | ts +./check-black -n |& tee /test_output/black_output.txt +runtime=$((`date +%s`-start)) +echo "Check python formatting with black. Done. $runtime seconds." start=`date +%s` ./check-pylint -n |& tee /test_output/pylint_output.txt diff --git a/utils/check-style/process_style_check_result.py b/utils/check-style/process_style_check_result.py index 460b5f841ab..2c349114a59 100755 --- a/utils/check-style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -17,7 +17,7 @@ def process_result(result_folder): "shellcheck", "style", "pylint", - # "black", + "black", "flake8", "mypy", "typos", From 7b43ede04e38a05e2e6ee46ea5ca6cb1a6d12d98 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 14 Jun 2024 18:45:08 +0200 Subject: [PATCH 726/856] style fix --- tests/ci/build_report_check.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 434cf83cfb1..664f6a7cbb9 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -24,7 +24,8 @@ from report import ( BuildResult, JobReport, create_build_html_report, - get_worst_status, FAILURE, + get_worst_status, + FAILURE, ) from stopwatch import Stopwatch from ci_config import CI From ba349ac73b778e949472d49996503e440d4f2839 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 14 Jun 2024 18:50:53 +0200 Subject: [PATCH 727/856] ci test improvement --- tests/ci/test_ci_config.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 76ab3b82fa5..3454ea1851a 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -285,6 +285,11 @@ class TestCIConfig(unittest.TestCase): else: assert batch in config_.pending_batches + for _, config_ in ci_cache.jobs_to_do.items(): + # jobs to do must have batches to run before/after await + # if it's an empty list after await - apparently job has not been removed after await + assert config_.batches + _test_await_for_batch(ci_cache, CiCache.RecordType.SUCCESSFUL, 0) # check all one-batch jobs are in jobs_to_skip for job in all_jobs_in_wf: From 799e5730082fb2060171b93ae25e735054d5308b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 14 Jun 2024 19:03:32 +0200 Subject: [PATCH 728/856] Fix silly typo that caused wrong tags messages --- tests/ci/release.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 7490ce0a373..7ab1149310f 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -550,7 +550,7 @@ class Release: def _create_tag( self, tag: str, commit: str, tag_message: str = "" ) -> Iterator[None]: - tag_message = tag_message or "Release {tag}" + tag_message = tag_message or f"Release {tag}" # Create tag even in dry-run self.run(f"git tag -a -m '{tag_message}' '{tag}' {commit}") rollback_cmd = f"git tag -d '{tag}'" From 02d76f60823b71e6b1de5f7562e749ed45b25312 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 17:05:03 +0000 Subject: [PATCH 729/856] 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 a3469098e7abaa05d30c7eba58d9be3727c7771f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jun 2024 20:35:59 +0200 Subject: [PATCH 730/856] Fix 01246_buffer_flush flakiness (by tunning timeouts) Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/01246_buffer_flush.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01246_buffer_flush.sql b/tests/queries/0_stateless/01246_buffer_flush.sql index 36bcaae383f..66f93371c29 100644 --- a/tests/queries/0_stateless/01246_buffer_flush.sql +++ b/tests/queries/0_stateless/01246_buffer_flush.sql @@ -9,14 +9,14 @@ create table data_01256 as system.numbers Engine=Memory(); select 'min'; create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, - 2, 100, /* time */ + 5, 100, /* time */ 4, 100, /* rows */ 1, 1e6 /* bytes */ ); insert into buffer_01256 select * from system.numbers limit 5; select count() from data_01256; --- sleep 2 (min time) + 1 (round up) + bias (1) = 4 -select sleepEachRow(2) from numbers(2) FORMAT Null; +-- It is enough to ensure that the buffer will be flushed earlier then 2*min_time (10 sec) +select sleepEachRow(9) FORMAT Null SETTINGS function_sleep_max_microseconds_per_block=10e6; select count() from data_01256; drop table buffer_01256; From 9bf849ca73537d1493c96618abb7b92013d64ba4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 14 Jun 2024 19:27:57 +0000 Subject: [PATCH 731/856] Automatic style fix --- tests/ci/ssh.py | 6 +++--- tests/ci/test_ci_options.py | 14 +++++++++----- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/ci/ssh.py b/tests/ci/ssh.py index 89d90d724d2..321826fcf44 100644 --- a/tests/ci/ssh.py +++ b/tests/ci/ssh.py @@ -37,9 +37,9 @@ class SSHAgent: ssh_options = ( "," + os.environ["SSH_OPTIONS"] if os.environ.get("SSH_OPTIONS") else "" ) - os.environ["SSH_OPTIONS"] = ( - f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" - ) + os.environ[ + "SSH_OPTIONS" + ] = f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" def add(self, key): key_pub = self._key_pub(key) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 42a1460691a..73499540b78 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -170,10 +170,14 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( + jobs_configs[ + "fuzzers" + ].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs["Integration tests (asan)"].release_only = ( + jobs_configs[ + "Integration tests (asan)" + ].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -305,9 +309,9 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" does not appears in the result - ) + jobs_configs[ + "fuzzers" + ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 74266dcdcdcfcc44472ee490273f655d495b496d Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 15 Jun 2024 08:37:28 +0200 Subject: [PATCH 732/856] Remove const --- src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h index 956ba57e712..b0b7c2ad8f4 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h @@ -17,7 +17,7 @@ public: String getName() const override { return "PrettyCompactBlockOutputFormat"; } private: - void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths, const bool write_footer); + void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths, bool write_footer); void writeBottom(const Widths & max_widths); void writeRow( size_t row_num, 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 733/856] 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 0d7a4915aef9ce023bc586c3d7486a68f354bf04 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 17:38:57 +0200 Subject: [PATCH 734/856] align job names, add more tests --- .github/workflows/backport_branches.yml | 6 +- .github/workflows/release_branches.yml | 6 +- tests/ci/ci_definitions.py | 10 +-- tests/ci/test_ci_config.py | 106 ++++++++++++++++++++++++ tests/ci/test_ci_options.py | 26 +++--- 5 files changed, 128 insertions(+), 26 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index c8c6ba30b0b..e1980ec9ef2 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -70,7 +70,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Compatibility check (amd64) + test_name: Compatibility check (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} CompatibilityCheckAarch64: @@ -194,7 +194,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (amd64) + test_name: Install packages (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} run_command: | @@ -204,7 +204,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (arm64) + test_name: Install packages (aarch64) runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} run_command: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index f9b8a4fa764..45eb7431bb4 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -65,7 +65,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Compatibility check (amd64) + test_name: Compatibility check (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} CompatibilityCheckAarch64: @@ -244,7 +244,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (amd64) + test_name: Install packages (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} run_command: | @@ -254,7 +254,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (arm64) + test_name: Install packages (aarch64) runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} run_command: | diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index e427d48505e..3d732a997f5 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -98,7 +98,7 @@ class JobNames(metaclass=WithIter): FAST_TEST = "Fast test" DOCKER_SERVER = "Docker server image" DOCKER_KEEPER = "Docker keeper image" - INSTALL_TEST_AMD = "Install packages (amd64)" + INSTALL_TEST_AMD = "Install packages (release)" INSTALL_TEST_ARM = "Install packages (aarch64)" STATELESS_TEST_DEBUG = "Stateless tests (debug)" @@ -167,8 +167,8 @@ class JobNames(metaclass=WithIter): JEPSEN_KEEPER = "ClickHouse Keeper Jepsen" JEPSEN_SERVER = "ClickHouse Server Jepsen" - PERFORMANCE_TEST_AMD64 = "Performance Comparison" - PERFORMANCE_TEST_ARM64 = "Performance Comparison Aarch64" + PERFORMANCE_TEST_AMD64 = "Performance Comparison (release)" + PERFORMANCE_TEST_ARM64 = "Performance Comparison (aarch64)" SQL_LOGIC_TEST = "Sqllogic test (release)" @@ -176,10 +176,10 @@ class JobNames(metaclass=WithIter): SQLANCER_DEBUG = "SQLancer (debug)" SQLTEST = "SQLTest" - COMPATIBILITY_TEST = "Compatibility check (amd64)" + COMPATIBILITY_TEST = "Compatibility check (release)" COMPATIBILITY_TEST_ARM = "Compatibility check (aarch64)" - CLICKBENCH_TEST = "ClickBench (amd64)" + CLICKBENCH_TEST = "ClickBench (release)" CLICKBENCH_TEST_ARM = "ClickBench (aarch64)" LIBFUZZER_TEST = "libFuzzer tests" diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 3454ea1851a..28d417d23ec 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -21,6 +21,112 @@ class TestCIConfig(unittest.TestCase): for job in CI.JobNames: self.assertIn(CI.JOB_CONFIGS[job].runner_type, CI.Runners) + def test_builds_configs(self): + """build name in the build config must match the job name""" + for job in CI.JobNames: + self.assertTrue(CI.JOB_CONFIGS[job].runner_type in CI.Runners) + if job in CI.BuildNames: + self.assertTrue(CI.JOB_CONFIGS[job].build_config.name == job) + self.assertTrue(CI.JOB_CONFIGS[job].required_builds is None) + else: + self.assertTrue(CI.JOB_CONFIGS[job].build_config is None) + if "asan" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_ASAN, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "msan" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_MSAN, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "tsan" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_TSAN, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "ubsan" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_UBSAN, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "debug" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_DEBUG, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "release" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + in ( + CI.BuildNames.PACKAGE_RELEASE, + CI.BuildNames.BINARY_RELEASE, + ), + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "coverage" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_RELEASE_COVERAGE, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "aarch" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_AARCH64, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "amd64" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_RELEASE, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "uzzer" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] == CI.BuildNames.FUZZERS, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "Docker" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + in ( + CI.BuildNames.PACKAGE_RELEASE, + CI.BuildNames.PACKAGE_AARCH64, + ), + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "SQLTest" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_RELEASE, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "Jepsen" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + in ( + CI.BuildNames.PACKAGE_RELEASE, + CI.BuildNames.BINARY_RELEASE, + ), + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif job in ( + CI.JobNames.STYLE_CHECK, + CI.JobNames.FAST_TEST, + CI.JobNames.BUILD_CHECK, + CI.JobNames.DOCS_CHECK, + CI.JobNames.BUGFIX_VALIDATE, + ): + self.assertTrue(CI.JOB_CONFIGS[job].required_builds is None) + else: + print(f"Job [{job}] required build not checked") + def test_job_stage_config(self): """ check runner is provided w/o exception diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 73499540b78..fac6662b1aa 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -64,8 +64,8 @@ _TEST_JOB_LIST = [ "fuzzers", "Docker server image", "Docker keeper image", - "Install packages (amd64)", - "Install packages (arm64)", + "Install packages (release)", + "Install packages (aarch64)", "Stateless tests (debug)", "Stateless tests (release)", "Stateless tests (coverage)", @@ -120,15 +120,15 @@ _TEST_JOB_LIST = [ "AST fuzzer (ubsan)", "ClickHouse Keeper Jepsen", "ClickHouse Server Jepsen", - "Performance Comparison", - "Performance Comparison Aarch64", + "Performance Comparison (release)", + "Performance Comparison (aarch64)", "Sqllogic test (release)", "SQLancer (release)", "SQLancer (debug)", "SQLTest", - "Compatibility check (amd64)", + "Compatibility check (release)", "Compatibility check (aarch64)", - "ClickBench (amd64)", + "ClickBench (release)", "ClickBench (aarch64)", "libFuzzer tests", "ClickHouse build check", @@ -170,14 +170,10 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs[ - "fuzzers" - ].run_by_label = ( + jobs_configs["fuzzers"].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs[ - "Integration tests (asan)" - ].release_only = ( + jobs_configs["Integration tests (asan)"].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -309,9 +305,9 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs[ - "fuzzers" - ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result + jobs_configs["fuzzers"].run_by_label = ( + "TEST_LABEL" # check "fuzzers" does not appears in the result + ) jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 2af2fc148694fbd9cd837825192fa78781875d85 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 17:59:42 +0200 Subject: [PATCH 735/856] more tests --- tests/ci/ci_config.py | 1 + tests/ci/test_ci_config.py | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 3dc0040663b..7eb4b7fdec2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -24,6 +24,7 @@ class CI: from ci_definitions import BuildNames as BuildNames from ci_definitions import StatusNames as StatusNames from ci_definitions import CHECK_DESCRIPTIONS as CHECK_DESCRIPTIONS + from ci_definitions import REQUIRED_CHECKS as REQUIRED_CHECKS from ci_definitions import MQ_JOBS as MQ_JOBS from ci_definitions import WorkflowStages as WorkflowStages from ci_definitions import Runners as Runners diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 28d417d23ec..c2d1a743d42 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -21,9 +21,16 @@ class TestCIConfig(unittest.TestCase): for job in CI.JobNames: self.assertIn(CI.JOB_CONFIGS[job].runner_type, CI.Runners) + def test_required_checks(self): + for job in CI.REQUIRED_CHECKS: + if job in (CI.StatusNames.PR_CHECK, CI.StatusNames.SYNC): + continue + self.assertTrue(job in CI.JOB_CONFIGS, f"Job [{job}] not in job config") + def test_builds_configs(self): """build name in the build config must match the job name""" for job in CI.JobNames: + self.assertTrue(job in CI.JOB_CONFIGS) self.assertTrue(CI.JOB_CONFIGS[job].runner_type in CI.Runners) if job in CI.BuildNames: self.assertTrue(CI.JOB_CONFIGS[job].build_config.name == job) From ed9bfbe393d217c56d8677677c023100cdb6b1ac Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 15 Jun 2024 16:06:31 +0000 Subject: [PATCH 736/856] Automatic style fix --- tests/ci/test_ci_options.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index fac6662b1aa..fc21c7dda4e 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -170,10 +170,14 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( + jobs_configs[ + "fuzzers" + ].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs["Integration tests (asan)"].release_only = ( + jobs_configs[ + "Integration tests (asan)" + ].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -305,9 +309,9 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" does not appears in the result - ) + jobs_configs[ + "fuzzers" + ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 1d48bd7846d3e89809b42d55e87ce96e74c19561 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 18:13:11 +0200 Subject: [PATCH 737/856] add SyncState class fir further adoption --- tests/ci/ci.py | 2 +- tests/ci/ci_config.py | 1 + tests/ci/ci_definitions.py | 8 ++++++++ tests/ci/test_ci_config.py | 2 +- 4 files changed, 11 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index cdf1be96b68..63d4120a210 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -913,7 +913,7 @@ def _set_pending_statuses(pr_info: PRInfo) -> None: commit.create_status( state=PENDING, target_url="", - description="", + description=CI.SyncState.PENDING, context=CI.StatusNames.SYNC, ) except Exception as ex: diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7eb4b7fdec2..655314cb629 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -25,6 +25,7 @@ class CI: from ci_definitions import StatusNames as StatusNames from ci_definitions import CHECK_DESCRIPTIONS as CHECK_DESCRIPTIONS from ci_definitions import REQUIRED_CHECKS as REQUIRED_CHECKS + from ci_definitions import SyncState as SyncState from ci_definitions import MQ_JOBS as MQ_JOBS from ci_definitions import WorkflowStages as WorkflowStages from ci_definitions import Runners as Runners diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 3d732a997f5..9a4b845a61e 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -212,6 +212,14 @@ class StatusNames(metaclass=WithIter): PR_CHECK = "PR Check" +class SyncState(metaclass=WithIter): + PENDING = "awaiting merge" + MERGE_FAILED = "merge failed" + TESTING = "awaiting test results" + TESTS_FAILED = "tests failed" + COMPLETED = "completed" + + @dataclass class DigestConfig: # all files, dirs to include into digest, glob supported diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index c2d1a743d42..d8208a5d9bc 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -12,7 +12,7 @@ from ci_utils import normalize_string _TEST_EVENT_JSON = {"dummy": "dummy"} -# pylint:disable=protected-access +# pylint:disable=protected-access,union-attr class TestCIConfig(unittest.TestCase): From 0aa7665f048f25b275f25d2a4e2133029d9d2463 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 18:25:49 +0200 Subject: [PATCH 738/856] do not pylint ci unittests --- utils/check-style/check-pylint | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-pylint b/utils/check-style/check-pylint index 7959a414023..8cfbc68ac96 100755 --- a/utils/check-style/check-pylint +++ b/utils/check-style/check-pylint @@ -10,6 +10,7 @@ function xargs-pylint { xargs -P "$(nproc)" -n "$1" pylint --rcfile="$ROOT_PATH/pyproject.toml" --persistent=no --score=n } -find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 50 +# exclude ci unittest scripts from check: test_* +find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | grep -v "/test_" | xargs-pylint 50 # Beware, there lambdas are checked. All of them contain `app`, and it causes brain-cucumber-zalgo find "$ROOT_PATH/tests/ci" -mindepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 1 From f81cbbab2ab7963b0ed90e814e28c6ab20db0c82 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 19:02:15 +0200 Subject: [PATCH 739/856] do not mypy ci unittests --- utils/check-style/check-mypy | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-mypy b/utils/check-style/check-mypy index 42cb7fbbd15..4434377e627 100755 --- a/utils/check-style/check-mypy +++ b/utils/check-style/check-mypy @@ -11,13 +11,15 @@ GIT_ROOT=${GIT_ROOT:-.} CONFIG="$GIT_ROOT/tests/ci/.mypy.ini" DIRS=("$GIT_ROOT/tests/ci/" "$GIT_ROOT/tests/ci/"*/) tmp=$(mktemp) + for dir in "${DIRS[@]}"; do if ! compgen -G "$dir"/*.py > /dev/null; then continue fi - if ! mypy --config-file="$CONFIG" --sqlite-cache "$dir"/*.py > "$tmp" 2>&1; then + if ! mypy --config-file="$CONFIG" --sqlite-cache $(find "$dir" -maxdepth 1 -name "*.py" | grep -v "test_") > "$tmp" 2>&1; then echo "Errors while processing $dir": cat "$tmp" fi done + rm -rf "$tmp" From 38f01bd831aa36efb76b43bf2f53357cd10ef92b Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Sun, 16 Jun 2024 14:31:49 +0800 Subject: [PATCH 740/856] 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 a6ff0941e91fef1bf5635e6cf1aabb89ac17ebf2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 09:01:15 +0200 Subject: [PATCH 741/856] Rename tests --- .../{03170_esc_crash.reference => 03170_ecs_crash.reference} | 0 .../0_stateless/{03170_esc_crash.sh => 03170_ecs_crash.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03170_esc_crash.reference => 03170_ecs_crash.reference} (100%) rename tests/queries/0_stateless/{03170_esc_crash.sh => 03170_ecs_crash.sh} (100%) diff --git a/tests/queries/0_stateless/03170_esc_crash.reference b/tests/queries/0_stateless/03170_ecs_crash.reference similarity index 100% rename from tests/queries/0_stateless/03170_esc_crash.reference rename to tests/queries/0_stateless/03170_ecs_crash.reference diff --git a/tests/queries/0_stateless/03170_esc_crash.sh b/tests/queries/0_stateless/03170_ecs_crash.sh similarity index 100% rename from tests/queries/0_stateless/03170_esc_crash.sh rename to tests/queries/0_stateless/03170_ecs_crash.sh From 1594f84daf18ed4c810c5b4023968194474ef618 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 Jun 2024 11:36:12 +0200 Subject: [PATCH 742/856] 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 24dfc107556af61f2e5c2fbd1ace4f7b6c5368d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 12:57:05 +0200 Subject: [PATCH 743/856] Remove outdated override in stress tests --- docker/test/stateless/stress_tests.lib | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 3b6ad244c82..c069ccbdd8d 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -89,10 +89,6 @@ function configure() # since we run clickhouse from root sudo chown root: /var/lib/clickhouse - # Set more frequent update period of asynchronous metrics to more frequently update information about real memory usage (less chance of OOM). - echo "1" \ - > /etc/clickhouse-server/config.d/asynchronous_metrics_update_period_s.xml - local total_mem total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB total_mem=$(( total_mem*1024 )) # bytes From cb390cc3d99731bf90f81646c406808e31fd0eaa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 13:14:59 +0200 Subject: [PATCH 744/856] Fix alloc/dealloc mismatch or prevent false-positive ASan error in the AWS SDK --- contrib/aws-crt-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index f532d6abc0d..16c304dffe0 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit f532d6abc0d2b0d8b5d6fe9e7c51eaedbe4afbd0 +Subproject commit 16c304dffe0727819b05a0bd6630e29b074008c6 From 2ecc53787ea2229dcd09c2c80b5054d952669be3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 Jun 2024 14:30:09 +0200 Subject: [PATCH 745/856] Ping CI From 20ca8f8714f2846077bddb82fdf38aedff56d7f0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 15:20:26 +0200 Subject: [PATCH 746/856] Debug AWS --- contrib/aws-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt index abde20addaf..5b447865db6 100644 --- a/contrib/aws-cmake/CMakeLists.txt +++ b/contrib/aws-cmake/CMakeLists.txt @@ -52,6 +52,8 @@ if (USE_S2N) list(APPEND AWS_PRIVATE_COMPILE_DEFS "-DUSE_S2N") endif() +list(APPEND AWS_PRIVATE_COMPILE_DEFS "-O0") + # Directories. SET(AWS_SDK_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws") From 642dc35a76b9a3753589c4b7eb377748103e6803 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 15:21:17 +0200 Subject: [PATCH 747/856] Debug AWS --- contrib/aws-cmake/CMakeLists.txt | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt index 5b447865db6..be563179a1e 100644 --- a/contrib/aws-cmake/CMakeLists.txt +++ b/contrib/aws-cmake/CMakeLists.txt @@ -52,9 +52,6 @@ if (USE_S2N) list(APPEND AWS_PRIVATE_COMPILE_DEFS "-DUSE_S2N") endif() -list(APPEND AWS_PRIVATE_COMPILE_DEFS "-O0") - - # Directories. SET(AWS_SDK_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws") SET(AWS_SDK_CORE_DIR "${AWS_SDK_DIR}/src/aws-cpp-sdk-core") @@ -374,6 +371,7 @@ target_include_directories(_aws SYSTEM BEFORE PUBLIC ${AWS_PUBLIC_INCLUDES}) target_include_directories(_aws SYSTEM BEFORE PRIVATE ${AWS_PRIVATE_INCLUDES}) target_compile_definitions(_aws PUBLIC ${AWS_PUBLIC_COMPILE_DEFS}) target_compile_definitions(_aws PRIVATE ${AWS_PRIVATE_COMPILE_DEFS}) +target_compile_options(_aws PRIVATE "-O0") target_link_libraries(_aws PRIVATE ${AWS_PRIVATE_LIBS}) aws_set_thread_affinity_method(_aws) From 37eb815b174476f4d0ceba7d2b5d68b5ad9c75a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 17:12:05 +0200 Subject: [PATCH 748/856] Fix error --- contrib/aws-crt-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index 16c304dffe0..0217761556a 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit 16c304dffe0727819b05a0bd6630e29b074008c6 +Subproject commit 0217761556a7ba7ec537fe933d0ab1159096746e From b41bb966056a82cd8d1435de8d6a02d7c4063fa2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 17:12:19 +0200 Subject: [PATCH 749/856] Revert "Debug AWS" This reverts commit 642dc35a76b9a3753589c4b7eb377748103e6803. --- contrib/aws-cmake/CMakeLists.txt | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt index be563179a1e..5b447865db6 100644 --- a/contrib/aws-cmake/CMakeLists.txt +++ b/contrib/aws-cmake/CMakeLists.txt @@ -52,6 +52,9 @@ if (USE_S2N) list(APPEND AWS_PRIVATE_COMPILE_DEFS "-DUSE_S2N") endif() +list(APPEND AWS_PRIVATE_COMPILE_DEFS "-O0") + + # Directories. SET(AWS_SDK_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws") SET(AWS_SDK_CORE_DIR "${AWS_SDK_DIR}/src/aws-cpp-sdk-core") @@ -371,7 +374,6 @@ target_include_directories(_aws SYSTEM BEFORE PUBLIC ${AWS_PUBLIC_INCLUDES}) target_include_directories(_aws SYSTEM BEFORE PRIVATE ${AWS_PRIVATE_INCLUDES}) target_compile_definitions(_aws PUBLIC ${AWS_PUBLIC_COMPILE_DEFS}) target_compile_definitions(_aws PRIVATE ${AWS_PRIVATE_COMPILE_DEFS}) -target_compile_options(_aws PRIVATE "-O0") target_link_libraries(_aws PRIVATE ${AWS_PRIVATE_LIBS}) aws_set_thread_affinity_method(_aws) From 2c1918908a69ba6277312a3031bb278485645ef7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 17:12:55 +0200 Subject: [PATCH 750/856] Revert "Debug AWS" This reverts commit 20ca8f8714f2846077bddb82fdf38aedff56d7f0. --- contrib/aws-cmake/CMakeLists.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt index 5b447865db6..abde20addaf 100644 --- a/contrib/aws-cmake/CMakeLists.txt +++ b/contrib/aws-cmake/CMakeLists.txt @@ -52,8 +52,6 @@ if (USE_S2N) list(APPEND AWS_PRIVATE_COMPILE_DEFS "-DUSE_S2N") endif() -list(APPEND AWS_PRIVATE_COMPILE_DEFS "-O0") - # Directories. SET(AWS_SDK_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws") From 35ce8c6e722833604f5d23bf00fc28f596277186 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 17:32:41 +0200 Subject: [PATCH 751/856] Version in User-Agent --- src/IO/S3/PocoHTTPClient.cpp | 3 ++- src/IO/S3/PocoHTTPClientFactory.cpp | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index dcd644c2d81..04982f14f36 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -1,4 +1,5 @@ #include +#include #include "config.h" #if USE_AWS_S3 @@ -110,7 +111,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , error_report(error_report_) { /// This is used to identify configurations created by us. - userAgent = "ClickHouse"; + userAgent = std::string(VERSION_FULL) + VERSION_OFFICIAL; } void PocoHTTPClientConfiguration::updateSchemeAndRegion() diff --git a/src/IO/S3/PocoHTTPClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp index b2f84c5e827..abec907778c 100644 --- a/src/IO/S3/PocoHTTPClientFactory.cpp +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -15,7 +15,7 @@ namespace DB::S3 std::shared_ptr PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & client_configuration) const { - if (client_configuration.userAgent == "ClickHouse") + if (client_configuration.userAgent.starts_with("ClickHouse")) return std::make_shared(static_cast(client_configuration)); else /// This client is created inside the AWS SDK with default settings to obtain ECS credentials from localhost. return std::make_shared(client_configuration); From 8b1f1947f7c3270d01fae2b362b554c4db31aaa7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jun 2024 17:39:36 +0200 Subject: [PATCH 752/856] Slightly better user-agent --- src/IO/ReadWriteBufferFromHTTP.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 303ffb744b5..4f883a9b4ed 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -221,7 +221,7 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( if (iter == http_header_entries.end()) { - http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}", VERSION_STRING)); + http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}{}", VERSION_STRING, VERSION_OFFICIAL)); } if (!delay_initialization && use_external_buffer) 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 753/856] 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 754/856] 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 755/856] 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 756/856] 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 757/856] 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 9cc4cc6d3c4512a7ac8b897be66372795d2d7380 Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Thu, 13 Jun 2024 16:05:12 +0800 Subject: [PATCH 758/856] add tests for 'boom filter index with map' --- ...67_boom_filter_index_with_map.reference.j2 | 4 +++ .../03167_boom_filter_index_with_map.sql.j2 | 31 +++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 tests/queries/0_stateless/03167_boom_filter_index_with_map.reference.j2 create mode 100644 tests/queries/0_stateless/03167_boom_filter_index_with_map.sql.j2 diff --git a/tests/queries/0_stateless/03167_boom_filter_index_with_map.reference.j2 b/tests/queries/0_stateless/03167_boom_filter_index_with_map.reference.j2 new file mode 100644 index 00000000000..71dc879f28e --- /dev/null +++ b/tests/queries/0_stateless/03167_boom_filter_index_with_map.reference.j2 @@ -0,0 +1,4 @@ +{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%} +{'xxx':56} +{56:'xxx'} +{% endfor -%} diff --git a/tests/queries/0_stateless/03167_boom_filter_index_with_map.sql.j2 b/tests/queries/0_stateless/03167_boom_filter_index_with_map.sql.j2 new file mode 100644 index 00000000000..4147bd84e8e --- /dev/null +++ b/tests/queries/0_stateless/03167_boom_filter_index_with_map.sql.j2 @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS boom_filter_map_1; +DROP TABLE IF EXISTS boom_filter_map_2; + +{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%} + +CREATE TABLE boom_filter_map_1 +( + `m` Map(String, {{ type }}), + INDEX index_models_value_bloom_filter mapValues(m) TYPE bloom_filter GRANULARITY 1 +) + ENGINE = MergeTree +ORDER BY tuple(); + +CREATE TABLE boom_filter_map_2 +( + `m` Map({{ type }}, String), + INDEX index_models_value_bloom_filter mapKeys(m) TYPE bloom_filter GRANULARITY 1 +) + ENGINE = MergeTree +ORDER BY tuple(); + +INSERT INTO boom_filter_map_1 (m) values (map('xxx', 56)); +INSERT INTO boom_filter_map_2 (m) values (map(56, 'xxx')); + +SELECT m FROM boom_filter_map_1 WHERE (m['xxx']) = 56; +SELECT m FROM boom_filter_map_2 WHERE (m[56]) = 'xxx'; + +DROP TABLE IF EXISTS boom_filter_map_1; +DROP TABLE IF EXISTS boom_filter_map_2; + +{% endfor -%} From 9ecbc568903abba46fc0f874bb938882224591ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 05:12:45 +0200 Subject: [PATCH 759/856] 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 92f538ae504b10f826fd01a3d91fdc7b66bb3e1c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 05:21:09 +0200 Subject: [PATCH 760/856] Disable async loading for MaterializedMySQL tests --- .../configs/no_async_load.xml | 3 +++ tests/integration/test_materialized_mysql_database/test.py | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_materialized_mysql_database/configs/no_async_load.xml diff --git a/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml b/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml new file mode 100644 index 00000000000..1100874ad48 --- /dev/null +++ b/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml @@ -0,0 +1,3 @@ + + false + diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 080a850a8c6..5efef3624db 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -24,7 +24,7 @@ mysql8_node = None node_db = cluster.add_instance( "node1", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users.xml"], with_mysql57=True, with_mysql8=True, @@ -32,7 +32,7 @@ node_db = cluster.add_instance( ) node_disable_bytes_settings = cluster.add_instance( "node2", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users_disable_bytes_settings.xml"], with_mysql57=False, with_mysql8=False, @@ -40,7 +40,7 @@ node_disable_bytes_settings = cluster.add_instance( ) node_disable_rows_settings = cluster.add_instance( "node3", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users_disable_rows_settings.xml"], with_mysql57=False, with_mysql8=False, From 279716519d88861d65ed8cbf8ef6cfc71a38de9e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 05:37:08 +0200 Subject: [PATCH 761/856] Fix tests --- .../configs/no_async_load.xml | 3 +++ tests/integration/test_disk_over_web_server/test.py | 10 +++++----- .../test_replicated_database/configs/config.xml | 1 + .../test_replicated_database/configs/config2.xml | 1 + 4 files changed, 10 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_disk_over_web_server/configs/no_async_load.xml diff --git a/tests/integration/test_disk_over_web_server/configs/no_async_load.xml b/tests/integration/test_disk_over_web_server/configs/no_async_load.xml new file mode 100644 index 00000000000..1100874ad48 --- /dev/null +++ b/tests/integration/test_disk_over_web_server/configs/no_async_load.xml @@ -0,0 +1,3 @@ + + false + diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index f4ea7d54571..891ee8f00f5 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -11,13 +11,13 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "node1", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, use_old_analyzer=True, ) cluster.add_instance( "node2", - main_configs=["configs/storage_conf_web.xml"], + main_configs=["configs/storage_conf_web.xml", "configs/no_async_load.xml"], with_nginx=True, stay_alive=True, with_zookeeper=True, @@ -25,7 +25,7 @@ def cluster(): ) cluster.add_instance( "node3", - main_configs=["configs/storage_conf_web.xml"], + main_configs=["configs/storage_conf_web.xml", "configs/no_async_load.xml"], with_nginx=True, with_zookeeper=True, use_old_analyzer=True, @@ -33,7 +33,7 @@ def cluster(): cluster.add_instance( "node4", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, stay_alive=True, with_installed_binary=True, @@ -42,7 +42,7 @@ def cluster(): ) cluster.add_instance( "node5", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, use_old_analyzer=True, ) diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 5150e9096de..706628cf93b 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -6,4 +6,5 @@ 50 42 + false diff --git a/tests/integration/test_replicated_database/configs/config2.xml b/tests/integration/test_replicated_database/configs/config2.xml index 727461697ca..8192c191952 100644 --- a/tests/integration/test_replicated_database/configs/config2.xml +++ b/tests/integration/test_replicated_database/configs/config2.xml @@ -7,4 +7,5 @@ 50 42 group + false From 188f8a3df74caf830ad1ced3c4cf6dfb0aa90093 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 07:06:30 +0200 Subject: [PATCH 762/856] Fix test --- .../queries/0_stateless/01676_dictget_in_default_expression.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql index 1785979f60b..db23ae1919c 100644 --- a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql +++ b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql @@ -23,7 +23,7 @@ ATTACH DATABASE test_01676; SELECT 'status_after_detach_and_attach:'; -- It can be not loaded, or not even finish attaching in case of asynchronous tables loading. -SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict')::String, 'NOT_LOADED'); +SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict')::Nullable(String), 'NOT_LOADED'); INSERT INTO test_01676.table (x) VALUES (toInt64(4)); SELECT * FROM test_01676.table ORDER BY x; From 7bed33012db0047db31e302c42193887138281f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 07:51:58 +0200 Subject: [PATCH 763/856] 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 f00750a152babd8bab7141abfac47b626e8a12e2 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 17 Jun 2024 08:33:31 +0200 Subject: [PATCH 764/856] do not overwrite Sync status if set --- tests/ci/ci.py | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 63d4120a210..135a4c91c56 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -31,6 +31,7 @@ from commit_status_helper import ( get_commit, post_commit_status, set_status_comment, + get_commit_filtered_statuses, ) from digest_helper import DockerDigester from env_helper import ( @@ -909,13 +910,23 @@ def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> No def _set_pending_statuses(pr_info: PRInfo) -> None: commit = get_commit(GitHub(get_best_robot_token(), per_page=100), pr_info.sha) try: - print("Set SYNC status to pending") - commit.create_status( - state=PENDING, - target_url="", - description=CI.SyncState.PENDING, - context=CI.StatusNames.SYNC, - ) + found = False + statuses = get_commit_filtered_statuses(commit) + for commit_status in statuses: + if commit_status.context == CI.StatusNames.SYNC: + print( + f"Sync status found [{commit_status.state}], [{commit_status.description}] - won't be overwritten" + ) + found = True + break + if not found: + print("Set Sync status to pending") + commit.create_status( + state=PENDING, + target_url="", + description=CI.SyncState.PENDING, + context=CI.StatusNames.SYNC, + ) except Exception as ex: print(f"ERROR: failed to set GH commit status, ex: {ex}") From 016a680d309d6bace02712325f0fa765926444c5 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Mon, 17 Jun 2024 12:07:03 +0300 Subject: [PATCH 765/856] reload-ci From 59b871663fa0296d5f91d862bedf18915db233d0 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 17 Jun 2024 08:33:31 +0200 Subject: [PATCH 766/856] More ci unit tests --- tests/ci/ci_config.py | 18 +++++++++++++----- tests/ci/ci_definitions.py | 25 ++++++++++++++++++++++--- tests/ci/test_ci_config.py | 36 ++++++++++++++++++++++++++++++++++++ tests/ci/test_ci_options.py | 14 +++++--------- 4 files changed, 76 insertions(+), 17 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 655314cb629..60c6a60af1a 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -260,7 +260,8 @@ class CI: required_builds=[BuildNames.PACKAGE_RELEASE] ), JobNames.INSTALL_TEST_ARM: CommonJobConfigs.INSTALL_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_AARCH64] + required_builds=[BuildNames.PACKAGE_AARCH64], + runner_type=Runners.STYLE_CHECKER_ARM, ), JobNames.STATEFUL_TEST_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN] @@ -284,7 +285,8 @@ class CI: required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE] ), JobNames.STATEFUL_TEST_AARCH64: CommonJobConfigs.STATEFUL_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_AARCH64] + required_builds=[BuildNames.PACKAGE_AARCH64], + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE] @@ -331,6 +333,7 @@ class CI: ), JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 @@ -397,7 +400,9 @@ class CI: required_builds=[BuildNames.PACKAGE_TSAN], num_batches=6 ), JobNames.INTEGRATION_TEST_ARM: CommonJobConfigs.INTEGRATION_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=6 + required_builds=[BuildNames.PACKAGE_AARCH64], + num_batches=6, + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.INTEGRATION_TEST: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -414,6 +419,7 @@ class CI: JobNames.COMPATIBILITY_TEST_ARM: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], required_on_release_branch=True, + runner_type=Runners.STYLE_CHECKER_ARM, ), JobNames.UNIT_TEST: CommonJobConfigs.UNIT_TEST.with_properties( required_builds=[BuildNames.BINARY_RELEASE], @@ -467,6 +473,7 @@ class CI: required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=4, run_by_label="pr-performance", + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.SQLANCER: CommonJobConfigs.SQLLANCER_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -480,11 +487,12 @@ class CI: JobNames.SQLTEST: CommonJobConfigs.SQL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.CLICKBENCH_TEST: CommonJobConfigs.SQL_TEST.with_properties( + JobNames.CLICKBENCH_TEST: CommonJobConfigs.CLICKBENCH_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.CLICKBENCH_TEST_ARM: CommonJobConfigs.SQL_TEST.with_properties( + JobNames.CLICKBENCH_TEST_ARM: CommonJobConfigs.CLICKBENCH_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 9a4b845a61e..fdd5dc7a671 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -280,6 +280,8 @@ class JobConfig: # GH Runner type (tag from @Runners) runner_type: str + # used for config validation in ci unittests + job_name_keyword: str = "" # builds required for the job (applicable for test jobs) required_builds: Optional[List[str]] = None # build config for the build job (applicable for builds) @@ -328,6 +330,7 @@ class CommonJobConfigs: """ BUILD_REPORT = JobConfig( + job_name_keyword="build_check", run_command="build_report_check.py", digest=DigestConfig( include_paths=[ @@ -338,23 +341,26 @@ class CommonJobConfigs: runner_type=Runners.STYLE_CHECKER_ARM, ) COMPATIBILITY_TEST = JobConfig( + job_name_keyword="compatibility", digest=DigestConfig( include_paths=["./tests/ci/compatibility_check.py"], docker=["clickhouse/test-old-ubuntu", "clickhouse/test-old-centos"], ), run_command="compatibility_check.py", - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER, ) INSTALL_TEST = JobConfig( + job_name_keyword="install", digest=DigestConfig( include_paths=["./tests/ci/install_check.py"], docker=["clickhouse/install-deb-test", "clickhouse/install-rpm-test"], ), run_command='install_check.py "$CHECK_NAME"', - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER, timeout=900, ) STATELESS_TEST = JobConfig( + job_name_keyword="stateless", digest=DigestConfig( include_paths=[ "./tests/ci/functional_test_check.py", @@ -371,6 +377,7 @@ class CommonJobConfigs: timeout=10800, ) STATEFUL_TEST = JobConfig( + job_name_keyword="stateful", digest=DigestConfig( include_paths=[ "./tests/ci/functional_test_check.py", @@ -387,6 +394,7 @@ class CommonJobConfigs: timeout=3600, ) STRESS_TEST = JobConfig( + job_name_keyword="stress", digest=DigestConfig( include_paths=[ "./tests/queries/0_stateless/", @@ -403,6 +411,7 @@ class CommonJobConfigs: timeout=9000, ) UPGRADE_TEST = JobConfig( + job_name_keyword="upgrade", digest=DigestConfig( include_paths=["./tests/ci/upgrade_check.py"], exclude_files=[".md"], @@ -412,6 +421,7 @@ class CommonJobConfigs: runner_type=Runners.STRESS_TESTER, ) INTEGRATION_TEST = JobConfig( + job_name_keyword="integration", digest=DigestConfig( include_paths=[ "./tests/ci/integration_test_check.py", @@ -425,12 +435,14 @@ class CommonJobConfigs: runner_type=Runners.STRESS_TESTER, ) ASTFUZZER_TEST = JobConfig( + job_name_keyword="ast", digest=DigestConfig(), run_command="ast_fuzzer_check.py", run_always=True, runner_type=Runners.FUZZER_UNIT_TESTER, ) UNIT_TEST = JobConfig( + job_name_keyword="unit", digest=DigestConfig( include_paths=["./tests/ci/unit_tests_check.py"], exclude_files=[".md"], @@ -440,6 +452,7 @@ class CommonJobConfigs: runner_type=Runners.FUZZER_UNIT_TESTER, ) PERF_TESTS = JobConfig( + job_name_keyword="performance", digest=DigestConfig( include_paths=[ "./tests/ci/performance_comparison_check.py", @@ -452,6 +465,7 @@ class CommonJobConfigs: runner_type=Runners.STRESS_TESTER, ) SQLLANCER_TEST = JobConfig( + job_name_keyword="lancer", digest=DigestConfig(), run_command="sqlancer_check.py", release_only=True, @@ -459,6 +473,7 @@ class CommonJobConfigs: runner_type=Runners.FUZZER_UNIT_TESTER, ) SQLLOGIC_TEST = JobConfig( + job_name_keyword="logic", digest=DigestConfig( include_paths=["./tests/ci/sqllogic_test.py"], exclude_files=[".md"], @@ -467,9 +482,10 @@ class CommonJobConfigs: run_command="sqllogic_test.py", timeout=10800, release_only=True, - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER, ) SQL_TEST = JobConfig( + job_name_keyword="sqltest", digest=DigestConfig( include_paths=["./tests/ci/sqltest.py"], exclude_files=[".md"], @@ -481,12 +497,14 @@ class CommonJobConfigs: runner_type=Runners.FUZZER_UNIT_TESTER, ) BUGFIX_TEST = JobConfig( + job_name_keyword="bugfix", digest=DigestConfig(), run_command="bugfix_validate_check.py", timeout=900, runner_type=Runners.FUNC_TESTER, ) DOCKER_SERVER = JobConfig( + job_name_keyword="docker", required_on_release_branch=True, run_command='docker_server.py --check-name "$CHECK_NAME" --release-type head --allow-build-reuse', digest=DigestConfig( @@ -498,6 +516,7 @@ class CommonJobConfigs: runner_type=Runners.STYLE_CHECKER, ) CLICKBENCH_TEST = JobConfig( + job_name_keyword="clickbench", digest=DigestConfig( include_paths=[ "tests/ci/clickbench.py", diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index d8208a5d9bc..7a51a65b5d5 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -20,6 +20,42 @@ class TestCIConfig(unittest.TestCase): """check runner is provided w/o exception""" for job in CI.JobNames: self.assertIn(CI.JOB_CONFIGS[job].runner_type, CI.Runners) + if ( + job + in ( + CI.JobNames.STYLE_CHECK, + CI.JobNames.BUILD_CHECK, + ) + or "jepsen" in job.lower() + ): + self.assertTrue( + "style" in CI.JOB_CONFIGS[job].runner_type, + f"Job [{job}] must have style-checker(-aarch64) runner", + ) + elif "binary_" in job.lower() or "package_" in job.lower(): + self.assertTrue( + CI.JOB_CONFIGS[job].runner_type == CI.Runners.BUILDER, + f"Job [{job}] must have [{CI.Runners.BUILDER}] runner", + ) + elif "aarch64" in job.lower(): + self.assertTrue( + "aarch" in CI.JOB_CONFIGS[job].runner_type, + f"Job [{job}] does not match runner [{CI.JOB_CONFIGS[job].runner_type}]", + ) + else: + self.assertTrue( + "aarch" not in CI.JOB_CONFIGS[job].runner_type, + f"Job [{job}] does not match runner [{CI.JOB_CONFIGS[job].runner_type}]", + ) + + def test_common_configs_applied_properly(self): + for job in CI.JobNames: + if CI.JOB_CONFIGS[job].job_name_keyword: + self.assertTrue( + CI.JOB_CONFIGS[job].job_name_keyword.lower() + in normalize_string(job), + f"Job [{job}] apparently uses wrong common config with job keyword [{CI.JOB_CONFIGS[job].job_name_keyword}]", + ) def test_required_checks(self): for job in CI.REQUIRED_CHECKS: diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index fc21c7dda4e..fac6662b1aa 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -170,14 +170,10 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs[ - "fuzzers" - ].run_by_label = ( + jobs_configs["fuzzers"].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs[ - "Integration tests (asan)" - ].release_only = ( + jobs_configs["Integration tests (asan)"].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -309,9 +305,9 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs[ - "fuzzers" - ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result + jobs_configs["fuzzers"].run_by_label = ( + "TEST_LABEL" # check "fuzzers" does not appears in the result + ) jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 9f64e46f5e5e9cb4b14ab94fa5badbed8850d8bf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 17 Jun 2024 10:55:34 +0000 Subject: [PATCH 767/856] Automatic style fix --- tests/ci/test_ci_options.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index fac6662b1aa..fc21c7dda4e 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -170,10 +170,14 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( + jobs_configs[ + "fuzzers" + ].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs["Integration tests (asan)"].release_only = ( + jobs_configs[ + "Integration tests (asan)" + ].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -305,9 +309,9 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" does not appears in the result - ) + jobs_configs[ + "fuzzers" + ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From b845a242cc1f616eca3919a9d509afabc63b4f7f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jun 2024 11:06:22 +0000 Subject: [PATCH 768/856] 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 769/856] 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 770/856] 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 b2947d9b4939ec3440b955578fd7e1d2430f2185 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 17 Jun 2024 13:52:30 +0200 Subject: [PATCH 771/856] Fix crash in 03036_dynamic_read_subcolumns --- src/DataTypes/Serializations/SerializationVariantElement.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 1f9a81ac671..ec0b4019c2f 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -146,7 +146,7 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( } /// If we started to read a new column, reinitialize variant column in deserialization state. - if (!variant_element_state->variant || result_column->empty()) + if (!variant_element_state->variant || mutable_column->empty()) { variant_element_state->variant = mutable_column->cloneEmpty(); From a826d7130f4b92dc5983dd9656707dd8793468c6 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 17 Jun 2024 13:57:59 +0200 Subject: [PATCH 772/856] Rearranged includes --- .../AggregateFunctionGroupConcat.cpp | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp index 767c536a8fd..7541d64af4a 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp @@ -1,25 +1,25 @@ +#include #include #include -#include + +#include +#include +#include + #include +#include -#include -#include +#include +#include +#include +#include #include #include #include -#include -#include - -#include -#include -#include -#include -#include - -#include +#include +#include namespace DB From e13ff4ec43ab8f8d9738a11ff4372232874b7d19 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 17 Jun 2024 14:00:58 +0200 Subject: [PATCH 773/856] add Builds_2 dependency for Build_Report --- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 837dbba6174..88bc50a729d 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -119,7 +119,7 @@ jobs: Builds_Report: # run report check for failed builds to indicate the CI error if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} - needs: [RunConfig, Builds_1] + needs: [RunConfig, Builds_1, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 422bcf551ae..70b71da8fa5 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -147,7 +147,7 @@ jobs: Builds_Report: # run report check for failed builds to indicate the CI error if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} - needs: [RunConfig, StyleCheck, Builds_1] + needs: [RunConfig, StyleCheck, Builds_1, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check From 6360a0a357a7b8feb172b5811a48e8a14543f881 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 17 Jun 2024 12:03:37 +0000 Subject: [PATCH 774/856] Move tests 02942_variant_cast and 02944_variant_as_common_type to analyzer_tech_debt.txt and remove set allow_experimental_analyzer=0 --- tests/analyzer_tech_debt.txt | 2 ++ tests/queries/0_stateless/02942_variant_cast.sql | 1 - tests/queries/0_stateless/02944_variant_as_common_type.sql | 2 -- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 5f798158a41..1f7357b6494 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -9,3 +9,5 @@ 01287_max_execution_speed # Check after ConstantNode refactoring 02154_parser_backtracking +02944_variant_as_common_type +02942_variant_cast diff --git a/tests/queries/0_stateless/02942_variant_cast.sql b/tests/queries/0_stateless/02942_variant_cast.sql index fc2d1d63657..33587e3e438 100644 --- a/tests/queries/0_stateless/02942_variant_cast.sql +++ b/tests/queries/0_stateless/02942_variant_cast.sql @@ -1,5 +1,4 @@ set allow_experimental_variant_type=1; -set allow_experimental_analyzer=0; -- It's currently doesn't work with analyzer because of the way it works with constants, but it will be refactored and fixed in future select NULL::Variant(String, UInt64); select 42::UInt64::Variant(String, UInt64); diff --git a/tests/queries/0_stateless/02944_variant_as_common_type.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql index e985cf365dd..49ea5f2769c 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,5 +1,3 @@ -set allow_experimental_analyzer=0; -- The result type for if function with constant is different with analyzer. It wil be fixed after refactoring around constants in analyzer. - set allow_experimental_variant_type=1; set use_variant_as_common_type=1; From e3818a97944bb10d56646b4145696a60a7aa0edd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 15:55:42 +0000 Subject: [PATCH 775/856] 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 776/856] 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 777/856] 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 778/856] 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 779/856] 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 699f6334cbe0c2aea65d47618e885ed821add212 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 17 Jun 2024 17:38:18 +0200 Subject: [PATCH 780/856] Fix condition --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 68337aeae4c..8860050c5b9 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -985,8 +985,8 @@ std::string QueryAnalyzer::rewriteAggregateFunctionNameIfNeeded( { result_aggregate_function_name = settings.count_distinct_implementation; } - else if (settings.rewrite_count_distinct_if_with_count_distinct_implementation && - (aggregate_function_name_lowercase == "countdistinctif" || aggregate_function_name_lowercase == "countifdistinct")) + else if (aggregate_function_name_lowercase == "countifdistinct" || + (settings.rewrite_count_distinct_if_with_count_distinct_implementation && aggregate_function_name_lowercase == "countdistinctif")) { result_aggregate_function_name = settings.count_distinct_implementation; result_aggregate_function_name += "If"; From c9178418ff2e3f36ab9801791f1fd9a3cf2c28e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jun 2024 15:56:51 +0000 Subject: [PATCH 781/856] 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 782/856] 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 783/856] 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 784/856] 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 785/856] reload-ci From 8d072ade18b61b6eafef0899f5ce551030b6662d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jun 2024 22:35:49 +0200 Subject: [PATCH 786/856] 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 b150b0f5faf1ee5a0702bdadb1bc081253acd253 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 17 Jun 2024 23:11:59 +0200 Subject: [PATCH 787/856] Revert "Fix AWS ECS" --- contrib/aws-crt-cpp | 2 +- .../ProxyConfigurationResolverProvider.cpp | 35 ++++++++++++------- .../ProxyConfigurationResolverProvider.h | 5 +-- src/IO/ReadWriteBufferFromHTTP.cpp | 2 +- src/IO/S3/Client.cpp | 6 ++-- src/IO/S3/PocoHTTPClient.cpp | 20 ++--------- src/IO/S3/PocoHTTPClient.h | 14 ++++---- src/IO/S3/PocoHTTPClientFactory.cpp | 5 +-- .../0_stateless/03170_ecs_crash.reference | 4 --- tests/queries/0_stateless/03170_ecs_crash.sh | 9 ----- 10 files changed, 41 insertions(+), 61 deletions(-) delete mode 100644 tests/queries/0_stateless/03170_ecs_crash.reference delete mode 100755 tests/queries/0_stateless/03170_ecs_crash.sh diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index 0217761556a..f532d6abc0d 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit 0217761556a7ba7ec537fe933d0ab1159096746e +Subproject commit f532d6abc0d2b0d8b5d6fe9e7c51eaedbe4afbd0 diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index a46837bfdb9..b06073121e7 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -112,8 +112,9 @@ namespace return configuration.has(config_prefix + ".uri"); } - /* New syntax requires protocol prefix " or " - */ + /* + * New syntax requires protocol prefix " or " + * */ std::optional getProtocolPrefix( ProxyConfiguration::Protocol request_protocol, const String & config_prefix, @@ -129,18 +130,22 @@ namespace return protocol_prefix; } + template std::optional calculatePrefixBasedOnSettingsSyntax( - bool new_syntax, ProxyConfiguration::Protocol request_protocol, const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration ) { if (!configuration.has(config_prefix)) + { return std::nullopt; + } - if (new_syntax) + if constexpr (new_syntax) + { return getProtocolPrefix(request_protocol, config_prefix, configuration); + } return config_prefix; } @@ -150,21 +155,24 @@ std::shared_ptr ProxyConfigurationResolverProvider:: Protocol request_protocol, const Poco::Util::AbstractConfiguration & configuration) { - if (auto resolver = getFromSettings(true, request_protocol, "proxy", configuration)) + if (auto resolver = getFromSettings(request_protocol, "proxy", configuration)) + { return resolver; + } return std::make_shared( request_protocol, isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); } +template std::shared_ptr ProxyConfigurationResolverProvider::getFromSettings( - bool new_syntax, Protocol request_protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration) + const Poco::Util::AbstractConfiguration & configuration +) { - auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(new_syntax, request_protocol, config_prefix, configuration); + auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(request_protocol, config_prefix, configuration); if (!prefix_opt) { @@ -187,17 +195,20 @@ std::shared_ptr ProxyConfigurationResolverProvider:: std::shared_ptr ProxyConfigurationResolverProvider::getFromOldSettingsFormat( Protocol request_protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration) + const Poco::Util::AbstractConfiguration & configuration +) { - /* First try to get it from settings only using the combination of config_prefix and configuration. + /* + * First try to get it from settings only using the combination of config_prefix and configuration. * This logic exists for backward compatibility with old S3 storage specific proxy configuration. * */ - if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(false, request_protocol, config_prefix + ".proxy", configuration)) + if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(request_protocol, config_prefix + ".proxy", configuration)) { return resolver; } - /* In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings. + /* + * In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings. * Falls back to Environment resolver if no configuration is found. * */ return ProxyConfigurationResolverProvider::get(request_protocol, configuration); diff --git a/src/Common/ProxyConfigurationResolverProvider.h b/src/Common/ProxyConfigurationResolverProvider.h index 357b218e499..ebf22f7e92a 100644 --- a/src/Common/ProxyConfigurationResolverProvider.h +++ b/src/Common/ProxyConfigurationResolverProvider.h @@ -33,11 +33,12 @@ public: ); private: + template static std::shared_ptr getFromSettings( - bool is_new_syntax, Protocol protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration); + const Poco::Util::AbstractConfiguration & configuration + ); }; } diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 4f883a9b4ed..303ffb744b5 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -221,7 +221,7 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( if (iter == http_header_entries.end()) { - http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}{}", VERSION_STRING, VERSION_OFFICIAL)); + http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}", VERSION_STRING)); } if (!delay_initialization && use_external_buffer) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index cbb61deea9f..9229342b8c1 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -972,10 +972,10 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT { auto context = Context::getGlobalContextInstance(); chassert(context); - auto proxy_configuration_resolver = ProxyConfigurationResolverProvider::get(ProxyConfiguration::protocolFromString(protocol), context->getConfigRef()); + auto proxy_configuration_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::protocolFromString(protocol), context->getConfigRef()); - auto per_request_configuration = [=]{ return proxy_configuration_resolver->resolve(); }; - auto error_report = [=](const ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); }; + auto per_request_configuration = [=] () { return proxy_configuration_resolver->resolve(); }; + auto error_report = [=] (const DB::ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); }; auto config = PocoHTTPClientConfiguration( per_request_configuration, diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 04982f14f36..1cef43530e0 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -1,5 +1,4 @@ #include -#include #include "config.h" #if USE_AWS_S3 @@ -18,7 +17,6 @@ #include #include #include -#include #include #include @@ -31,7 +29,6 @@ #include - static const int SUCCESS_RESPONSE_MIN = 200; static const int SUCCESS_RESPONSE_MAX = 299; @@ -87,7 +84,7 @@ namespace DB::S3 { PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( - std::function per_request_configuration_, + std::function per_request_configuration_, const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, @@ -97,7 +94,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, - std::function error_report_) + std::function error_report_) : per_request_configuration(per_request_configuration_) , force_region(force_region_) , remote_host_filter(remote_host_filter_) @@ -110,8 +107,6 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_) , error_report(error_report_) { - /// This is used to identify configurations created by us. - userAgent = std::string(VERSION_FULL) + VERSION_OFFICIAL; } void PocoHTTPClientConfiguration::updateSchemeAndRegion() @@ -171,17 +166,6 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config { } -PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration) - : timeouts(ConnectionTimeouts() - .withConnectionTimeout(Poco::Timespan(client_configuration.connectTimeoutMs * 1000)) - .withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) - .withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) - .withTCPKeepAliveTimeout(Poco::Timespan( - client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0))), - remote_host_filter(Context::getGlobalContextInstance()->getRemoteHostFilter()) -{ -} - std::shared_ptr PocoHTTPClient::MakeRequest( const std::shared_ptr & request, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 18a21649167..88251b964e2 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -38,7 +38,7 @@ class PocoHTTPClient; struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration { - std::function per_request_configuration; + std::function per_request_configuration; String force_region; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; @@ -54,13 +54,13 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration size_t http_keep_alive_timeout = DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT; size_t http_keep_alive_max_requests = DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST; - std::function error_report; + std::function error_report; void updateSchemeAndRegion(); private: PocoHTTPClientConfiguration( - std::function per_request_configuration_, + std::function per_request_configuration_, const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, @@ -70,7 +70,8 @@ private: bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, - std::function error_report_); + std::function error_report_ + ); /// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization. friend ClientFactory; @@ -119,7 +120,6 @@ class PocoHTTPClient : public Aws::Http::HttpClient { public: explicit PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration); - explicit PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration); ~PocoHTTPClient() override = default; std::shared_ptr MakeRequest( @@ -166,8 +166,8 @@ protected: static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; - std::function per_request_configuration; - std::function error_report; + std::function per_request_configuration; + std::function error_report; ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; diff --git a/src/IO/S3/PocoHTTPClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp index abec907778c..ef7af2d01ba 100644 --- a/src/IO/S3/PocoHTTPClientFactory.cpp +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -15,10 +15,7 @@ namespace DB::S3 std::shared_ptr PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & client_configuration) const { - if (client_configuration.userAgent.starts_with("ClickHouse")) - return std::make_shared(static_cast(client_configuration)); - else /// This client is created inside the AWS SDK with default settings to obtain ECS credentials from localhost. - return std::make_shared(client_configuration); + return std::make_shared(static_cast(client_configuration)); } std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest( diff --git a/tests/queries/0_stateless/03170_ecs_crash.reference b/tests/queries/0_stateless/03170_ecs_crash.reference deleted file mode 100644 index acd7c60768b..00000000000 --- a/tests/queries/0_stateless/03170_ecs_crash.reference +++ /dev/null @@ -1,4 +0,0 @@ -1 2 3 -4 5 6 -7 8 9 -0 0 0 diff --git a/tests/queries/0_stateless/03170_ecs_crash.sh b/tests/queries/0_stateless/03170_ecs_crash.sh deleted file mode 100755 index fa6870c4cf2..00000000000 --- a/tests/queries/0_stateless/03170_ecs_crash.sh +++ /dev/null @@ -1,9 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -# Previous versions crashed in attempt to use this authentication method (regardless of whether it was able to authenticate): -AWS_CONTAINER_CREDENTIALS_FULL_URI=http://localhost:1338/latest/meta-data/container/security-credentials $CLICKHOUSE_LOCAL -q "select * from s3('http://localhost:11111/test/a.tsv')" From fb110827f82746964fc91ef73d45719244bbcad8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 14 Jun 2024 19:16:37 +0200 Subject: [PATCH 788/856] 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 789/856] 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 579e94319c8e93e1e58dac1942691be195ca97a3 Mon Sep 17 00:00:00 2001 From: haohang Date: Tue, 18 Jun 2024 10:58:44 +0800 Subject: [PATCH 790/856] [docs] pin the gimli-rs/addr2line version in Update allocation-profiling.md after upstream changed the folder, the build command is broken: https://github.com/gimli-rs/addr2line/pull/291/files#diff-2e9d962a08321605940b5a657135052fbcef87b5e360662bb527c96d9a615542R70-R72 for a more stable way, I intend to pin the version --- docs/en/operations/allocation-profiling.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/allocation-profiling.md b/docs/en/operations/allocation-profiling.md index 64b4106a7e1..574e1ae2ff3 100644 --- a/docs/en/operations/allocation-profiling.md +++ b/docs/en/operations/allocation-profiling.md @@ -59,10 +59,10 @@ For that, we need to use `jemalloc`'s tool called [jeprof](https://github.com/je If that’s the case, we recommend installing an [alternative implementation](https://github.com/gimli-rs/addr2line) of the tool. ``` -git clone https://github.com/gimli-rs/addr2line +git clone https://github.com/gimli-rs/addr2line.git --depth=1 --branch=0.23.0 cd addr2line -cargo b --examples -r -cp ./target/release/examples/addr2line path/to/current/addr2line +cargo build --features bin --release +cp ./target/release/addr2line path/to/current/addr2line ``` ::: From f2a162a4a784af00fc788084eab8bf6763f06f73 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 18 Jun 2024 11:09:13 +0800 Subject: [PATCH 791/856] 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 792/856] 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 793/856] 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 794/856] 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 795/856] 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 796/856] 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 797/856] 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 798/856] 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 815/856] 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 816/856] 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 817/856] 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 818/856] 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 819/856] 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 820/856] 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 821/856] 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 822/856] 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 823/856] 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 824/856] 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 825/856] 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 11e844c953380bbfda3b61dcd2dfda60b809f5e2 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 18 Jun 2024 19:12:58 +0000 Subject: [PATCH 826/856] 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 827/856] 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 828/856] 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 829/856] 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 830/856] 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 831/856] 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 832/856] 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 833/856] 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 834/856] 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 835/856] 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 84d282c5738414b18b6f5a7cc269c018b75f3143 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 19 Jun 2024 17:29:24 +0200 Subject: [PATCH 836/856] 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 837/856] 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 838/856] 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 839/856] 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 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 840/856] 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 841/856] 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 842/856] 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 843/856] 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 9c66375e6849bf2b427900018bb5ae553f4f5ae9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 19 Jun 2024 17:54:40 +0300 Subject: [PATCH 844/856] 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 845/856] 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 846/856] 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 fbd9fc9680a2b7e413e2879100f345ff8b458789 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 20 Jun 2024 15:18:22 +0100 Subject: [PATCH 847/856] 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 848/856] 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 849/856] 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 400596756effd74ab50a1237f083c8c45c5dae4c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 20 Jun 2024 17:09:08 +0200 Subject: [PATCH 850/856] 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 6277d97105db7b3c529ba28685c916d1c9e28655 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 20 Jun 2024 15:35:09 +0000 Subject: [PATCH 851/856] 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 5982aa3d19f74ec16fd7156495baf64716045bcb Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 20 Jun 2024 15:40:50 +0000 Subject: [PATCH 852/856] 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 853/856] 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 3b7d69a2fe49bd9eee8ff763d8ee90f0c26667b1 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 21 Jun 2024 00:35:52 +0200 Subject: [PATCH 854/856] 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 f0ae3a7c9f76abeae6f100f0a1bf2bedc060f96c Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 21 Jun 2024 15:05:31 +0200 Subject: [PATCH 855/856] 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 856/856] 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