From e51bee58ecf8f684abfb0f1f9ac1806fb47b5efd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Apr 2022 22:32:45 +0200 Subject: [PATCH 001/813] Remove useless "install" from CMake (step 1) --- CMakeLists.txt | 1 - base/glibc-compatibility/CMakeLists.txt | 6 ---- base/harmful/CMakeLists.txt | 1 - tests/CMakeLists.txt | 26 --------------- tests/integration/CMakeLists.txt | 24 -------------- utils/CMakeLists.txt | 5 --- utils/config-processor/CMakeLists.txt | 2 -- utils/config-processor/config-processor.cpp | 35 --------------------- utils/report/CMakeLists.txt | 1 - 9 files changed, 101 deletions(-) delete mode 100644 tests/CMakeLists.txt delete mode 100644 tests/integration/CMakeLists.txt delete mode 100644 utils/config-processor/CMakeLists.txt delete mode 100644 utils/config-processor/config-processor.cpp delete mode 100644 utils/report/CMakeLists.txt diff --git a/CMakeLists.txt b/CMakeLists.txt index e8b6e9217d2..bffdd810686 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -577,7 +577,6 @@ include (cmake/print_flags.cmake) add_subdirectory (base) add_subdirectory (src) add_subdirectory (programs) -add_subdirectory (tests) add_subdirectory (utils) include (cmake/sanitize_target_link_libraries.cmake) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index ef7ec6d7fc0..37423bb68a6 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -43,12 +43,6 @@ if (GLIBC_COMPATIBILITY) target_link_libraries(global-libs INTERFACE glibc-compatibility ${MEMCPY_LIBRARY}) - install( - TARGETS glibc-compatibility ${MEMCPY_LIBRARY} - EXPORT global - ARCHIVE DESTINATION lib - ) - message (STATUS "Some symbols from glibc will be replaced for compatibility") elseif (CLICKHOUSE_OFFICIAL_BUILD) diff --git a/base/harmful/CMakeLists.txt b/base/harmful/CMakeLists.txt index 399f6ecc625..c19661875be 100644 --- a/base/harmful/CMakeLists.txt +++ b/base/harmful/CMakeLists.txt @@ -1,2 +1 @@ add_library(harmful harmful.c) -install(TARGETS harmful EXPORT global ARCHIVE DESTINATION lib) diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt deleted file mode 100644 index 22c89aaafa7..00000000000 --- a/tests/CMakeLists.txt +++ /dev/null @@ -1,26 +0,0 @@ -enable_testing() - -# Run tests with "ninja check" or "make check" -if (TARGET check) - message (STATUS "Target check already exists") -else () - include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake) -endif () - -option (ENABLE_CLICKHOUSE_TEST "Install clickhouse-test script and relevant tests scenarios" OFF) - -if (ENABLE_CLICKHOUSE_TEST) - install (PROGRAMS clickhouse-test DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - install ( - DIRECTORY queries performance config - DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test - USE_SOURCE_PERMISSIONS - COMPONENT clickhouse - PATTERN "CMakeLists.txt" EXCLUDE - PATTERN ".gitignore" EXCLUDE - ) -endif () - -if (ENABLE_TEST_INTEGRATION) - add_subdirectory (integration) -endif () diff --git a/tests/integration/CMakeLists.txt b/tests/integration/CMakeLists.txt deleted file mode 100644 index 68c695f57a0..00000000000 --- a/tests/integration/CMakeLists.txt +++ /dev/null @@ -1,24 +0,0 @@ -if(CLICKHOUSE_SPLIT_BINARY) - set (TEST_USE_BINARIES CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/programs/clickhouse-server CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/programs/clickhouse-client) -else() - set (TEST_USE_BINARIES CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/programs/clickhouse CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/programs/clickhouse) -endif() - -find_program(DOCKER_CMD docker) -find_program(DOCKER_COMPOSE_CMD docker-compose) -find_program(PYTEST_CMD pytest) -find_program(SUDO_CMD sudo) - -# will mount only one binary to docker container - build with .so cant work -if(USE_STATIC_LIBRARIES AND DOCKER_CMD) - if(INTEGRATION_USE_RUNNER AND SUDO_CMD) - add_test(NAME integration-runner WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND ${SUDO_CMD} ${CMAKE_CURRENT_SOURCE_DIR}/runner --binary ${ClickHouse_BINARY_DIR}/programs/clickhouse --configs-dir ${ClickHouse_SOURCE_DIR}/programs/server/) - message(STATUS "Using tests in docker with runner SUDO=${SUDO_CMD}; DOCKER=${DOCKER_CMD};") - endif() - if(NOT INTEGRATION_USE_RUNNER AND DOCKER_COMPOSE_CMD AND PYTEST_CMD) - # To run one test with debug: - # cmake . -DPYTEST_OPT="-ss;test_cluster_copier" - add_test(NAME integration-pytest WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/programs/server/" "CLICKHOUSE_TESTS_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/tests/config/" ${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}) - message(STATUS "Using tests in docker DOCKER=${DOCKER_CMD}; DOCKER_COMPOSE=${DOCKER_COMPOSE_CMD}; PYTEST=${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}") - endif() -endif() diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 51300472ed1..d4f22d8065d 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -9,11 +9,6 @@ else() endif() include(../cmake/limit_jobs.cmake) -# Utils used in package -add_subdirectory (config-processor) -add_subdirectory (report) - -# Not used in package if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (compressor) add_subdirectory (iotest) diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt deleted file mode 100644 index 76c10b5f2fd..00000000000 --- a/utils/config-processor/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) diff --git a/utils/config-processor/config-processor.cpp b/utils/config-processor/config-processor.cpp deleted file mode 100644 index 242a6782b3b..00000000000 --- a/utils/config-processor/config-processor.cpp +++ /dev/null @@ -1,35 +0,0 @@ -#include -#include - -int main(int argc, char ** argv) -{ - try - { - if (argc != 2) - { - std::cerr << "usage: " << argv[0] << " path" << std::endl; - return 3; - } - - DB::ConfigProcessor processor(argv[1], false, true); - DB::XMLDocumentPtr document = processor.processConfig(); - Poco::XML::DOMWriter().writeNode(std::cout, document); - } - catch (Poco::Exception & e) - { - std::cerr << "Exception: " << e.displayText() << std::endl; - return 1; - } - catch (std::exception & e) - { - std::cerr << "std::exception: " << e.what() << std::endl; - return 3; - } - catch (...) - { - std::cerr << "Some exception" << std::endl; - return 2; - } - - return 0; -} diff --git a/utils/report/CMakeLists.txt b/utils/report/CMakeLists.txt deleted file mode 100644 index e39dd155b15..00000000000 --- a/utils/report/CMakeLists.txt +++ /dev/null @@ -1 +0,0 @@ -install (PROGRAMS clickhouse-report DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) From f40b2b0ffbb7c7b7dd7f45a665d036395ae624fa Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 27 Jul 2022 01:26:40 +0200 Subject: [PATCH 002/813] WIP: window function parallel execution --- src/Processors/QueryPlan/WindowStep.cpp | 17 ++- .../ScatterByPartitionTransform.cpp | 127 ++++++++++++++++++ .../Transforms/ScatterByPartitionTransform.h | 34 +++++ 3 files changed, 177 insertions(+), 1 deletion(-) create mode 100644 src/Processors/Transforms/ScatterByPartitionTransform.cpp create mode 100644 src/Processors/Transforms/ScatterByPartitionTransform.h diff --git a/src/Processors/QueryPlan/WindowStep.cpp b/src/Processors/QueryPlan/WindowStep.cpp index b67b394b57b..67edf13997e 100644 --- a/src/Processors/QueryPlan/WindowStep.cpp +++ b/src/Processors/QueryPlan/WindowStep.cpp @@ -7,6 +7,11 @@ #include #include +#include "Columns/ColumnConst.h" +#include "DataTypes/DataTypesNumber.h" +#include "Functions/FunctionFactory.h" +#include "Processors/Transforms/ScatterByPartitionTransform.h" + namespace DB { @@ -64,7 +69,17 @@ void WindowStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ // This resize is needed for cases such as `over ()` when we don't have a // sort node, and the input might have multiple streams. The sort node would // have resized it. - pipeline.resize(1); + Block header = pipeline.getHeader(); + if (!window_description.partition_by.empty()) + { + ColumnNumbers key_columns; + key_columns.reserve(window_description.partition_by.size()); + for (auto & col : window_description.partition_by) + { + key_columns.push_back(header.getPositionByName(col.column_name)); + } + pipeline.addTransform(std::make_shared(header, pipeline.getNumThreads(), std::move(key_columns))); + } pipeline.addSimpleTransform( [&](const Block & /*header*/) diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.cpp b/src/Processors/Transforms/ScatterByPartitionTransform.cpp new file mode 100644 index 00000000000..ec2add1b9d0 --- /dev/null +++ b/src/Processors/Transforms/ScatterByPartitionTransform.cpp @@ -0,0 +1,127 @@ +#include +#include +#include + +#include "Common/PODArray.h" +#include "Core/ColumnNumbers.h" + +namespace DB +{ +ScatterByPartitionTransform::ScatterByPartitionTransform(Block header, size_t output_size_, ColumnNumbers key_columns_) + : IProcessor(InputPorts{header}, OutputPorts{output_size_, header}) + , output_size(output_size_) + , key_columns(std::move(key_columns_)) + , hash(0) +{} + +IProcessor::Status ScatterByPartitionTransform::prepare() +{ + auto & input = getInputs().front(); + + /// Check all outputs are finished or ready to get data. + + bool all_finished = true; + for (auto & output : outputs) + { + if (output.isFinished()) + continue; + + all_finished = false; + } + + if (all_finished) + { + input.close(); + return Status::Finished; + } + + if (!all_outputs_processed) + return Status::Ready; + + /// Try get chunk from input. + + 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); + + return Status::Ready; +} + +void ScatterByPartitionTransform::work() +{ + if (all_outputs_processed) + generateOutputChunks(); + all_outputs_processed = true; + + size_t chunk_number = 0; + for (auto & output : outputs) + { + auto & was_processed = was_output_processed[chunk_number]; + auto & output_chunk = output_chunks[chunk_number]; + ++chunk_number; + + if (was_processed) + continue; + + if (output.isFinished()) + continue; + + if (!output.canPush()) + { + all_outputs_processed = false; + continue; + } + + output.push(std::move(output_chunk)); + was_processed = true; + } + + if (all_outputs_processed) + { + has_data = false; + output_chunks.clear(); + } +} + +void ScatterByPartitionTransform::generateOutputChunks() +{ + auto num_rows = chunk.getNumRows(); + const auto & columns = chunk.getColumns(); + + hash.reset(num_rows); + + for (const auto & column_number : key_columns) + columns[column_number]->updateWeakHash32(hash); + + const auto & hash_data = hash.getData(); + IColumn::Selector selector(num_rows); + + for (size_t row = 0; row < num_rows; ++row) + { + selector[row] = hash_data[row]; /// [0, 2^32) + selector[row] *= output_size; /// [0, output_size * 2^32), selector stores 64 bit values. + selector[row] >>= 32u; /// [0, output_size) + } + + output_chunks.resize(output_size); + for (const auto & column : columns) + { + auto filtered_columns = column->scatter(output_size, selector); + for (size_t i = 0; i < output_size; ++i) + output_chunks[i].addColumn(std::move(filtered_columns[i])); + } +} + +} diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.h b/src/Processors/Transforms/ScatterByPartitionTransform.h new file mode 100644 index 00000000000..78a86d4a1db --- /dev/null +++ b/src/Processors/Transforms/ScatterByPartitionTransform.h @@ -0,0 +1,34 @@ +#pragma once +#include "Common/WeakHash.h" +#include "Core/ColumnNumbers.h" +#include "Processors/IProcessor.h" + +namespace DB +{ + +struct ScatterByPartitionTransform : IProcessor +{ + ScatterByPartitionTransform(Block header, size_t output_size_, ColumnNumbers key_columns_); + + String getName() const override { return "ScatterByPartitionTransform"; } + + Status prepare() override; + void work() override; + +private: + + void generateOutputChunks(); + + size_t output_size; + ColumnNumbers key_columns; + + bool has_data = false; + bool all_outputs_processed = true; + std::vector was_output_processed; + Chunk chunk; + + WeakHash32 hash; + Chunks output_chunks; +}; + +} From 64d71b0ce0c8dbb0d4907f8728896f8f98c7fea1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 29 Jul 2022 19:44:10 +0200 Subject: [PATCH 003/813] Do scatter before sort --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Processors/QueryPlan/SortingStep.cpp | 61 ++++++++++++++++++- src/Processors/QueryPlan/SortingStep.h | 17 ++++++ src/Processors/QueryPlan/WindowStep.cpp | 13 +--- .../ScatterByPartitionTransform.cpp | 16 +++-- .../Transforms/ScatterByPartitionTransform.h | 6 +- 6 files changed, 94 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index de01115abec..cc5fc5bd1bb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2561,6 +2561,7 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan) auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), window.full_sort_description, + window.partition_by, settings.max_block_size, 0 /* LIMIT */, SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 46588ada225..2760c04dc19 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -9,6 +10,9 @@ #include #include +#include "Processors/ResizeProcessor.h" +#include "Processors/Transforms/ScatterByPartitionTransform.h" + namespace DB { @@ -55,6 +59,15 @@ SortingStep::SortingStep( output_stream->sort_mode = DataStream::SortMode::Stream; } +SortingStep::SortingStep(const DataStream& input_stream, const SortDescription& description_, + const SortDescription& partition_by_description_, size_t max_block_size_, UInt64 limit_, SizeLimits size_limits_, + size_t max_bytes_before_remerge_, double remerge_lowered_memory_bytes_ratio_, + size_t max_bytes_before_external_sort_, VolumePtr tmp_volume_, size_t min_free_disk_space_) + : SortingStep(input_stream, description_, max_block_size_, limit_, size_limits_, max_bytes_before_remerge_, remerge_lowered_memory_bytes_ratio_, max_bytes_before_external_sort_, tmp_volume_, min_free_disk_space_) +{ + partition_by_description = partition_by_description_; +} + SortingStep::SortingStep( const DataStream & input_stream_, SortDescription prefix_description_, @@ -160,6 +173,52 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build } else if (type == Type::Full) { + size_t threads = pipeline.getNumThreads(); + size_t streams = pipeline.getNumStreams(); + + if (!partition_by_description.empty() && threads > 1) + { + Block stream_header = pipeline.getHeader(); + + ColumnNumbers key_columns; + key_columns.reserve(partition_by_description.size()); + for (auto & col : partition_by_description) + { + key_columns.push_back(stream_header.getPositionByName(col.column_name)); + } + + pipeline.transform([&](OutputPortRawPtrs ports) + { + Processors processors; + for (auto * port : ports) + { + auto scatter = std::make_shared(stream_header, threads, key_columns); + connect(*port, scatter->getInputs().front()); + processors.push_back(scatter); + } + return processors; + }); + + if (streams > 1) + { + pipeline.transform([&](OutputPortRawPtrs ports) + { + Processors processors; + for (size_t i = 0; i < threads; ++i) + { + size_t output_it = i; + auto resize = std::make_shared(ports[output_it]->getHeader(), streams, 1); + auto & inputs = resize->getInputs(); + + for (auto input_it = inputs.begin(); input_it != inputs.end(); output_it += threads, ++input_it) + connect(*ports[output_it], *input_it); + processors.push_back(resize); + } + return processors; + }); + } + } + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) @@ -206,7 +265,7 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build }); /// If there are several streams, then we merge them into one - if (pipeline.getNumStreams() > 1) + if (pipeline.getNumStreams() > 1 && partition_by_description.empty()) { auto transform = std::make_shared( pipeline.getHeader(), diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index ce78bb863bf..b67fd748c1f 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -24,6 +24,20 @@ public: VolumePtr tmp_volume_, size_t min_free_disk_space_); + /// Full with partitioning + SortingStep( + const DataStream & input_stream, + const SortDescription & description_, + const SortDescription & partition_by_description_, + size_t max_block_size_, + UInt64 limit_, + SizeLimits size_limits_, + size_t max_bytes_before_remerge_, + double remerge_lowered_memory_bytes_ratio_, + size_t max_bytes_before_external_sort_, + VolumePtr tmp_volume_, + size_t min_free_disk_space_); + /// FinishSorting SortingStep( const DataStream & input_stream_, @@ -67,6 +81,9 @@ private: SortDescription prefix_description; SortDescription result_description; + + SortDescription partition_by_description; + size_t max_block_size; UInt64 limit; SizeLimits size_limits; diff --git a/src/Processors/QueryPlan/WindowStep.cpp b/src/Processors/QueryPlan/WindowStep.cpp index 67edf13997e..545bb5f4f79 100644 --- a/src/Processors/QueryPlan/WindowStep.cpp +++ b/src/Processors/QueryPlan/WindowStep.cpp @@ -69,17 +69,8 @@ void WindowStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ // This resize is needed for cases such as `over ()` when we don't have a // sort node, and the input might have multiple streams. The sort node would // have resized it. - Block header = pipeline.getHeader(); - if (!window_description.partition_by.empty()) - { - ColumnNumbers key_columns; - key_columns.reserve(window_description.partition_by.size()); - for (auto & col : window_description.partition_by) - { - key_columns.push_back(header.getPositionByName(col.column_name)); - } - pipeline.addTransform(std::make_shared(header, pipeline.getNumThreads(), std::move(key_columns))); - } + if (window_description.full_sort_description.empty()) + pipeline.resize(1); pipeline.addSimpleTransform( [&](const Block & /*header*/) diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.cpp b/src/Processors/Transforms/ScatterByPartitionTransform.cpp index ec2add1b9d0..336371dae8b 100644 --- a/src/Processors/Transforms/ScatterByPartitionTransform.cpp +++ b/src/Processors/Transforms/ScatterByPartitionTransform.cpp @@ -1,9 +1,7 @@ -#include -#include #include -#include "Common/PODArray.h" -#include "Core/ColumnNumbers.h" +#include +#include namespace DB { @@ -36,8 +34,16 @@ IProcessor::Status ScatterByPartitionTransform::prepare() } if (!all_outputs_processed) + { + auto output_it = outputs.begin(); + bool can_push = false; + for (size_t i = 0; i < output_size; ++i, ++output_it) + if (!was_output_processed[i] && output_it->canPush()) + can_push = true; + if (!can_push) + return Status::PortFull; return Status::Ready; - + } /// Try get chunk from input. if (input.isFinished()) diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.h b/src/Processors/Transforms/ScatterByPartitionTransform.h index 78a86d4a1db..327f6dd62b4 100644 --- a/src/Processors/Transforms/ScatterByPartitionTransform.h +++ b/src/Processors/Transforms/ScatterByPartitionTransform.h @@ -1,7 +1,7 @@ #pragma once -#include "Common/WeakHash.h" -#include "Core/ColumnNumbers.h" -#include "Processors/IProcessor.h" +#include +#include +#include namespace DB { From cd96bcad98f89b77b39bb7170f6ac4b27ebccb51 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 1 Aug 2022 16:33:06 +0200 Subject: [PATCH 004/813] Produce several outputs in FinishSort --- src/Processors/QueryPlan/SortingStep.cpp | 99 +++++++++++++----------- src/Processors/QueryPlan/SortingStep.h | 1 + 2 files changed, 54 insertions(+), 46 deletions(-) diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 2760c04dc19..d0170026f0f 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -124,12 +124,61 @@ void SortingStep::convertToFinishSorting(SortDescription prefix_description_) prefix_description = std::move(prefix_description_); } +void SortingStep::scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline) +{ + size_t threads = pipeline.getNumThreads(); + size_t streams = pipeline.getNumStreams(); + + if (!partition_by_description.empty() && threads > 1) + { + Block stream_header = pipeline.getHeader(); + + ColumnNumbers key_columns; + key_columns.reserve(partition_by_description.size()); + for (auto & col : partition_by_description) + { + key_columns.push_back(stream_header.getPositionByName(col.column_name)); + } + + pipeline.transform([&](OutputPortRawPtrs ports) + { + Processors processors; + for (auto * port : ports) + { + auto scatter = std::make_shared(stream_header, threads, key_columns); + connect(*port, scatter->getInputs().front()); + processors.push_back(scatter); + } + return processors; + }); + + if (streams > 1) + { + pipeline.transform([&](OutputPortRawPtrs ports) + { + Processors processors; + for (size_t i = 0; i < threads; ++i) + { + size_t output_it = i; + auto resize = std::make_shared(ports[output_it]->getHeader(), streams, 1); + auto & inputs = resize->getInputs(); + + for (auto input_it = inputs.begin(); input_it != inputs.end(); output_it += threads, ++input_it) + connect(*ports[output_it], *input_it); + processors.push_back(resize); + } + return processors; + }); + } + } +} + void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { if (type == Type::FinishSorting) { bool need_finish_sorting = (prefix_description.size() < result_description.size()); - if (pipeline.getNumStreams() > 1) + if (pipeline.getNumStreams() > 1 && partition_by_description.empty()) { UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); auto transform = std::make_shared( @@ -143,6 +192,8 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.addTransform(std::move(transform)); } + scatterByPartitionIfNeeded(pipeline); + if (need_finish_sorting) { pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -173,51 +224,7 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build } else if (type == Type::Full) { - size_t threads = pipeline.getNumThreads(); - size_t streams = pipeline.getNumStreams(); - - if (!partition_by_description.empty() && threads > 1) - { - Block stream_header = pipeline.getHeader(); - - ColumnNumbers key_columns; - key_columns.reserve(partition_by_description.size()); - for (auto & col : partition_by_description) - { - key_columns.push_back(stream_header.getPositionByName(col.column_name)); - } - - pipeline.transform([&](OutputPortRawPtrs ports) - { - Processors processors; - for (auto * port : ports) - { - auto scatter = std::make_shared(stream_header, threads, key_columns); - connect(*port, scatter->getInputs().front()); - processors.push_back(scatter); - } - return processors; - }); - - if (streams > 1) - { - pipeline.transform([&](OutputPortRawPtrs ports) - { - Processors processors; - for (size_t i = 0; i < threads; ++i) - { - size_t output_it = i; - auto resize = std::make_shared(ports[output_it]->getHeader(), streams, 1); - auto & inputs = resize->getInputs(); - - for (auto input_it = inputs.begin(); input_it != inputs.end(); output_it += threads, ++input_it) - connect(*ports[output_it], *input_it); - processors.push_back(resize); - } - return processors; - }); - } - } + scatterByPartitionIfNeeded(pipeline); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index b67fd748c1f..e3d41f921d7 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -68,6 +68,7 @@ public: void convertToFinishSorting(SortDescription prefix_description); private: + void scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline); void updateOutputStream() override; enum class Type From b680b1ac1ed8623e5dc57707a5fec74726f0c9fb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Jun 2023 23:58:49 +0000 Subject: [PATCH 005/813] propagate insertion values storage into subquery --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 36 +++++++++---------- src/Analyzer/Passes/QueryAnalysisPass.h | 4 ++- src/Analyzer/QueryTreePassManager.cpp | 4 +-- src/Analyzer/QueryTreePassManager.h | 3 +- .../InterpreterSelectQueryAnalyzer.cpp | 5 ++- 5 files changed, 29 insertions(+), 23 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index aa915e48d35..4ef01524ac8 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1056,7 +1056,7 @@ private: class QueryAnalyzer { public: - void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression, ContextPtr context) + void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression, ContextPtr context, const StoragePtr & storage = nullptr) { IdentifierResolveScope scope(node, nullptr /*parent_scope*/); @@ -1073,7 +1073,7 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "For query analysis table expression must be empty"); - resolveQuery(node, scope); + resolveQuery(node, scope, storage); break; } case QueryTreeNodeType::UNION: @@ -1187,7 +1187,7 @@ private: QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context); - void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope); + void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr); static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope); @@ -1292,9 +1292,9 @@ private: ProjectionNames resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); - ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); + ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage = nullptr); - ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); + ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage = nullptr); ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope); @@ -1302,7 +1302,7 @@ private: void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope); - NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope); + NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr); void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); @@ -1316,7 +1316,7 @@ private: void resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor); - void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope); + void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr); void resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope); @@ -1834,7 +1834,7 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunctions(const st } /// Evaluate scalar subquery and perform constant folding if scalar subquery does not have constant value -void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, IdentifierResolveScope & scope) +void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, IdentifierResolveScope & scope, const StoragePtr & storage) { auto * query_node = node->as(); auto * union_node = node->as(); @@ -1870,7 +1870,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden subquery_context->setSettings(subquery_settings); auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_subquery*/); - auto interpreter = std::make_unique(node->toAST(), subquery_context, options); + auto interpreter = std::make_unique(node->toAST(), subquery_context, storage, options); auto io = interpreter->execute(); @@ -5230,7 +5230,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi * * 4. If node has alias, update its value in scope alias map. Deregister alias from expression_aliases_in_resolve_process. */ -ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) +ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage) { checkStackSize(); @@ -5487,7 +5487,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id resolveUnion(node, subquery_scope); if (!allow_table_expression) - evaluateScalarSubqueryIfNeeded(node, subquery_scope); + evaluateScalarSubqueryIfNeeded(node, subquery_scope, storage); if (result_projection_names.empty()) result_projection_names.push_back(std::move(projection_name)); @@ -5572,7 +5572,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id * Example: CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; SELECT plus(*) FROM test_table; * Example: SELECT *** FROM system.one; */ -ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) +ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage) { auto & node_list_typed = node_list->as(); size_t node_list_size = node_list_typed.getNodes().size(); @@ -5585,7 +5585,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node for (auto & node : node_list_typed.getNodes()) { auto node_to_resolve = node; - auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression); + auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression, storage); size_t expected_projection_names_size = 1; if (auto * expression_list = node_to_resolve->as()) @@ -5772,9 +5772,9 @@ void QueryAnalyzer::resolveWindowNodeList(QueryTreeNodePtr & window_node_list, I resolveWindow(node, scope); } -NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope) +NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope, const StoragePtr & storage) { - ProjectionNames projection_names = resolveExpressionNodeList(projection_node_list, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + ProjectionNames projection_names = resolveExpressionNodeList(projection_node_list, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/, storage); auto projection_nodes = projection_node_list->as().getNodes(); size_t projection_nodes_size = projection_nodes.size(); @@ -6681,7 +6681,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, * 10. Remove aliases from expression and lambda nodes. * 11. Resolve query tree node with projection columns. */ -void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) +void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope, const StoragePtr & storage) { size_t max_subquery_depth = scope.context->getSettingsRef().max_subquery_depth; if (max_subquery_depth && scope.subquery_depth > max_subquery_depth) @@ -6838,7 +6838,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (!scope.group_by_use_nulls) { - projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope); + projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope, storage); if (query_node_typed.getProjection().getNodes().empty()) throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, "Empty list of columns in projection. In scope {}", @@ -7077,7 +7077,7 @@ QueryAnalysisPass::QueryAnalysisPass(QueryTreeNodePtr table_expression_) void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { QueryAnalyzer analyzer; - analyzer.resolve(query_tree_node, table_expression, context); + analyzer.resolve(query_tree_node, table_expression, context, storage); } } diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index fa8778ebf76..e0b9612f9e2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -71,7 +72,7 @@ public: /** Construct query analysis pass for query or union analysis. * Available columns are extracted from query node join tree. */ - QueryAnalysisPass() = default; + explicit QueryAnalysisPass(const StoragePtr & storage_ = nullptr) : storage(storage_) {} /** Construct query analysis pass for expression or list of expressions analysis. * Available expression columns are extracted from table expression. @@ -93,6 +94,7 @@ public: private: QueryTreeNodePtr table_expression; + const StoragePtr storage = nullptr; }; } diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index a6da2a66615..7178c994490 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -238,9 +238,9 @@ void QueryTreePassManager::dump(WriteBuffer & buffer, size_t up_to_pass_index) } } -void addQueryTreePasses(QueryTreePassManager & manager) +void addQueryTreePasses(QueryTreePassManager & manager, const StoragePtr & storage) { - manager.addPass(std::make_unique()); + manager.addPass(std::make_unique(storage)); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Analyzer/QueryTreePassManager.h b/src/Analyzer/QueryTreePassManager.h index 3c67fc36178..478f04440d0 100644 --- a/src/Analyzer/QueryTreePassManager.h +++ b/src/Analyzer/QueryTreePassManager.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB { @@ -44,6 +45,6 @@ private: std::vector passes; }; -void addQueryTreePasses(QueryTreePassManager & manager); +void addQueryTreePasses(QueryTreePassManager & manager, const StoragePtr & storage = nullptr); } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 98f70c25dcd..6f9e28deb10 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -109,6 +109,9 @@ void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr & } } + if (auto * table_node = table_expression_to_replace->as(); table_node && table_node->getStorageID().getFullNameNotQuoted() != storage->getStorageID().getFullTableName()) + return; + auto replacement_table_expression = std::make_shared(storage, context); std::optional table_expression_modifiers; @@ -133,7 +136,7 @@ QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, auto query_tree = buildQueryTree(query, context); QueryTreePassManager query_tree_pass_manager(context); - addQueryTreePasses(query_tree_pass_manager); + addQueryTreePasses(query_tree_pass_manager, storage); if (select_query_options.ignore_ast_optimizations) query_tree_pass_manager.run(query_tree, 1 /*up_to_pass_index*/); From 4efa82abf406056ff53b5205e21bf21e8de8f99c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 14 Jun 2023 09:54:43 -0400 Subject: [PATCH 006/813] remove 01268_mv_scalars --- tests/broken_tests.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index faee1c5b295..38a51194279 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -33,7 +33,6 @@ 01232_extremes 01244_optimize_distributed_group_by_sharding_key 01247_optimize_distributed_group_by_sharding_key_dist_on_dist -01268_mv_scalars 01268_shard_avgweighted 01270_optimize_skip_unused_shards_low_cardinality 01319_optimize_skip_unused_shards_nesting From 3f5682a792699980cfce10b7f6c510ee68d0419d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 14 Jun 2023 21:34:43 +0000 Subject: [PATCH 007/813] use view_source from context --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 36 +++++++++---------- src/Analyzer/Passes/QueryAnalysisPass.h | 4 +-- src/Analyzer/QueryTreePassManager.cpp | 4 +-- src/Analyzer/QueryTreePassManager.h | 3 +- .../InterpreterSelectQueryAnalyzer.cpp | 2 +- 5 files changed, 23 insertions(+), 26 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 4ef01524ac8..6e24125880e 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1056,7 +1056,7 @@ private: class QueryAnalyzer { public: - void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression, ContextPtr context, const StoragePtr & storage = nullptr) + void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression, ContextPtr context) { IdentifierResolveScope scope(node, nullptr /*parent_scope*/); @@ -1073,7 +1073,7 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "For query analysis table expression must be empty"); - resolveQuery(node, scope, storage); + resolveQuery(node, scope); break; } case QueryTreeNodeType::UNION: @@ -1187,7 +1187,7 @@ private: QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context); - void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr); + void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope); static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope); @@ -1292,9 +1292,9 @@ private: ProjectionNames resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); - ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage = nullptr); + ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage = nullptr); + ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope); @@ -1302,7 +1302,7 @@ private: void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope); - NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr); + NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope); void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); @@ -1316,7 +1316,7 @@ private: void resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor); - void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr); + void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope); void resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope); @@ -1834,7 +1834,7 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunctions(const st } /// Evaluate scalar subquery and perform constant folding if scalar subquery does not have constant value -void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, IdentifierResolveScope & scope, const StoragePtr & storage) +void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, IdentifierResolveScope & scope) { auto * query_node = node->as(); auto * union_node = node->as(); @@ -1870,7 +1870,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden subquery_context->setSettings(subquery_settings); auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_subquery*/); - auto interpreter = std::make_unique(node->toAST(), subquery_context, storage, options); + auto interpreter = std::make_unique(node->toAST(), subquery_context, subquery_context->getViewSource(), options); auto io = interpreter->execute(); @@ -5230,7 +5230,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi * * 4. If node has alias, update its value in scope alias map. Deregister alias from expression_aliases_in_resolve_process. */ -ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage) +ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { checkStackSize(); @@ -5487,7 +5487,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id resolveUnion(node, subquery_scope); if (!allow_table_expression) - evaluateScalarSubqueryIfNeeded(node, subquery_scope, storage); + evaluateScalarSubqueryIfNeeded(node, subquery_scope); if (result_projection_names.empty()) result_projection_names.push_back(std::move(projection_name)); @@ -5572,7 +5572,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id * Example: CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; SELECT plus(*) FROM test_table; * Example: SELECT *** FROM system.one; */ -ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage) +ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { auto & node_list_typed = node_list->as(); size_t node_list_size = node_list_typed.getNodes().size(); @@ -5585,7 +5585,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node for (auto & node : node_list_typed.getNodes()) { auto node_to_resolve = node; - auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression, storage); + auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression); size_t expected_projection_names_size = 1; if (auto * expression_list = node_to_resolve->as()) @@ -5772,9 +5772,9 @@ void QueryAnalyzer::resolveWindowNodeList(QueryTreeNodePtr & window_node_list, I resolveWindow(node, scope); } -NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope, const StoragePtr & storage) +NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope) { - ProjectionNames projection_names = resolveExpressionNodeList(projection_node_list, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/, storage); + ProjectionNames projection_names = resolveExpressionNodeList(projection_node_list, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); auto projection_nodes = projection_node_list->as().getNodes(); size_t projection_nodes_size = projection_nodes.size(); @@ -6681,7 +6681,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, * 10. Remove aliases from expression and lambda nodes. * 11. Resolve query tree node with projection columns. */ -void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope, const StoragePtr & storage) +void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) { size_t max_subquery_depth = scope.context->getSettingsRef().max_subquery_depth; if (max_subquery_depth && scope.subquery_depth > max_subquery_depth) @@ -6838,7 +6838,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (!scope.group_by_use_nulls) { - projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope, storage); + projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope); if (query_node_typed.getProjection().getNodes().empty()) throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, "Empty list of columns in projection. In scope {}", @@ -7077,7 +7077,7 @@ QueryAnalysisPass::QueryAnalysisPass(QueryTreeNodePtr table_expression_) void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { QueryAnalyzer analyzer; - analyzer.resolve(query_tree_node, table_expression, context, storage); + analyzer.resolve(query_tree_node, table_expression, context); } } diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index e0b9612f9e2..fa8778ebf76 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB @@ -72,7 +71,7 @@ public: /** Construct query analysis pass for query or union analysis. * Available columns are extracted from query node join tree. */ - explicit QueryAnalysisPass(const StoragePtr & storage_ = nullptr) : storage(storage_) {} + QueryAnalysisPass() = default; /** Construct query analysis pass for expression or list of expressions analysis. * Available expression columns are extracted from table expression. @@ -94,7 +93,6 @@ public: private: QueryTreeNodePtr table_expression; - const StoragePtr storage = nullptr; }; } diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 7178c994490..a6da2a66615 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -238,9 +238,9 @@ void QueryTreePassManager::dump(WriteBuffer & buffer, size_t up_to_pass_index) } } -void addQueryTreePasses(QueryTreePassManager & manager, const StoragePtr & storage) +void addQueryTreePasses(QueryTreePassManager & manager) { - manager.addPass(std::make_unique(storage)); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Analyzer/QueryTreePassManager.h b/src/Analyzer/QueryTreePassManager.h index 478f04440d0..3c67fc36178 100644 --- a/src/Analyzer/QueryTreePassManager.h +++ b/src/Analyzer/QueryTreePassManager.h @@ -3,7 +3,6 @@ #include #include -#include namespace DB { @@ -45,6 +44,6 @@ private: std::vector passes; }; -void addQueryTreePasses(QueryTreePassManager & manager, const StoragePtr & storage = nullptr); +void addQueryTreePasses(QueryTreePassManager & manager); } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 6f9e28deb10..9e675f0f25a 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -136,7 +136,7 @@ QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, auto query_tree = buildQueryTree(query, context); QueryTreePassManager query_tree_pass_manager(context); - addQueryTreePasses(query_tree_pass_manager, storage); + addQueryTreePasses(query_tree_pass_manager); if (select_query_options.ignore_ast_optimizations) query_tree_pass_manager.run(query_tree, 1 /*up_to_pass_index*/); From 8f0aa8781ce37097cef8dd02b2e696b18fd411bb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 19 Jun 2023 02:14:44 +0000 Subject: [PATCH 008/813] replace storage in function 'in' subquery --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 6e24125880e..b7bd606b6cf 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4723,6 +4723,22 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } else { + /// Replace storage with values storage of insertion block + if (auto * query_node = in_second_argument->as()) + { + auto table_expression = extractLeftTableExpression(query_node->getJoinTree()); + if (auto * query_table_node = table_expression->as()) + { + if (StoragePtr storage = scope.context->getViewSource(); storage && query_table_node->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullTableName()) + { + auto replacement_table_expression = std::make_shared(storage, scope.context); + if (std::optional table_expression_modifiers = query_table_node->getTableExpressionModifiers()) + replacement_table_expression->setTableExpressionModifiers(*table_expression_modifiers); + in_second_argument = in_second_argument->cloneAndReplace(table_expression, std::move(replacement_table_expression)); + } + } + } + resolveExpressionNode(in_second_argument, scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/); } } From 16275168cb515cab7b734cd56629f9d8f32961a6 Mon Sep 17 00:00:00 2001 From: root Date: Wed, 23 Aug 2023 16:13:05 +0800 Subject: [PATCH 009/813] [bugfix] possible postgresql logical replication error: wrong type coversion --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 2c97c92ba99..e9ffdebc583 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -263,7 +263,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "attnotnull AS not_null, attndims AS dims, atttypid as type_id, atttypmod as type_modifier " "FROM pg_attribute " "WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) " - "AND NOT attisdropped AND attnum > 0", where); + "AND NOT attisdropped AND attnum > 0 order by attnum asc", where); auto postgres_table_with_schema = postgres_schema.empty() ? postgres_table : doubleQuoteString(postgres_schema) + '.' + doubleQuoteString(postgres_table); table.physical_columns = readNamesAndTypesList(tx, postgres_table_with_schema, query, use_nulls, false); From e1bc6cb0a702cf8f7aed19fa3a78d221ae6c6702 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 5 Sep 2023 15:24:21 -0300 Subject: [PATCH 010/813] increase background_fetches_pool_size to 16, background_schedule_pool_size to 512 --- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- .../ru/operations/server-configuration-parameters/settings.md | 4 ++-- src/Core/ServerSettings.h | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 7d0ab494926..d62a80d60f3 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -74,7 +74,7 @@ The maximum number of threads that will be used for fetching data parts from ano Type: UInt64 -Default: 8 +Default: 16 ## background_merges_mutations_concurrency_ratio @@ -136,7 +136,7 @@ The maximum number of threads that will be used for constantly executing some li Type: UInt64 -Default: 128 +Default: 512 ## backup_threads diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 2c7f0b773e8..742cac639c9 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -993,7 +993,7 @@ ClickHouse использует потоки из глобального пул - Положительное целое число. -Значение по умолчанию: 128. +Значение по умолчанию: 512. ## background_fetches_pool_size {#background_fetches_pool_size} @@ -1003,7 +1003,7 @@ ClickHouse использует потоки из глобального пул - Положительное целое число. -Значение по умолчанию: 8. +Значение по умолчанию: 16. ## background_distributed_schedule_pool_size {#background_distributed_schedule_pool_size} diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 7e346f3596c..ecf6b4aa53e 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -85,10 +85,10 @@ namespace DB M(Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0) \ M(String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0) \ M(UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0) \ - M(UInt64, background_fetches_pool_size, 8, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0) \ + M(UInt64, background_fetches_pool_size, 16, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0) \ M(UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0) \ M(UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0) \ - M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ + M(UInt64, background_schedule_pool_size, 512, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ From 7883ae0d8cf60657a7a225f5d00bbe3655ca8a70 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 22 Sep 2023 13:22:35 +0800 Subject: [PATCH 011/813] Revert "Revert "Avoid excessive calls to getifaddrs in isLocalAddress"" --- src/Common/isLocalAddress.cpp | 43 ++++++++++++++++++++++++++++++++--- 1 file changed, 40 insertions(+), 3 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 7569c6fc14e..902505404a6 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -1,9 +1,14 @@ #include #include +#include #include +#include +#include +#include #include #include +#include #include #include #include @@ -20,7 +25,7 @@ namespace ErrorCodes namespace { -struct NetworkInterfaces +struct NetworkInterfaces : public boost::noncopyable { ifaddrs * ifaddr; NetworkInterfaces() @@ -31,6 +36,13 @@ struct NetworkInterfaces } } + void swap(NetworkInterfaces && other) + { + auto * tmp = ifaddr; + ifaddr = other.ifaddr; + other.ifaddr = tmp; + } + bool hasAddress(const Poco::Net::IPAddress & address) const { ifaddrs * iface; @@ -74,6 +86,32 @@ struct NetworkInterfaces { freeifaddrs(ifaddr); } + + static const NetworkInterfaces & instance() + { + static constexpr int NET_INTERFACE_VALID_PERIOD_MS = 30000; + static NetworkInterfaces nf; + static std::atomic last_updated_time = std::chrono::steady_clock::now(); + static std::shared_mutex nf_mtx; + + auto now = std::chrono::steady_clock::now(); + auto last_updated_time_snapshot = last_updated_time.load(); + + if (std::chrono::duration_cast(now - last_updated_time_snapshot).count() > NET_INTERFACE_VALID_PERIOD_MS) + { + std::unique_lock lock(nf_mtx); + if (last_updated_time.load() != last_updated_time_snapshot) /// it's possible that last_updated_time after we get the snapshot + return nf; + nf.swap(NetworkInterfaces()); + last_updated_time.store(now); + return nf; + } + else + { + std::shared_lock lock(nf_mtx); + return nf; + } + } }; } @@ -111,8 +149,7 @@ bool isLocalAddress(const Poco::Net::IPAddress & address) } } - NetworkInterfaces interfaces; - return interfaces.hasAddress(address); + return NetworkInterfaces::instance().hasAddress(address); } From 3b48b5aa5e7706d27bb3692d27dee7898e5111c9 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 22 Sep 2023 04:57:29 +0000 Subject: [PATCH 012/813] extend lifetime of returned network interface object Signed-off-by: Duc Canh Le --- src/Common/isLocalAddress.cpp | 26 ++++++++++---------------- 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 902505404a6..772e0363904 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -36,13 +36,6 @@ struct NetworkInterfaces : public boost::noncopyable } } - void swap(NetworkInterfaces && other) - { - auto * tmp = ifaddr; - ifaddr = other.ifaddr; - other.ifaddr = tmp; - } - bool hasAddress(const Poco::Net::IPAddress & address) const { ifaddrs * iface; @@ -87,23 +80,24 @@ struct NetworkInterfaces : public boost::noncopyable freeifaddrs(ifaddr); } - static const NetworkInterfaces & instance() + static std::shared_ptr instance() { static constexpr int NET_INTERFACE_VALID_PERIOD_MS = 30000; - static NetworkInterfaces nf; + static std::shared_ptr nf = std::make_shared(); static std::atomic last_updated_time = std::chrono::steady_clock::now(); static std::shared_mutex nf_mtx; auto now = std::chrono::steady_clock::now(); - auto last_updated_time_snapshot = last_updated_time.load(); - if (std::chrono::duration_cast(now - last_updated_time_snapshot).count() > NET_INTERFACE_VALID_PERIOD_MS) + if (std::chrono::duration_cast(now - last_updated_time.load()).count() > NET_INTERFACE_VALID_PERIOD_MS) { std::unique_lock lock(nf_mtx); - if (last_updated_time.load() != last_updated_time_snapshot) /// it's possible that last_updated_time after we get the snapshot - return nf; - nf.swap(NetworkInterfaces()); - last_updated_time.store(now); + /// It's possible that last_updated_time after we get lock + if (std::chrono::duration_cast(now - last_updated_time.load()).count() > NET_INTERFACE_VALID_PERIOD_MS) + { + nf = std::make_shared(); + last_updated_time.store(now); + } return nf; } else @@ -149,7 +143,7 @@ bool isLocalAddress(const Poco::Net::IPAddress & address) } } - return NetworkInterfaces::instance().hasAddress(address); + return NetworkInterfaces::instance()->hasAddress(address); } From 3b0579b33946835dc09d058d568271f66f54b822 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 22 Sep 2023 14:44:40 +0000 Subject: [PATCH 013/813] Fix splitting into buckets --- src/Processors/Transforms/ScatterByPartitionTransform.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.cpp b/src/Processors/Transforms/ScatterByPartitionTransform.cpp index 336371dae8b..6e3cdc0fda1 100644 --- a/src/Processors/Transforms/ScatterByPartitionTransform.cpp +++ b/src/Processors/Transforms/ScatterByPartitionTransform.cpp @@ -115,11 +115,7 @@ void ScatterByPartitionTransform::generateOutputChunks() IColumn::Selector selector(num_rows); for (size_t row = 0; row < num_rows; ++row) - { - selector[row] = hash_data[row]; /// [0, 2^32) - selector[row] *= output_size; /// [0, output_size * 2^32), selector stores 64 bit values. - selector[row] >>= 32u; /// [0, output_size) - } + selector[row] = hash_data[row] % output_size; output_chunks.resize(output_size); for (const auto & column : columns) From c1d84605016564eff3f8e3dc39a97df0f9cecea2 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 22 Sep 2023 15:10:40 +0000 Subject: [PATCH 014/813] Fix queries in the test --- .../0_stateless/01568_window_functions_distributed.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.sql b/tests/queries/0_stateless/01568_window_functions_distributed.sql index 140b02bfa6d..ef8e9541b05 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.sql +++ b/tests/queries/0_stateless/01568_window_functions_distributed.sql @@ -13,11 +13,11 @@ create table t_01568 engine Memory as select intDiv(number, 3) p, modulo(number, 3) o, number from numbers(9); -select sum(number) over w, max(number) over w from t_01568 window w as (partition by p); +select sum(number) over w, max(number) over w from t_01568 window w as (partition by p) order by p; -select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by p; -select distinct sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select distinct sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by p; -- window functions + aggregation w/shards select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3); From b7cfc4d82d8949a174dc3c08390f65d0531c748a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 25 Sep 2023 15:34:00 +0000 Subject: [PATCH 015/813] Support in Planner --- src/Planner/Planner.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 9dab68e3f00..3c2f01f9881 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -813,6 +813,7 @@ void addWindowSteps(QueryPlan & query_plan, auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), window_description.full_sort_description, + window_description.partition_by, 0 /*limit*/, sort_settings, settings.optimize_sorting_by_input_stream_properties); From d4acd9ec61da01e8c5dad97196c6a35e5c794475 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 25 Sep 2023 15:34:35 +0000 Subject: [PATCH 016/813] Set correct stream sorting --- src/Processors/QueryPlan/SortingStep.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 814d818d227..ec32f6f6a28 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -89,6 +89,9 @@ SortingStep::SortingStep( : SortingStep(input_stream, description_, limit_, settings_, optimize_sorting_by_input_stream_properties_) { partition_by_description = partition_by_description_; + + output_stream->sort_description = result_description; + output_stream->sort_scope = DataStream::SortScope::Stream; } SortingStep::SortingStep( @@ -132,7 +135,11 @@ void SortingStep::updateOutputStream() { output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); output_stream->sort_description = result_description; - output_stream->sort_scope = DataStream::SortScope::Global; + + if (partition_by_description.empty()) + output_stream->sort_scope = DataStream::SortScope::Global; + else + output_stream->sort_scope = DataStream::SortScope::Stream; } void SortingStep::updateLimit(size_t limit_) From 32dd65cb22f424e352ee8d41d13577b4d7e5bb6c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 25 Sep 2023 15:35:02 +0000 Subject: [PATCH 017/813] Update reference file --- .../01568_window_functions_distributed.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.reference b/tests/queries/0_stateless/01568_window_functions_distributed.reference index 0b439ef759a..1284624fad1 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.reference +++ b/tests/queries/0_stateless/01568_window_functions_distributed.reference @@ -12,7 +12,7 @@ drop table if exists t_01568; create table t_01568 engine Memory as select intDiv(number, 3) p, modulo(number, 3) o, number from numbers(9); -select sum(number) over w, max(number) over w from t_01568 window w as (partition by p); +select sum(number) over w, max(number) over w from t_01568 window w as (partition by p) order by p; 3 2 3 2 3 2 @@ -22,7 +22,7 @@ select sum(number) over w, max(number) over w from t_01568 window w as (partitio 21 8 21 8 21 8 -select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by p; 6 2 6 2 6 2 @@ -41,7 +41,7 @@ select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t 42 8 42 8 42 8 -select distinct sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select distinct sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by p; 6 2 24 5 42 8 From 15fe392fed647ece1d18fea8d2a6b8197c1a9ae6 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 27 Sep 2023 10:13:23 +0000 Subject: [PATCH 018/813] no update network_interfaces Signed-off-by: Duc Canh Le --- src/Common/isLocalAddress.cpp | 30 ++---------------------------- 1 file changed, 2 insertions(+), 28 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 772e0363904..1e6cfb1c592 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -79,33 +79,6 @@ struct NetworkInterfaces : public boost::noncopyable { freeifaddrs(ifaddr); } - - static std::shared_ptr instance() - { - static constexpr int NET_INTERFACE_VALID_PERIOD_MS = 30000; - static std::shared_ptr nf = std::make_shared(); - static std::atomic last_updated_time = std::chrono::steady_clock::now(); - static std::shared_mutex nf_mtx; - - auto now = std::chrono::steady_clock::now(); - - if (std::chrono::duration_cast(now - last_updated_time.load()).count() > NET_INTERFACE_VALID_PERIOD_MS) - { - std::unique_lock lock(nf_mtx); - /// It's possible that last_updated_time after we get lock - if (std::chrono::duration_cast(now - last_updated_time.load()).count() > NET_INTERFACE_VALID_PERIOD_MS) - { - nf = std::make_shared(); - last_updated_time.store(now); - } - return nf; - } - else - { - std::shared_lock lock(nf_mtx); - return nf; - } - } }; } @@ -143,7 +116,8 @@ bool isLocalAddress(const Poco::Net::IPAddress & address) } } - return NetworkInterfaces::instance()->hasAddress(address); + static NetworkInterfaces network_interfaces; + return network_interfaces.hasAddress(address); } From 2ce527364dd6ee143882b02b2348b89eb4bfcb6c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 27 Sep 2023 18:21:15 +0800 Subject: [PATCH 019/813] remove unnecessary headers --- src/Common/isLocalAddress.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 1e6cfb1c592..86259eb1ba4 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -1,11 +1,7 @@ #include #include -#include #include -#include -#include -#include #include #include #include From a72541d93f1cf46aa0fb577bcb59e5f2845c7a81 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Sep 2023 17:39:38 +0200 Subject: [PATCH 020/813] Improvement for big reads --- src/Common/ProfileEvents.cpp | 1 + src/Core/Settings.h | 1 + .../IO/CachedOnDiskReadBufferFromFile.cpp | 42 ++++++++++++------- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 6 ++- src/IO/ReadSettings.h | 1 + src/Interpreters/Cache/FileCache.cpp | 14 ++++++- src/Interpreters/Cache/FileCache.h | 4 +- src/Interpreters/Cache/FileCache_fwd.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 5 +-- src/Interpreters/Cache/FileSegment.h | 5 +-- src/Interpreters/Context.cpp | 1 + 11 files changed, 54 insertions(+), 28 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0df0bc89b38..033335bd5ec 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -415,6 +415,7 @@ The server successfully detected this situation and will download merged part fr M(FilesystemCacheEvictMicroseconds, "Filesystem cache eviction time") \ M(FilesystemCacheGetOrSetMicroseconds, "Filesystem cache getOrSet() time") \ M(FilesystemCacheGetMicroseconds, "Filesystem cache get() time") \ + M(FilesystemCacheUnusedHoldFileSegments, "Filesystem cache file segments count, which were hold, but not used (because of seek or LIMIT n, etc)") \ M(FileSegmentWaitMicroseconds, "Wait on DOWNLOADING state") \ M(FileSegmentCompleteMicroseconds, "Duration of FileSegment::complete() in filesystem cache") \ M(FileSegmentLockMicroseconds, "Lock file segment time") \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1fada4ae569..416d8f481b1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -717,6 +717,7 @@ class IColumn; M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \ M(UInt64, filesystem_cache_max_download_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be downloaded by a single query", 0) \ M(Bool, throw_on_error_from_cache_on_write_operations, false, "Ignore error from cache when caching on write operations (INSERT, merges)", 0) \ + M(UInt64, filesystem_cache_getorset_batch_size, 100, "A batch size for holding file segments for a single read range", 0) \ \ M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \ M(Bool, enable_filesystem_read_prefetches_log, false, "Log to system.filesystem prefetch_log during query. Should be used only for testing or debugging, not recommended to be turned on by default", 0) \ diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 1cfdd96b271..3c16d3d9ae2 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -114,30 +114,40 @@ void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( cache_log->add(std::move(elem)); } -void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) +bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch() +{ + size_t size = getRemainingSizeToRead(); + if (!size) + return false; + + if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) + { + file_segments = cache->get(cache_key, file_offset_of_buffer_end, size, settings.filesystem_cache_getorset_batch_size); + } + else + { + CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); + file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), settings.filesystem_cache_getorset_batch_size, create_settings); + } + return !file_segments->empty(); +} + +void CachedOnDiskReadBufferFromFile::initialize() { if (initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "Caching buffer already initialized"); implementation_buffer.reset(); - if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) - { - file_segments = cache->get(cache_key, offset, size); - } - else - { - CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); - file_segments = cache->getOrSet(cache_key, offset, size, file_size.value(), create_settings); - } - /** * Segments in returned list are ordered in ascending order and represent a full contiguous * interval (no holes). Each segment in returned list has state: DOWNLOADED, DOWNLOADING or EMPTY. */ - if (file_segments->empty()) + if (!nextFileSegmentsBatch()) throw Exception(ErrorCodes::LOGICAL_ERROR, "List of file segments cannot be empty"); + chassert(!file_segments->empty()); + LOG_TEST( log, "Having {} file segments to read: {}, current offset: {}", @@ -512,7 +522,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() cache_file_reader.reset(); file_segments->popFront(); - if (file_segments->empty()) + if (file_segments->empty() && !nextFileSegmentsBatch()) return false; current_file_segment = &file_segments->front(); @@ -788,9 +798,9 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() return false; if (!initialized) - initialize(file_offset_of_buffer_end, getTotalSizeToRead()); + initialize(); - if (file_segments->empty()) + if (file_segments->empty() && !nextFileSegmentsBatch()) return false; const size_t original_buffer_size = internal_buffer.size(); @@ -1159,7 +1169,7 @@ off_t CachedOnDiskReadBufferFromFile::seek(off_t offset, int whence) return new_pos; } -size_t CachedOnDiskReadBufferFromFile::getTotalSizeToRead() +size_t CachedOnDiskReadBufferFromFile::getRemainingSizeToRead() { /// Last position should be guaranteed to be set, as at least we always know file size. if (!read_until_position) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 0b9b01b8a94..f1eea66d41d 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -63,7 +63,7 @@ public: private: using ImplementationBufferPtr = std::shared_ptr; - void initialize(size_t offset, size_t size); + void initialize(); /** * Return a list of file segments ordered in ascending order. This list represents @@ -85,7 +85,7 @@ private: bool nextImplStep(); - size_t getTotalSizeToRead(); + size_t getRemainingSizeToRead(); bool completeFileSegmentAndGetNext(); @@ -95,6 +95,8 @@ private: static bool canStartFromCache(size_t current_offset, const FileSegment & file_segment); + bool nextFileSegmentsBatch(); + Poco::Logger * log; FileCache::Key cache_key; String source_file_path; diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 87f249823b2..197ae563d25 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -100,6 +100,7 @@ struct ReadSettings bool enable_filesystem_cache_log = false; /// Don't populate cache when the read is not part of query execution (e.g. background thread). bool avoid_readthrough_cache_outside_query_context = true; + size_t filesystem_cache_getorset_batch_size = 100; size_t filesystem_cache_max_download_size = (128UL * 1024 * 1024 * 1024); bool skip_download_if_exceeds_query_cache = true; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 3ed2c9c2dd6..bef1f3086df 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -400,6 +400,7 @@ FileCache::getOrSet( size_t offset, size_t size, size_t file_size, + size_t file_segments_limit, const CreateFileSegmentSettings & settings) { ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheGetOrSetMicroseconds); @@ -432,11 +433,17 @@ FileCache::getOrSet( while (!file_segments.empty() && file_segments.back()->range().left >= offset + size) file_segments.pop_back(); + if (file_segments_limit) + { + while (file_segments.size() > file_segments_limit) + file_segments.pop_back(); + } + chassert(!file_segments.empty()); return std::make_unique(std::move(file_segments)); } -FileSegmentsHolderPtr FileCache::get(const Key & key, size_t offset, size_t size) +FileSegmentsHolderPtr FileCache::get(const Key & key, size_t offset, size_t size, size_t file_segments_limit) { ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheGetMicroseconds); @@ -454,6 +461,11 @@ FileSegmentsHolderPtr FileCache::get(const Key & key, size_t offset, size_t size fillHolesWithEmptyFileSegments( *locked_key, file_segments, range, /* fill_with_detached */true, CreateFileSegmentSettings{}); + if (file_segments_limit) + { + while (file_segments.size() > file_segments_limit) + file_segments.pop_back(); + } return std::make_unique(std::move(file_segments)); } } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index b5c2fa28f4b..1a1a25cd9c1 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -85,7 +85,7 @@ public: * it is guaranteed that these file segments are not removed from cache. */ FileSegmentsHolderPtr - getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings); + getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, size_t file_segments_limit, const CreateFileSegmentSettings & settings); /** * Segments in returned list are ordered in ascending order and represent a full contiguous @@ -96,7 +96,7 @@ public: * with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change * it's state (and become DOWNLOADED). */ - FileSegmentsHolderPtr get(const Key & key, size_t offset, size_t size); + FileSegmentsHolderPtr get(const Key & key, size_t offset, size_t size, size_t file_segments_limit); FileSegmentsHolderPtr set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 3e7150ad253..1f61617668e 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -4,7 +4,7 @@ namespace DB { -static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi +static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 16 * 1024 * 1024; /// 16Mi static constexpr int FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT = 4 * 1024 * 1024; /// 4Mi static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 2; static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 1; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index bb3216cb20e..a351df33b4b 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -23,6 +23,7 @@ namespace ProfileEvents extern const Event FileSegmentWriteMicroseconds; extern const Event FileSegmentUseMicroseconds; extern const Event FileSegmentHolderCompleteMicroseconds; + extern const Event FilesystemCacheUnusedHoldFileSegments; } namespace DB @@ -916,11 +917,9 @@ FileSegments::iterator FileSegmentsHolder::completeAndPopFrontImpl() FileSegmentsHolder::~FileSegmentsHolder() { + ProfileEvents::increment(ProfileEvents::FilesystemCacheUnusedHoldFileSegments, file_segments.size()); ProfileEventTimeIncrement watch(ProfileEvents::FileSegmentHolderCompleteMicroseconds); - if (!complete_on_dtor) - return; - for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end();) file_segment_it = completeAndPopFrontImpl(); } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 8948b67fe2a..7c145664fe0 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -320,8 +320,8 @@ struct FileSegmentsHolder : private boost::noncopyable { FileSegmentsHolder() = default; - explicit FileSegmentsHolder(FileSegments && file_segments_, bool complete_on_dtor_ = true) - : file_segments(std::move(file_segments_)), complete_on_dtor(complete_on_dtor_) {} + explicit FileSegmentsHolder(FileSegments && file_segments_) + : file_segments(std::move(file_segments_)) {} ~FileSegmentsHolder(); @@ -351,7 +351,6 @@ struct FileSegmentsHolder : private boost::noncopyable private: FileSegments file_segments{}; - const bool complete_on_dtor = true; FileSegments::iterator completeAndPopFrontImpl(); }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 58d60c640e7..807215412c2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4591,6 +4591,7 @@ ReadSettings Context::getReadSettings() const res.enable_filesystem_cache = settings.enable_filesystem_cache; res.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; res.enable_filesystem_cache_log = settings.enable_filesystem_cache_log; + res.filesystem_cache_getorset_batch_size = settings.filesystem_cache_getorset_batch_size; res.filesystem_cache_max_download_size = settings.filesystem_cache_max_download_size; res.skip_download_if_exceeds_query_cache = settings.skip_download_if_exceeds_query_cache; From 020d76a383b3a489a3868c3a7c1d770cfaba5549 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 24 May 2023 21:59:46 +0000 Subject: [PATCH 021/813] merge_row_policy: initial --- src/Interpreters/InterpreterSelectQuery.cpp | 14 ++++++++- src/Storages/StorageMerge.cpp | 33 ++++++++++++++++++++- 2 files changed, 45 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e1faa8c8958..6d8028f628b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -93,6 +93,7 @@ #include #include +#include namespace ProfileEvents { @@ -546,7 +547,18 @@ InterpreterSelectQuery::InterpreterSelectQuery( std::shared_ptr table_join = joined_tables.makeTableJoin(query); if (storage) + { + + LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), "table name: {}", table_id.getTableName()); + row_policy_filter = context->getRowPolicyFilter(table_id.getDatabaseName(), table_id.getTableName(), RowPolicyFilterType::SELECT_FILTER); + } + else + { + LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), "no storage"); + } + + StorageView * view = nullptr; if (storage) @@ -832,7 +844,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (query.prewhere() && !query.where()) analysis_result.prewhere_info->need_filter = true; - if (table_id && got_storage_from_query && !joined_tables.isLeftTableFunction()) + if (table_id && got_storage_from_query /* && !joined_tables.isLeftTableFunction() */) { /// The current user should have the SELECT privilege. If this table_id is for a table /// function we don't check access rights here because in this case they have been already diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c14abfc9ab2..e917553d704 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -43,6 +43,8 @@ #include #include +#include + namespace { @@ -515,6 +517,8 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu } } + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), "table name: {}", storage->getStorageID().getTableName()); + auto source_pipeline = createSources( nested_storage_snaphsot, modified_query_info, @@ -574,6 +578,8 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer modified_query_info.table_expression = replacement_table_expression; modified_query_info.planner_context->getOrCreateTableExpressionData(replacement_table_expression); + + auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); if (storage_snapshot->storage.supportsSubcolumns()) get_column_options.withSubcolumns(); @@ -594,6 +600,10 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer } modified_query_info.query = queryNodeToSelectQuery(modified_query_info.query_tree); + TreeRewriterResult new_analyzer_res = *modified_query_info.syntax_analyzer_result; + new_analyzer_res.has_explicit_columns = false; + + modified_query_info.syntax_analyzer_result = std::make_shared(std::move(new_analyzer_res)); } else { @@ -656,8 +666,9 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryPlan & plan = child_plans.emplace_back(); StorageView * view = dynamic_cast(storage.get()); - if (!view || allow_experimental_analyzer) + if (/* !view || */ allow_experimental_analyzer) { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "direct storage->read"); storage->read(plan, real_column_names, storage_snapshot, @@ -667,6 +678,24 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( max_block_size, UInt32(streams_num)); } + else if (!view) + { + /// 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); + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "creating InterpreterSelectQuery 1.0"); + InterpreterSelectQuery interpreter(modified_query_info.query, + modified_context, + storage, + storage->getInMemoryMetadataPtr(), // view->getInMemoryMetadataPtr(), + SelectQueryOptions(/* processed_stage*/)); + interpreter.buildQueryPlan(plan); + } else { /// For view storage, we need to rewrite the `modified_query_info.view_query` to optimize read. @@ -677,6 +706,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( modified_context->setSetting("max_streams_to_max_threads_ratio", 1); modified_context->setSetting("max_block_size", max_block_size); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "creating InterpreterSelectQuery 1.5"); InterpreterSelectQuery interpreter(modified_query_info.query, modified_context, storage, @@ -719,6 +749,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( { modified_select.replaceDatabaseAndTable(database_name, table_name); /// TODO: Find a way to support projections for StorageMerge + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "creating InterpreterSelectQuery 2"); InterpreterSelectQuery interpreter{modified_query_info.query, modified_context, SelectQueryOptions(processed_stage).ignoreProjections()}; From 1e318599360840e0a581cba4ec064f3a907d7746 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 25 May 2023 10:51:45 +0000 Subject: [PATCH 022/813] merge_row_policy: tiny cleanup --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +--- src/Storages/StorageMerge.cpp | 7 ------- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6d8028f628b..2b010228c9a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -548,9 +548,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) { - LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), "table name: {}", table_id.getTableName()); - row_policy_filter = context->getRowPolicyFilter(table_id.getDatabaseName(), table_id.getTableName(), RowPolicyFilterType::SELECT_FILTER); } else @@ -844,7 +842,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (query.prewhere() && !query.where()) analysis_result.prewhere_info->need_filter = true; - if (table_id && got_storage_from_query /* && !joined_tables.isLeftTableFunction() */) + if (table_id && got_storage_from_query && !joined_tables.isLeftTableFunction()) { /// The current user should have the SELECT privilege. If this table_id is for a table /// function we don't check access rights here because in this case they have been already diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e917553d704..19792aeca9f 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -45,7 +45,6 @@ #include - namespace { @@ -578,8 +577,6 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer modified_query_info.table_expression = replacement_table_expression; modified_query_info.planner_context->getOrCreateTableExpressionData(replacement_table_expression); - - auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); if (storage_snapshot->storage.supportsSubcolumns()) get_column_options.withSubcolumns(); @@ -600,10 +597,6 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer } modified_query_info.query = queryNodeToSelectQuery(modified_query_info.query_tree); - TreeRewriterResult new_analyzer_res = *modified_query_info.syntax_analyzer_result; - new_analyzer_res.has_explicit_columns = false; - - modified_query_info.syntax_analyzer_result = std::make_shared(std::move(new_analyzer_res)); } else { From 0464b4fd061b806c2e79f58d114423f7244dbd70 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 27 May 2023 22:23:19 +0000 Subject: [PATCH 023/813] merge_row_policy: with QueryProcessingStage::Complete --- src/Interpreters/InterpreterSelectQuery.cpp | 9 +++-- src/Storages/StorageMerge.cpp | 38 +++++++++++++++++++-- 2 files changed, 42 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2b010228c9a..d2d13bacac8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -100,6 +100,7 @@ namespace ProfileEvents extern const Event SelectQueriesWithSubqueries; extern const Event QueriesWithSubqueries; } +#pragma GCC diagnostic ignored "-Wold-style-cast" namespace DB { @@ -548,12 +549,12 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) { - LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), "table name: {}", table_id.getTableName()); + LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), " {}, table name: {}", (void*)this, table_id.getTableName()); row_policy_filter = context->getRowPolicyFilter(table_id.getDatabaseName(), table_id.getTableName(), RowPolicyFilterType::SELECT_FILTER); } else { - LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), "no storage"); + LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), " {}, no storage", (void*)this); } @@ -1451,7 +1452,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

{}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); + LOG_TRACE(log, "executeImpl {}, {} -> {}", (void*) this, QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); } if (query_info.projection && query_info.projection->input_order_info && query_info.input_order_info) @@ -1523,6 +1524,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

column_name, expressions.filter_info->do_remove_column); + LOG_TRACE(log, "executeImpl, adding Row-level security filter"); + row_level_security_step->setStepDescription("Row-level security filter"); query_plan.addStep(std::move(row_level_security_step)); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 19792aeca9f..6be1ce7db3f 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -45,6 +45,10 @@ #include +#pragma GCC diagnostic ignored "-Wunused-parameter" +#pragma GCC diagnostic ignored "-Wunused-but-set-variable" + + namespace { @@ -264,6 +268,8 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( /// (see removeJoin()) /// /// And for this we need to return FetchColumns. + LOG_TRACE(&Poco::Logger::get("StorageMerge::getQueryProcessingStage"), "to_stage {}", to_stage); + if (const auto * select = query_info.query->as(); select && hasJoin(*select)) return QueryProcessingStage::FetchColumns; @@ -287,13 +293,15 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( stage_in_source_tables, table->getQueryProcessingStage(local_context, to_stage, table->getStorageSnapshot(table->getInMemoryMetadataPtr(), local_context), query_info)); + LOG_TRACE(&Poco::Logger::get("StorageMerge::getQueryProcessingStage"), "stage_in_source_tables {}", stage_in_source_tables); } iterator->next(); } } - return selected_table_size == 1 ? stage_in_source_tables : std::min(stage_in_source_tables, QueryProcessingStage::WithMergeableState); + // return selected_table_size == 1 ? stage_in_source_tables : std::min(stage_in_source_tables, QueryProcessingStage::WithMergeableState); + return QueryProcessingStage::Complete; } void StorageMerge::read( @@ -312,6 +320,9 @@ void StorageMerge::read( auto modified_context = Context::createCopy(local_context); modified_context->setSetting("optimize_move_to_prewhere", false); + LOG_TRACE(&Poco::Logger::get("StorageMerge::read"), "processed_stage {}", QueryProcessingStage::toString(processed_stage)); + + bool has_database_virtual_column = false; bool has_table_virtual_column = false; Names real_column_names; @@ -324,7 +335,10 @@ void StorageMerge::read( else if (column_name == "_table" && isVirtualColumn(column_name, storage_snapshot->metadata)) has_table_virtual_column = true; else + { real_column_names.push_back(column_name); + LOG_TRACE(&Poco::Logger::get("StorageMerge::read"), "column_name {}", column_name); + } } StorageListWithLocks selected_tables @@ -353,7 +367,7 @@ void StorageMerge::read( query_plan.addInterpreterContext(modified_context); /// What will be result structure depending on query processed stage in source tables? - Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage); + Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, QueryProcessingStage::Complete /* processed_stage */); auto step = std::make_unique( common_header, @@ -477,6 +491,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu bool with_aliases = common_processed_stage == QueryProcessingStage::FetchColumns && !storage_columns.getAliases().empty(); if (with_aliases) { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), "with_aliases"); ASTPtr required_columns_expr_list = std::make_shared(); ASTPtr column_expr; @@ -650,6 +665,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage_snapshot, modified_query_info); +#pragma GCC diagnostic ignored "-Wunreachable-code" if (processed_stage <= storage_stage || (allow_experimental_analyzer && processed_stage == QueryProcessingStage::FetchColumns)) { /// If there are only virtual columns in query, you must request at least one other column. @@ -660,6 +676,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( StorageView * view = dynamic_cast(storage.get()); if (/* !view || */ allow_experimental_analyzer) + // if (!view || allow_experimental_analyzer) { LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "direct storage->read"); storage->read(plan, @@ -687,6 +704,8 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage, storage->getInMemoryMetadataPtr(), // view->getInMemoryMetadataPtr(), SelectQueryOptions(/* processed_stage*/)); + // SelectQueryOptions(processed_stage)); + // SelectQueryOptions(QueryProcessingStage::WithMergeableState)); interpreter.buildQueryPlan(plan); } else @@ -807,6 +826,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( /// 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. + convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, *builder, processed_stage); } @@ -1018,6 +1038,20 @@ void ReadFromMerge::convertingSourceStream( if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns) convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; + + for (const auto & column_with_type_and_name : builder.getHeader().getColumnsWithTypeAndName()) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "column name: {} (builder.getHeader().getColumnsWithTypeAndName())", column_with_type_and_name.name); + } + + for (const auto & column_with_type_and_name : header.getColumnsWithTypeAndName()) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "column name: {} (header.getColumnsWithTypeAndName())", column_with_type_and_name.name); + } + + + + auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); From 2acc4c223d9644da82d100163875de413bcea730 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 29 May 2023 11:15:42 +0000 Subject: [PATCH 024/813] merge_row_policy - extra debug --- src/Interpreters/InterpreterSelectQuery.cpp | 2 ++ src/Storages/StorageMerge.cpp | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d2d13bacac8..54b7b2a3137 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -861,6 +861,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Add prewhere actions with alias columns and record needed columns from storage. if (storage) { + LOG_TRACE(log, "calling addPrewhereAliasActions"); + addPrewhereAliasActions(); analysis_result.required_columns = required_columns; } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 6be1ce7db3f..098d0992738 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -732,9 +732,13 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (auto * read_from_merge_tree = typeid_cast(plan.getRootNode()->step.get())) { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "ReadFromMergeTree detected"); size_t filters_dags_size = filter_dags.size(); for (size_t i = 0; i < filters_dags_size; ++i) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "adding filter"); read_from_merge_tree->addFilter(filter_dags[i], filter_nodes.nodes[i]); + } } builder = plan.buildQueryPipeline( From eebdff472e73bb64d04b82edea3cb98cd176291d Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 31 May 2023 11:48:20 +0000 Subject: [PATCH 025/813] merge_row_policy: original behavior restored + extra debug --- src/Interpreters/InterpreterSelectQuery.cpp | 10 +++++++++- src/Storages/StorageMerge.cpp | 13 +++++++------ 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 54b7b2a3137..499dedf2ff5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -134,6 +134,9 @@ FilterDAGInfoPtr generateFilterActions( Names & prerequisite_columns, PreparedSetsPtr prepared_sets) { + LOG_TRACE(&Poco::Logger::get("generateFilterActions"), "top of"); + + auto filter_info = std::make_shared(); const auto & db_name = table_id.getDatabaseName(); @@ -549,7 +552,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) { - LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), " {}, table name: {}", (void*)this, table_id.getTableName()); + LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), " {}, table name: {}, calling getRowPolicyFilter", (void*)this, table_id.getTableName()); row_policy_filter = context->getRowPolicyFilter(table_id.getDatabaseName(), table_id.getTableName(), RowPolicyFilterType::SELECT_FILTER); } else @@ -2065,11 +2068,15 @@ void InterpreterSelectQuery::addPrewhereAliasActions() auto & expressions = analysis_result; if (expressions.filter_info) { + LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info", (void*)this); + if (!expressions.prewhere_info) { + LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 1", (void*)this); const bool does_storage_support_prewhere = !input_pipe && storage && storage->supportsPrewhere(); if (does_storage_support_prewhere && shouldMoveToPrewhere()) { + LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 1.5", (void*)this); /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. expressions.prewhere_info = std::make_shared( std::move(expressions.filter_info->actions), @@ -2082,6 +2089,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() } else { + LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 2", (void*)this); /// Add row level security actions to prewhere. expressions.prewhere_info->row_level_filter = std::move(expressions.filter_info->actions); expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 098d0992738..c0a0ffec7fd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -300,8 +300,8 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( } } - // return selected_table_size == 1 ? stage_in_source_tables : std::min(stage_in_source_tables, QueryProcessingStage::WithMergeableState); - return QueryProcessingStage::Complete; + return selected_table_size == 1 ? stage_in_source_tables : std::min(stage_in_source_tables, QueryProcessingStage::WithMergeableState); + // return QueryProcessingStage::Complete; } void StorageMerge::read( @@ -367,7 +367,8 @@ void StorageMerge::read( query_plan.addInterpreterContext(modified_context); /// What will be result structure depending on query processed stage in source tables? - Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, QueryProcessingStage::Complete /* processed_stage */); + // Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, QueryProcessingStage::Complete /* processed_stage */); + Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage ); auto step = std::make_unique( common_header, @@ -703,8 +704,8 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( modified_context, storage, storage->getInMemoryMetadataPtr(), // view->getInMemoryMetadataPtr(), - SelectQueryOptions(/* processed_stage*/)); - // SelectQueryOptions(processed_stage)); + // SelectQueryOptions(/* processed_stage*/)); + SelectQueryOptions(processed_stage)); // SelectQueryOptions(QueryProcessingStage::WithMergeableState)); interpreter.buildQueryPlan(plan); } @@ -732,8 +733,8 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (auto * read_from_merge_tree = typeid_cast(plan.getRootNode()->step.get())) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "ReadFromMergeTree detected"); size_t filters_dags_size = filter_dags.size(); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "ReadFromMergeTree detected, DAG size {}", filters_dags_size); for (size_t i = 0; i < filters_dags_size; ++i) { LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "adding filter"); From b6f682dc3fe9d9f6ca541d6c06a96711b50b7342 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 2 Jun 2023 23:06:10 +0000 Subject: [PATCH 026/813] merge_row_policy: FilterTransform --- src/Interpreters/InterpreterSelectQuery.cpp | 21 +++- .../MergeTree/MergeTreeSequentialSource.cpp | 2 +- src/Storages/StorageMerge.cpp | 98 ++++++++++++++++++- src/Storages/StorageMerge.h | 4 +- 4 files changed, 117 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 499dedf2ff5..f4931afe89c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -961,6 +961,8 @@ Block InterpreterSelectQuery::getSampleBlockImpl() analysis_result = ExpressionAnalysisResult( *query_analyzer, metadata_snapshot, first_stage, second_stage, options.only_analyze, filter_info, additional_filter_info, source_header); + LOG_TRACE(log, "getSampleBlockImpl {} : source_header after ExpressionAnalysisResult {}", (void*) this, source_header.dumpStructure()); + if (options.to_stage == QueryProcessingStage::Enum::FetchColumns) { @@ -970,8 +972,12 @@ Block InterpreterSelectQuery::getSampleBlockImpl() { header = analysis_result.prewhere_info->prewhere_actions->updateHeader(header); if (analysis_result.prewhere_info->remove_prewhere_column) + { + LOG_TRACE(log, "getSampleBlockImpl {} : erasing column {}", (void*) this, analysis_result.prewhere_info->prewhere_column_name); header.erase(analysis_result.prewhere_info->prewhere_column_name); + } } + LOG_TRACE(log, "getSampleBlockImpl {} : returning header", (void*) this); return header; } @@ -1523,13 +1529,15 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

column_name, query_plan.getCurrentDataStream().header.dumpStructure()); + auto row_level_security_step = std::make_unique( query_plan.getCurrentDataStream(), expressions.filter_info->actions, expressions.filter_info->column_name, expressions.filter_info->do_remove_column); - LOG_TRACE(log, "executeImpl, adding Row-level security filter"); row_level_security_step->setStepDescription("Row-level security filter"); query_plan.addStep(std::move(row_level_security_step)); @@ -2072,8 +2080,9 @@ void InterpreterSelectQuery::addPrewhereAliasActions() if (!expressions.prewhere_info) { - LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 1", (void*)this); const bool does_storage_support_prewhere = !input_pipe && storage && storage->supportsPrewhere(); + LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 1 - does_storage_support_prewhere {} shouldMoveToPrewhere() {}", + (void*)this, does_storage_support_prewhere, shouldMoveToPrewhere()); if (does_storage_support_prewhere && shouldMoveToPrewhere()) { LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 1.5", (void*)this); @@ -2096,6 +2105,14 @@ void InterpreterSelectQuery::addPrewhereAliasActions() expressions.prewhere_info->row_level_filter->projectInput(false); expressions.filter_info = nullptr; } + if (expressions.prewhere_info) + { + LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {} dump: {}", (void*)this, expressions.prewhere_info->dump()); + } + else + { + LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " no prewhere_info"); + } } auto & prewhere_info = analysis_result.prewhere_info; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index a586997360a..927c8fb7440 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -312,7 +312,7 @@ public: const auto & primary_key = storage_snapshot->metadata->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; KeyCondition key_condition(filter, context, primary_key_column_names, primary_key.expression, NameSet{}); - LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); + LOG_DEBUG(log, "ReadFromPart (MergeTreeSequentialSource) Key condition: {}", key_condition.toString()); if (!key_condition.alwaysFalse()) mark_ranges = MergeTreeDataSelectExecutor::markRangesFromPKRange( diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c0a0ffec7fd..5d13d844eb7 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -318,7 +319,7 @@ void StorageMerge::read( * since there is no certainty that it works when one of table is MergeTree and other is not. */ auto modified_context = Context::createCopy(local_context); - modified_context->setSetting("optimize_move_to_prewhere", false); + // modified_context->setSetting("optimize_move_to_prewhere", false); LOG_TRACE(&Poco::Logger::get("StorageMerge::read"), "processed_stage {}", QueryProcessingStage::toString(processed_stage)); @@ -832,7 +833,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( /// 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. - convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, *builder, processed_stage); + convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, *builder, processed_stage, database_name, table_name); } return builder; @@ -1014,7 +1015,9 @@ void ReadFromMerge::convertingSourceStream( const Aliases & aliases, ContextPtr local_context, QueryPipelineBuilder & builder, - const QueryProcessingStage::Enum & processed_stage) + const QueryProcessingStage::Enum & processed_stage, + const String & database_name, + const String & table_name) { Block before_block_header = builder.getHeader(); @@ -1051,7 +1054,7 @@ void ReadFromMerge::convertingSourceStream( for (const auto & column_with_type_and_name : header.getColumnsWithTypeAndName()) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "column name: {} (header.getColumnsWithTypeAndName())", column_with_type_and_name.name); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "column name: {} (header.getColumnsWithTypeAndName())", column_with_type_and_name.name); } @@ -1068,6 +1071,93 @@ void ReadFromMerge::convertingSourceStream( { return std::make_shared(stream_header, actions); }); + + + bool explicit_row_policy_filter_needed = true; + + if (explicit_row_policy_filter_needed) + { + + auto row_policy_filter = local_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); + + // row_policy_filter->expression + // auto pipe_columns = builder.getHeader().getNamesAndTypesList(); + + + ASTPtr expr = row_policy_filter->expression; + + // auto * select_ast = expr /* query_ast */ ->as(); + // assert(select_ast); + + // select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); + // auto expr_list = select_ast->select(); + // expr_list->children.push_back(expr); + // String filter_column_name = expr_list->children.at(0)->getColumnName(); + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column_name: {} ", filter_column_name); + + auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); + // auto syntax_result = TreeRewriter(local_context).analyze(expr, NamesAndTypesList()); + auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, local_context}; + + auto actions_dag = expression_analyzer.getActionsDAG(true, false); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "actions_dag: {},<> {}", actions_dag->dumpNames(), actions_dag->dumpDAG()); + + + + auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); + auto required_columns = filter_actions->getRequiredColumns(); + for (const auto & req_col : required_columns) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "req_col: {}", req_col); + } + + + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); + + + auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); + std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); + + Names required_columns_sorted = required_columns; + std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); + + Names filter_columns; + + + std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), + required_columns.begin(), required_columns.end(), + std::inserter(filter_columns, filter_columns.begin())); + + for (const auto & filter_column : filter_columns) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column: {}", filter_column); + } + + // Block block; + // block = filter_actions->getActionsDAG().updateHeader(std::move(block)); + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "block from updateHeader {}", block.dumpStructure()); + + + + builder.addSimpleTransform([&](const Block & stream_header) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "stream_header {}", stream_header.dumpStructure()); + return std::make_shared(stream_header, filter_actions, filter_columns.front(), true /* remove fake column */); + }); + + + // auto row_level_filter_step = std::make_unique( + // query_plan.getCurrentDataStream(), + // expressions.prewhere_info->row_level_filter, + // expressions.prewhere_info->row_level_column_name, + // true); + + // row_level_filter_step->setStepDescription("Row-level security filter (PREWHERE)"); + // query_plan.addStep(std::move(row_level_filter_step)); + + } } bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index babf0dd92e8..fbe6dcec298 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -201,7 +201,9 @@ private: const Aliases & aliases, ContextPtr context, QueryPipelineBuilder & builder, - const QueryProcessingStage::Enum & processed_stage); + const QueryProcessingStage::Enum & processed_stage, + const String & database_name, + const String & table_name); }; } From 657c39c79e39c9df61f094114cf89e212b0671d5 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 5 Jun 2023 16:00:31 +0000 Subject: [PATCH 027/813] merge_row_policy: row_policy_storage_merge test --- src/Storages/StorageMerge.cpp | 152 +++++++++--------- .../02763_row_policy_storage_merge.reference | 114 +++++++++++++ .../02763_row_policy_storage_merge.sql.j2 | 60 +++++++ 3 files changed, 253 insertions(+), 73 deletions(-) create mode 100644 tests/queries/0_stateless/02763_row_policy_storage_merge.reference create mode 100644 tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5d13d844eb7..6d93d88804c 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1080,83 +1080,89 @@ void ReadFromMerge::convertingSourceStream( auto row_policy_filter = local_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); - // row_policy_filter->expression - // auto pipe_columns = builder.getHeader().getNamesAndTypesList(); - - - ASTPtr expr = row_policy_filter->expression; - - // auto * select_ast = expr /* query_ast */ ->as(); - // assert(select_ast); - - // select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); - // auto expr_list = select_ast->select(); - // expr_list->children.push_back(expr); - // String filter_column_name = expr_list->children.at(0)->getColumnName(); - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column_name: {} ", filter_column_name); - - auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); - // auto syntax_result = TreeRewriter(local_context).analyze(expr, NamesAndTypesList()); - auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, local_context}; - - auto actions_dag = expression_analyzer.getActionsDAG(true, false); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "actions_dag: {},<> {}", actions_dag->dumpNames(), actions_dag->dumpDAG()); - - - - auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - auto required_columns = filter_actions->getRequiredColumns(); - for (const auto & req_col : required_columns) + if (row_policy_filter) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "req_col: {}", req_col); + + + // row_policy_filter->expression + // auto pipe_columns = builder.getHeader().getNamesAndTypesList(); + + + ASTPtr expr = row_policy_filter->expression; + + // auto * select_ast = expr /* query_ast */ ->as(); + // assert(select_ast); + + // select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); + // auto expr_list = select_ast->select(); + // expr_list->children.push_back(expr); + // String filter_column_name = expr_list->children.at(0)->getColumnName(); + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column_name: {} ", filter_column_name); + + auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); + // auto syntax_result = TreeRewriter(local_context).analyze(expr, NamesAndTypesList()); + auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, local_context}; + + auto actions_dag = expression_analyzer.getActionsDAG(true, false); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "actions_dag: {},<> {}", actions_dag->dumpNames(), actions_dag->dumpDAG()); + + + + auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); + auto required_columns = filter_actions->getRequiredColumns(); + for (const auto & req_col : required_columns) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "req_col: {}", req_col); + } + + + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); + + + auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); + std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); + + Names required_columns_sorted = required_columns; + std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); + + Names filter_columns; + + + std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), + required_columns.begin(), required_columns.end(), + std::inserter(filter_columns, filter_columns.begin())); + + for (const auto & filter_column : filter_columns) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column: {}", filter_column); + } + + // Block block; + // block = filter_actions->getActionsDAG().updateHeader(std::move(block)); + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "block from updateHeader {}", block.dumpStructure()); + + + + builder.addSimpleTransform([&](const Block & stream_header) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "stream_header {}", stream_header.dumpStructure()); + return std::make_shared(stream_header, filter_actions, filter_columns.front(), true /* remove fake column */); + }); + + + // auto row_level_filter_step = std::make_unique( + // query_plan.getCurrentDataStream(), + // expressions.prewhere_info->row_level_filter, + // expressions.prewhere_info->row_level_column_name, + // true); + + // row_level_filter_step->setStepDescription("Row-level security filter (PREWHERE)"); + // query_plan.addStep(std::move(row_level_filter_step)); } - - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - - - auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); - std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); - - Names required_columns_sorted = required_columns; - std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); - - Names filter_columns; - - - std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), - required_columns.begin(), required_columns.end(), - std::inserter(filter_columns, filter_columns.begin())); - - for (const auto & filter_column : filter_columns) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column: {}", filter_column); - } - - // Block block; - // block = filter_actions->getActionsDAG().updateHeader(std::move(block)); - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "block from updateHeader {}", block.dumpStructure()); - - - - builder.addSimpleTransform([&](const Block & stream_header) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "stream_header {}", stream_header.dumpStructure()); - return std::make_shared(stream_header, filter_actions, filter_columns.front(), true /* remove fake column */); - }); - - - // auto row_level_filter_step = std::make_unique( - // query_plan.getCurrentDataStream(), - // expressions.prewhere_info->row_level_filter, - // expressions.prewhere_info->row_level_column_name, - // true); - - // row_level_filter_step->setStepDescription("Row-level security filter (PREWHERE)"); - // query_plan.addStep(std::move(row_level_filter_step)); - } } diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference new file mode 100644 index 00000000000..ab531c5e300 --- /dev/null +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference @@ -0,0 +1,114 @@ +1 +1 +1 +1 +2 +2 +2 +2 +3 +3 +3 +3 +4 +4 +4 +4 +1 +2 +3 +4 +SETTINGS optimize_move_to_prewhere= 0 +SELECT * FROM 02763_merge_log_1 +3 +SELECT * FROM merge(currentDatabase(), 02763_merge_log_1) +3 +SELECT * FROM merge(currentDatabase(), 02763_merge_log) +1 +2 +3 +3 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>2 +3 +3 +4 +SELECT * FROM 02763_merge_merge_1 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge) +1 +2 +3 +4 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2 +3 +4 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge) +1 +1 +2 +2 +3 +3 +3 +4 +4 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge) WHEER x>2 +3 +3 +3 +4 +4 +4 +SETTINGS optimize_move_to_prewhere= 1 +SELECT * FROM 02763_merge_log_1 +3 +SELECT * FROM merge(currentDatabase(), 02763_merge_log_1) +3 +SELECT * FROM merge(currentDatabase(), 02763_merge_log) +1 +2 +3 +3 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>2 +3 +3 +4 +SELECT * FROM 02763_merge_merge_1 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge) +1 +2 +3 +4 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2 +3 +4 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge) +1 +1 +2 +2 +3 +3 +3 +4 +4 +4 +SELECT * FROM merge(currentDatabase(), 02763_merge) WHEER x>2 +3 +3 +3 +4 +4 +4 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 new file mode 100644 index 00000000000..3883b357602 --- /dev/null +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -0,0 +1,60 @@ +DROP TABLE IF EXISTS 02763_merge_log_1; +DROP TABLE IF EXISTS 02763_merge_log_2; +DROP TABLE IF EXISTS 02763_merge_merge_1; +DROP TABLE IF EXISTS 02763_merge_merge_2; +DROP ROW POLICY IF EXISTS 02763_filter_1 ON 02763_merge_log_1; +DROP ROW POLICY IF EXISTS 02763_filter_2 ON 02763_merge_merge_1; + + +CREATE TABLE 02763_merge_log_1 (x UInt8) ENGINE = Log; +CREATE TABLE 02763_merge_log_2 (x UInt8) ENGINE = Log; + +CREATE TABLE 02763_merge_merge_1 (x UInt8) ENGINE = MergeTree ORDER BY x; +CREATE TABLE 02763_merge_merge_2 (x UInt8) ENGINE = MergeTree ORDER BY x; + +INSERT INTO 02763_merge_log_1 VALUES (1), (2), (3), (4); +INSERT INTO 02763_merge_log_2 VALUES (1), (2), (3), (4); +INSERT INTO 02763_merge_merge_1 VALUES (1), (2), (3), (4); +INSERT INTO 02763_merge_merge_2 VALUES (1), (2), (3), (4); + +SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x; + +SELECT * FROM 02763_merge_log_1 ORDER BY x; + + +{% for prew in [0 , 1] -%} + +SELECT 'SETTINGS optimize_move_to_prewhere= {{prew}}'; + +CREATE ROW POLICY 02763_filter_1 ON 02763_merge_log_1 USING x=3 AS permissive TO ALL; + +SELECT 'SELECT * FROM 02763_merge_log_1'; +SELECT * FROM 02763_merge_log_1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_log_1)'; +SELECT * FROM merge(currentDatabase(), '02763_merge_log_1') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_log)'; +SELECT * FROM merge(currentDatabase(), '02763_merge_log') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>2'; +SELECT * FROM merge(currentDatabase(), '02763_merge_log') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; + +CREATE ROW POLICY 02763_filter_2 ON 02763_merge_merge_1 USING x=4 AS permissive TO ALL; + +SELECT 'SELECT * FROM 02763_merge_merge_1'; +SELECT * FROM 02763_merge_merge_1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1'; +SELECT * FROM merge(currentDatabase(), '02763_merge_merge_1') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge)'; +SELECT * FROM merge(currentDatabase(), '02763_merge_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2'; +SELECT * FROM merge(currentDatabase(), '02763_merge_merge') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; + + +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge)'; +SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge) WHEER x>2'; +SELECT * FROM merge(currentDatabase(), '02763_merge') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; + +DROP ROW POLICY 02763_filter_1 ON 02763_merge_log_1; +DROP ROW POLICY 02763_filter_2 ON 02763_merge_merge_1; + +{% endfor %} From 359cd4d32ae292392aeccadf2d2dadbe689f2f38 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 6 Jun 2023 20:45:48 +0000 Subject: [PATCH 028/813] merge_row_policy: style fixes and back to storage->read --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/StorageMerge.cpp | 40 ++++++++++----------- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f4931afe89c..fb472cb791c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1529,7 +1529,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

column_name, query_plan.getCurrentDataStream().header.dumpStructure()); auto row_level_security_step = std::make_unique( diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 6d93d88804c..9b33f3aa268 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -369,7 +369,7 @@ void StorageMerge::read( /// What will be result structure depending on query processed stage in source tables? // Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, QueryProcessingStage::Complete /* processed_stage */); - Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage ); + Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage); auto step = std::make_unique( common_header, @@ -677,7 +677,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryPlan & plan = child_plans.emplace_back(); StorageView * view = dynamic_cast(storage.get()); - if (/* !view || */ allow_experimental_analyzer) + if ( !view || allow_experimental_analyzer) // if (!view || allow_experimental_analyzer) { LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "direct storage->read"); @@ -690,26 +690,26 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( max_block_size, UInt32(streams_num)); } - else if (!view) - { - /// For view storage, we need to rewrite the `modified_query_info.view_query` to optimize read. - /// The most intuitive way is to use InterpreterSelectQuery. + // else if (!view) + // { + // /// 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); + // /// 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); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "creating InterpreterSelectQuery 1.0"); - InterpreterSelectQuery interpreter(modified_query_info.query, - modified_context, - storage, - storage->getInMemoryMetadataPtr(), // view->getInMemoryMetadataPtr(), - // SelectQueryOptions(/* processed_stage*/)); - SelectQueryOptions(processed_stage)); - // SelectQueryOptions(QueryProcessingStage::WithMergeableState)); - interpreter.buildQueryPlan(plan); - } + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "creating InterpreterSelectQuery 1.0"); + // InterpreterSelectQuery interpreter(modified_query_info.query, + // modified_context, + // storage, + // storage->getInMemoryMetadataPtr(), // view->getInMemoryMetadataPtr(), + // // SelectQueryOptions(/* processed_stage*/)); + // SelectQueryOptions(processed_stage)); + // // SelectQueryOptions(QueryProcessingStage::WithMergeableState)); + // interpreter.buildQueryPlan(plan); + // } else { /// For view storage, we need to rewrite the `modified_query_info.view_query` to optimize read. From c457fa727e358d77802a1da1540caa16ae64b6a9 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 9 Jun 2023 09:03:47 +0000 Subject: [PATCH 029/813] merge_row_policy: addFilter() instead of FilterTransform --- src/Storages/StorageMerge.cpp | 71 ++++++++++++++++++++++++++++++++++- 1 file changed, 69 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 9b33f3aa268..c29c2af9b12 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -677,6 +677,8 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryPlan & plan = child_plans.emplace_back(); StorageView * view = dynamic_cast(storage.get()); + bool direct_read = false; + if ( !view || allow_experimental_analyzer) // if (!view || allow_experimental_analyzer) { @@ -689,6 +691,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( processed_stage, max_block_size, UInt32(streams_num)); + direct_read = true; } // else if (!view) // { @@ -742,10 +745,74 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( read_from_merge_tree->addFilter(filter_dags[i], filter_nodes.nodes[i]); } } - builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(modified_context), BuildQueryPipelineSettings::fromContext(modified_context)); + if (auto * source_step_with_filter = typeid_cast(plan.getRootNode()->step.get())) + { + auto row_policy_filter = modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); + + if (row_policy_filter) + { + + + // row_policy_filter->expression + // auto pipe_columns = builder.getHeader().getNamesAndTypesList(); + + + ASTPtr expr = row_policy_filter->expression; + + // auto * select_ast = expr /* query_ast */ ->as(); + // assert(select_ast); + + // select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); + // auto expr_list = select_ast->select(); + // expr_list->children.push_back(expr); + // String filter_column_name = expr_list->children.at(0)->getColumnName(); + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column_name: {} ", filter_column_name); + + auto syntax_result = TreeRewriter(modified_context).analyze(expr, builder->getHeader().getNamesAndTypesList() /* pipe_columns*/); + // auto syntax_result = TreeRewriter(local_context).analyze(expr, NamesAndTypesList()); + auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, modified_context}; + + auto filter_dag_ptr = expression_analyzer.getActionsDAG(true, false); + + + auto filter_actions = std::make_shared(filter_dag_ptr, ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + auto required_columns = filter_actions->getRequiredColumns(); + + for (const auto & req_col : required_columns) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "req_col: {}", req_col); + } + + + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); + + + auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); + std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); + + Names required_columns_sorted = required_columns; + std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); + + Names filter_columns; + + + std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), + required_columns.begin(), required_columns.end(), + std::inserter(filter_columns, filter_columns.begin())); + + + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "SourceStepWithFilter detected"); + source_step_with_filter->addFilter(filter_dag_ptr, filter_columns.front()); + } + } + + } else if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { @@ -1073,7 +1140,7 @@ void ReadFromMerge::convertingSourceStream( }); - bool explicit_row_policy_filter_needed = true; + bool explicit_row_policy_filter_needed = false; if (explicit_row_policy_filter_needed) { From a19cd8089b8620cf12e238181679a6492af30607 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 13 Jun 2023 17:19:20 +0000 Subject: [PATCH 030/813] merge_row_policy: add_filter() actually works --- src/Interpreters/InterpreterSelectQuery.cpp | 1 - src/Storages/StorageMerge.cpp | 39 ++++++++------------- 2 files changed, 15 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fb472cb791c..22f0feb195f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -561,7 +561,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( } - StorageView * view = nullptr; if (storage) view = dynamic_cast(storage.get()); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c29c2af9b12..5942c49237a 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -668,6 +669,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( modified_query_info); #pragma GCC diagnostic ignored "-Wunreachable-code" +#pragma GCC diagnostic ignored "-Wunused-variable" if (processed_stage <= storage_stage || (allow_experimental_analyzer && processed_stage == QueryProcessingStage::FetchColumns)) { /// If there are only virtual columns in query, you must request at least one other column. @@ -679,7 +681,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( StorageView * view = dynamic_cast(storage.get()); bool direct_read = false; - if ( !view || allow_experimental_analyzer) + if (!view || allow_experimental_analyzer) // if (!view || allow_experimental_analyzer) { LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "direct storage->read"); @@ -741,25 +743,18 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "ReadFromMergeTree detected, DAG size {}", filters_dags_size); for (size_t i = 0; i < filters_dags_size; ++i) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "adding filter"); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "adding filter {}", filter_dags[i]->dumpDAG()); read_from_merge_tree->addFilter(filter_dags[i], filter_nodes.nodes[i]); } } - builder = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(modified_context), - BuildQueryPipelineSettings::fromContext(modified_context)); - if (auto * source_step_with_filter = typeid_cast(plan.getRootNode()->step.get())) + if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) { auto row_policy_filter = modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); if (row_policy_filter) { - - // row_policy_filter->expression // auto pipe_columns = builder.getHeader().getNamesAndTypesList(); - - ASTPtr expr = row_policy_filter->expression; // auto * select_ast = expr /* query_ast */ ->as(); @@ -771,7 +766,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( // String filter_column_name = expr_list->children.at(0)->getColumnName(); // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column_name: {} ", filter_column_name); - auto syntax_result = TreeRewriter(modified_context).analyze(expr, builder->getHeader().getNamesAndTypesList() /* pipe_columns*/); + auto syntax_result = TreeRewriter(modified_context).analyze(expr, header/*builder->getHeader().*/.getNamesAndTypesList() /* pipe_columns*/); // auto syntax_result = TreeRewriter(local_context).analyze(expr, NamesAndTypesList()); auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, modified_context}; @@ -787,7 +782,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( } - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); @@ -806,12 +800,18 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( std::inserter(filter_columns, filter_columns.begin())); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "SourceStepWithFilter detected"); - source_step_with_filter->addFilter(filter_dag_ptr, filter_columns.front()); + auto found_column = filter_dag_ptr->tryFindInOutputs(filter_columns.front()); + assert(found_column); + + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "found column {}", found_column->dumpDAG()); + + source_step_with_filter->addFilter(/* filter_actions */ filter_dag_ptr, filter_columns.front()); } } - + builder = plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(modified_context), + BuildQueryPipelineSettings::fromContext(modified_context)); } else if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) @@ -1124,9 +1124,6 @@ void ReadFromMerge::convertingSourceStream( LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "column name: {} (header.getColumnsWithTypeAndName())", column_with_type_and_name.name); } - - - auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); @@ -1150,7 +1147,6 @@ void ReadFromMerge::convertingSourceStream( if (row_policy_filter) { - // row_policy_filter->expression // auto pipe_columns = builder.getHeader().getNamesAndTypesList(); @@ -1174,7 +1170,6 @@ void ReadFromMerge::convertingSourceStream( LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "actions_dag: {},<> {}", actions_dag->dumpNames(), actions_dag->dumpDAG()); - auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); auto required_columns = filter_actions->getRequiredColumns(); for (const auto & req_col : required_columns) @@ -1183,7 +1178,6 @@ void ReadFromMerge::convertingSourceStream( } - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); @@ -1210,8 +1204,6 @@ void ReadFromMerge::convertingSourceStream( // block = filter_actions->getActionsDAG().updateHeader(std::move(block)); // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "block from updateHeader {}", block.dumpStructure()); - - builder.addSimpleTransform([&](const Block & stream_header) { LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "stream_header {}", stream_header.dumpStructure()); @@ -1229,7 +1221,6 @@ void ReadFromMerge::convertingSourceStream( // query_plan.addStep(std::move(row_level_filter_step)); } - } } From 978a535849da2266f186b6e10ccea1a5fdd3d8d6 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 06:29:41 +0000 Subject: [PATCH 031/813] merge_row_policy: addFilter(() together with FilterTransform --- src/Storages/StorageMerge.cpp | 2 +- .../02763_row_policy_storage_merge.reference | 28 +++++++++++++++++-- .../02763_row_policy_storage_merge.sql.j2 | 23 ++++++++------- 3 files changed, 40 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5942c49237a..e1ff6cb1091 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1137,7 +1137,7 @@ void ReadFromMerge::convertingSourceStream( }); - bool explicit_row_policy_filter_needed = false; + bool explicit_row_policy_filter_needed = true; if (explicit_row_policy_filter_needed) { diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference index ab531c5e300..9dcc5f449ab 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference @@ -58,7 +58,19 @@ SELECT * FROM merge(currentDatabase(), 02763_merge) 4 4 4 -SELECT * FROM merge(currentDatabase(), 02763_merge) WHEER x>2 +SELECT * FROM merge(currentDatabase(), 02763_merge) WHERE x>2 +3 +3 +3 +4 +4 +4 +aaa 6 +aaa 6 +aaa 6 +aaa 8 +aaa 8 +aaa 8 3 3 3 @@ -105,7 +117,19 @@ SELECT * FROM merge(currentDatabase(), 02763_merge) 4 4 4 -SELECT * FROM merge(currentDatabase(), 02763_merge) WHEER x>2 +SELECT * FROM merge(currentDatabase(), 02763_merge) WHERE x>2 +3 +3 +3 +4 +4 +4 +aaa 6 +aaa 6 +aaa 6 +aaa 8 +aaa 8 +aaa 8 3 3 3 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index 3883b357602..33b02275d4a 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -31,28 +31,31 @@ CREATE ROW POLICY 02763_filter_1 ON 02763_merge_log_1 USING x=3 AS permissive TO SELECT 'SELECT * FROM 02763_merge_log_1'; SELECT * FROM 02763_merge_log_1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_log_1)'; -SELECT * FROM merge(currentDatabase(), '02763_merge_log_1') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT * FROM merge(currentDatabase(), '02763_merge_log_1') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_log)'; -SELECT * FROM merge(currentDatabase(), '02763_merge_log') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT * FROM merge(currentDatabase(), '02763_merge_log') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>2'; -SELECT * FROM merge(currentDatabase(), '02763_merge_log') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT * FROM merge(currentDatabase(), '02763_merge_log') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; CREATE ROW POLICY 02763_filter_2 ON 02763_merge_merge_1 USING x=4 AS permissive TO ALL; SELECT 'SELECT * FROM 02763_merge_merge_1'; -SELECT * FROM 02763_merge_merge_1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT * FROM 02763_merge_merge_1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1'; -SELECT * FROM merge(currentDatabase(), '02763_merge_merge_1') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT * FROM merge(currentDatabase(), '02763_merge_merge_1') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge)'; -SELECT * FROM merge(currentDatabase(), '02763_merge_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT * FROM merge(currentDatabase(), '02763_merge_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2'; -SELECT * FROM merge(currentDatabase(), '02763_merge_merge') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT * FROM merge(currentDatabase(), '02763_merge_merge') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge)'; -SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; -SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge) WHEER x>2'; -SELECT * FROM merge(currentDatabase(), '02763_merge') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}};; +SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge) WHERE x>2'; +SELECT * FROM merge(currentDatabase(), '02763_merge') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + +SELECT 'aaa', x*2 as x_2 FROM merge(currentDatabase(), '02763_merge') WHERE x>2 ORDER BY x_2 SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT x FROM (SELECT * FROM merge(currentDatabase(), '02763_merge') WHERE x IN (3,4)) ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; DROP ROW POLICY 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY 02763_filter_2 ON 02763_merge_merge_1; From da5f607242d1ff97d6380a600ac666651a9b08cc Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 21:31:17 +0000 Subject: [PATCH 032/813] merge_row_policy: cleanup, remove some debug output --- src/Interpreters/InterpreterSelectQuery.cpp | 41 +--- .../MergeTree/MergeTreeSequentialSource.cpp | 2 +- src/Storages/StorageMerge.cpp | 202 ++++-------------- 3 files changed, 43 insertions(+), 202 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 22f0feb195f..e1faa8c8958 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -93,14 +93,12 @@ #include #include -#include namespace ProfileEvents { extern const Event SelectQueriesWithSubqueries; extern const Event QueriesWithSubqueries; } -#pragma GCC diagnostic ignored "-Wold-style-cast" namespace DB { @@ -134,9 +132,6 @@ FilterDAGInfoPtr generateFilterActions( Names & prerequisite_columns, PreparedSetsPtr prepared_sets) { - LOG_TRACE(&Poco::Logger::get("generateFilterActions"), "top of"); - - auto filter_info = std::make_shared(); const auto & db_name = table_id.getDatabaseName(); @@ -551,15 +546,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( std::shared_ptr table_join = joined_tables.makeTableJoin(query); if (storage) - { - LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), " {}, table name: {}, calling getRowPolicyFilter", (void*)this, table_id.getTableName()); row_policy_filter = context->getRowPolicyFilter(table_id.getDatabaseName(), table_id.getTableName(), RowPolicyFilterType::SELECT_FILTER); - } - else - { - LOG_TRACE(&Poco::Logger::get("InterpretSelectQuery ctor"), " {}, no storage", (void*)this); - } - StorageView * view = nullptr; if (storage) @@ -863,8 +850,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Add prewhere actions with alias columns and record needed columns from storage. if (storage) { - LOG_TRACE(log, "calling addPrewhereAliasActions"); - addPrewhereAliasActions(); analysis_result.required_columns = required_columns; } @@ -960,8 +945,6 @@ Block InterpreterSelectQuery::getSampleBlockImpl() analysis_result = ExpressionAnalysisResult( *query_analyzer, metadata_snapshot, first_stage, second_stage, options.only_analyze, filter_info, additional_filter_info, source_header); - LOG_TRACE(log, "getSampleBlockImpl {} : source_header after ExpressionAnalysisResult {}", (void*) this, source_header.dumpStructure()); - if (options.to_stage == QueryProcessingStage::Enum::FetchColumns) { @@ -971,12 +954,8 @@ Block InterpreterSelectQuery::getSampleBlockImpl() { header = analysis_result.prewhere_info->prewhere_actions->updateHeader(header); if (analysis_result.prewhere_info->remove_prewhere_column) - { - LOG_TRACE(log, "getSampleBlockImpl {} : erasing column {}", (void*) this, analysis_result.prewhere_info->prewhere_column_name); header.erase(analysis_result.prewhere_info->prewhere_column_name); - } } - LOG_TRACE(log, "getSampleBlockImpl {} : returning header", (void*) this); return header; } @@ -1462,7 +1441,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

{}", (void*) this, QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); + LOG_TRACE(log, "{} -> {}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); } if (query_info.projection && query_info.projection->input_order_info && query_info.input_order_info) @@ -1528,16 +1507,12 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

column_name, query_plan.getCurrentDataStream().header.dumpStructure()); - auto row_level_security_step = std::make_unique( query_plan.getCurrentDataStream(), expressions.filter_info->actions, expressions.filter_info->column_name, expressions.filter_info->do_remove_column); - row_level_security_step->setStepDescription("Row-level security filter"); query_plan.addStep(std::move(row_level_security_step)); } @@ -2075,16 +2050,11 @@ void InterpreterSelectQuery::addPrewhereAliasActions() auto & expressions = analysis_result; if (expressions.filter_info) { - LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info", (void*)this); - if (!expressions.prewhere_info) { const bool does_storage_support_prewhere = !input_pipe && storage && storage->supportsPrewhere(); - LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 1 - does_storage_support_prewhere {} shouldMoveToPrewhere() {}", - (void*)this, does_storage_support_prewhere, shouldMoveToPrewhere()); if (does_storage_support_prewhere && shouldMoveToPrewhere()) { - LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 1.5", (void*)this); /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. expressions.prewhere_info = std::make_shared( std::move(expressions.filter_info->actions), @@ -2097,21 +2067,12 @@ void InterpreterSelectQuery::addPrewhereAliasActions() } else { - LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {}, expressions.filter_info 2", (void*)this); /// Add row level security actions to prewhere. expressions.prewhere_info->row_level_filter = std::move(expressions.filter_info->actions); expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name); expressions.prewhere_info->row_level_filter->projectInput(false); expressions.filter_info = nullptr; } - if (expressions.prewhere_info) - { - LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " {} dump: {}", (void*)this, expressions.prewhere_info->dump()); - } - else - { - LOG_TRACE(&Poco::Logger::get("addPrewhereAliasActions"), " no prewhere_info"); - } } auto & prewhere_info = analysis_result.prewhere_info; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 927c8fb7440..a586997360a 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -312,7 +312,7 @@ public: const auto & primary_key = storage_snapshot->metadata->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; KeyCondition key_condition(filter, context, primary_key_column_names, primary_key.expression, NameSet{}); - LOG_DEBUG(log, "ReadFromPart (MergeTreeSequentialSource) Key condition: {}", key_condition.toString()); + LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); if (!key_condition.alwaysFalse()) mark_ranges = MergeTreeDataSelectExecutor::markRangesFromPKRange( diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e1ff6cb1091..60ab9f86a2a 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -47,10 +47,6 @@ #include -#pragma GCC diagnostic ignored "-Wunused-parameter" -#pragma GCC diagnostic ignored "-Wunused-but-set-variable" - - namespace { @@ -270,7 +266,6 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( /// (see removeJoin()) /// /// And for this we need to return FetchColumns. - LOG_TRACE(&Poco::Logger::get("StorageMerge::getQueryProcessingStage"), "to_stage {}", to_stage); if (const auto * select = query_info.query->as(); select && hasJoin(*select)) return QueryProcessingStage::FetchColumns; @@ -295,7 +290,6 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( stage_in_source_tables, table->getQueryProcessingStage(local_context, to_stage, table->getStorageSnapshot(table->getInMemoryMetadataPtr(), local_context), query_info)); - LOG_TRACE(&Poco::Logger::get("StorageMerge::getQueryProcessingStage"), "stage_in_source_tables {}", stage_in_source_tables); } iterator->next(); @@ -322,9 +316,6 @@ void StorageMerge::read( auto modified_context = Context::createCopy(local_context); // modified_context->setSetting("optimize_move_to_prewhere", false); - LOG_TRACE(&Poco::Logger::get("StorageMerge::read"), "processed_stage {}", QueryProcessingStage::toString(processed_stage)); - - bool has_database_virtual_column = false; bool has_table_virtual_column = false; Names real_column_names; @@ -339,7 +330,6 @@ void StorageMerge::read( else { real_column_names.push_back(column_name); - LOG_TRACE(&Poco::Logger::get("StorageMerge::read"), "column_name {}", column_name); } } @@ -494,7 +484,6 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu bool with_aliases = common_processed_stage == QueryProcessingStage::FetchColumns && !storage_columns.getAliases().empty(); if (with_aliases) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), "with_aliases"); ASTPtr required_columns_expr_list = std::make_shared(); ASTPtr column_expr; @@ -534,8 +523,6 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu } } - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), "table name: {}", storage->getStorageID().getTableName()); - auto source_pipeline = createSources( nested_storage_snaphsot, modified_query_info, @@ -668,8 +655,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage_snapshot, modified_query_info); -#pragma GCC diagnostic ignored "-Wunreachable-code" -#pragma GCC diagnostic ignored "-Wunused-variable" if (processed_stage <= storage_stage || (allow_experimental_analyzer && processed_stage == QueryProcessingStage::FetchColumns)) { /// If there are only virtual columns in query, you must request at least one other column. @@ -679,12 +664,9 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryPlan & plan = child_plans.emplace_back(); StorageView * view = dynamic_cast(storage.get()); - bool direct_read = false; if (!view || allow_experimental_analyzer) - // if (!view || allow_experimental_analyzer) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "direct storage->read"); storage->read(plan, real_column_names, storage_snapshot, @@ -693,28 +675,45 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( processed_stage, max_block_size, UInt32(streams_num)); - direct_read = true; + + + if (!plan.isInitialized()) + return {}; + + if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) + { + auto row_policy_filter = modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); + + if (row_policy_filter) + { + ASTPtr expr = row_policy_filter->expression; + + auto syntax_result = TreeRewriter(modified_context).analyze(expr, header.getNamesAndTypesList()); + auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, modified_context}; + + auto filter_dag_ptr = expression_analyzer.getActionsDAG(true, false); + auto filter_actions = std::make_shared(filter_dag_ptr, ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + auto required_columns = filter_actions->getRequiredColumns(); + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); + + auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); + std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); + + Names required_columns_sorted = required_columns; + std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); + + Names filter_columns; + + std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), + required_columns.begin(), required_columns.end(), + std::inserter(filter_columns, filter_columns.begin())); + + source_step_with_filter->addFilter(filter_dag_ptr, filter_columns.front()); + } + } } - // else if (!view) - // { - // /// 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); - - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "creating InterpreterSelectQuery 1.0"); - // InterpreterSelectQuery interpreter(modified_query_info.query, - // modified_context, - // storage, - // storage->getInMemoryMetadataPtr(), // view->getInMemoryMetadataPtr(), - // // SelectQueryOptions(/* processed_stage*/)); - // SelectQueryOptions(processed_stage)); - // // SelectQueryOptions(QueryProcessingStage::WithMergeableState)); - // interpreter.buildQueryPlan(plan); - // } else { /// For view storage, we need to rewrite the `modified_query_info.view_query` to optimize read. @@ -725,90 +724,25 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( modified_context->setSetting("max_streams_to_max_threads_ratio", 1); modified_context->setSetting("max_block_size", max_block_size); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "creating InterpreterSelectQuery 1.5"); InterpreterSelectQuery interpreter(modified_query_info.query, modified_context, storage, view->getInMemoryMetadataPtr(), SelectQueryOptions(processed_stage)); interpreter.buildQueryPlan(plan); - } - if (!plan.isInitialized()) - return {}; + if (!plan.isInitialized()) + return {}; + } if (auto * read_from_merge_tree = typeid_cast(plan.getRootNode()->step.get())) { size_t filters_dags_size = filter_dags.size(); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "ReadFromMergeTree detected, DAG size {}", filters_dags_size); for (size_t i = 0; i < filters_dags_size; ++i) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "adding filter {}", filter_dags[i]->dumpDAG()); read_from_merge_tree->addFilter(filter_dags[i], filter_nodes.nodes[i]); } } - if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) - { - auto row_policy_filter = modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); - - if (row_policy_filter) - { - // row_policy_filter->expression - // auto pipe_columns = builder.getHeader().getNamesAndTypesList(); - ASTPtr expr = row_policy_filter->expression; - - // auto * select_ast = expr /* query_ast */ ->as(); - // assert(select_ast); - - // select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); - // auto expr_list = select_ast->select(); - // expr_list->children.push_back(expr); - // String filter_column_name = expr_list->children.at(0)->getColumnName(); - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column_name: {} ", filter_column_name); - - auto syntax_result = TreeRewriter(modified_context).analyze(expr, header/*builder->getHeader().*/.getNamesAndTypesList() /* pipe_columns*/); - // auto syntax_result = TreeRewriter(local_context).analyze(expr, NamesAndTypesList()); - auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, modified_context}; - - auto filter_dag_ptr = expression_analyzer.getActionsDAG(true, false); - - - auto filter_actions = std::make_shared(filter_dag_ptr, ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - auto required_columns = filter_actions->getRequiredColumns(); - - for (const auto & req_col : required_columns) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "req_col: {}", req_col); - } - - - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - - - auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); - std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); - - Names required_columns_sorted = required_columns; - std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); - - Names filter_columns; - - - std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), - required_columns.begin(), required_columns.end(), - std::inserter(filter_columns, filter_columns.begin())); - - - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "SourceStepWithFilter detected"); - auto found_column = filter_dag_ptr->tryFindInOutputs(filter_columns.front()); - assert(found_column); - - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "found column {}", found_column->dumpDAG()); - - source_step_with_filter->addFilter(/* filter_actions */ filter_dag_ptr, filter_columns.front()); - } - } builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(modified_context), BuildQueryPipelineSettings::fromContext(modified_context)); @@ -834,7 +768,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( { modified_select.replaceDatabaseAndTable(database_name, table_name); /// TODO: Find a way to support projections for StorageMerge - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "creating InterpreterSelectQuery 2"); InterpreterSelectQuery interpreter{modified_query_info.query, modified_context, SelectQueryOptions(processed_stage).ignoreProjections()}; @@ -1114,16 +1047,6 @@ void ReadFromMerge::convertingSourceStream( convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; - for (const auto & column_with_type_and_name : builder.getHeader().getColumnsWithTypeAndName()) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "column name: {} (builder.getHeader().getColumnsWithTypeAndName())", column_with_type_and_name.name); - } - - for (const auto & column_with_type_and_name : header.getColumnsWithTypeAndName()) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "column name: {} (header.getColumnsWithTypeAndName())", column_with_type_and_name.name); - } - auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); @@ -1147,36 +1070,14 @@ void ReadFromMerge::convertingSourceStream( if (row_policy_filter) { - // row_policy_filter->expression - // auto pipe_columns = builder.getHeader().getNamesAndTypesList(); - - ASTPtr expr = row_policy_filter->expression; - // auto * select_ast = expr /* query_ast */ ->as(); - // assert(select_ast); - - // select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); - // auto expr_list = select_ast->select(); - // expr_list->children.push_back(expr); - // String filter_column_name = expr_list->children.at(0)->getColumnName(); - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column_name: {} ", filter_column_name); - auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); - // auto syntax_result = TreeRewriter(local_context).analyze(expr, NamesAndTypesList()); auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, local_context}; auto actions_dag = expression_analyzer.getActionsDAG(true, false); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "actions_dag: {},<> {}", actions_dag->dumpNames(), actions_dag->dumpDAG()); - - auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); auto required_columns = filter_actions->getRequiredColumns(); - for (const auto & req_col : required_columns) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "req_col: {}", req_col); - } - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); @@ -1195,32 +1096,11 @@ void ReadFromMerge::convertingSourceStream( required_columns.begin(), required_columns.end(), std::inserter(filter_columns, filter_columns.begin())); - for (const auto & filter_column : filter_columns) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_column: {}", filter_column); - } - - // Block block; - // block = filter_actions->getActionsDAG().updateHeader(std::move(block)); - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "block from updateHeader {}", block.dumpStructure()); - builder.addSimpleTransform([&](const Block & stream_header) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "stream_header {}", stream_header.dumpStructure()); return std::make_shared(stream_header, filter_actions, filter_columns.front(), true /* remove fake column */); }); - - - // auto row_level_filter_step = std::make_unique( - // query_plan.getCurrentDataStream(), - // expressions.prewhere_info->row_level_filter, - // expressions.prewhere_info->row_level_column_name, - // true); - - // row_level_filter_step->setStepDescription("Row-level security filter (PREWHERE)"); - // query_plan.addStep(std::move(row_level_filter_step)); } - } } From e2ddf40cfd6fde71cc5ef3075f2e8190dd408353 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 21:36:20 +0000 Subject: [PATCH 033/813] merge_row_policy: further cleanup --- src/Storages/StorageMerge.cpp | 72 ++++++++++++++--------------------- 1 file changed, 29 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 60ab9f86a2a..483f592f819 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -297,7 +297,6 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( } return selected_table_size == 1 ? stage_in_source_tables : std::min(stage_in_source_tables, QueryProcessingStage::WithMergeableState); - // return QueryProcessingStage::Complete; } void StorageMerge::read( @@ -314,7 +313,6 @@ void StorageMerge::read( * since there is no certainty that it works when one of table is MergeTree and other is not. */ auto modified_context = Context::createCopy(local_context); - // modified_context->setSetting("optimize_move_to_prewhere", false); bool has_database_virtual_column = false; bool has_table_virtual_column = false; @@ -328,9 +326,7 @@ void StorageMerge::read( else if (column_name == "_table" && isVirtualColumn(column_name, storage_snapshot->metadata)) has_table_virtual_column = true; else - { real_column_names.push_back(column_name); - } } StorageListWithLocks selected_tables @@ -359,7 +355,6 @@ void StorageMerge::read( query_plan.addInterpreterContext(modified_context); /// What will be result structure depending on query processed stage in source tables? - // Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, QueryProcessingStage::Complete /* processed_stage */); Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage); auto step = std::make_unique( @@ -664,7 +659,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryPlan & plan = child_plans.emplace_back(); StorageView * view = dynamic_cast(storage.get()); - if (!view || allow_experimental_analyzer) { storage->read(plan, @@ -1046,7 +1040,6 @@ void ReadFromMerge::convertingSourceStream( if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns) convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; - auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); @@ -1060,47 +1053,40 @@ void ReadFromMerge::convertingSourceStream( }); - bool explicit_row_policy_filter_needed = true; + auto row_policy_filter = local_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); - if (explicit_row_policy_filter_needed) + if (row_policy_filter) { + ASTPtr expr = row_policy_filter->expression; - auto row_policy_filter = local_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); + auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); + auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, local_context}; - if (row_policy_filter) + auto actions_dag = expression_analyzer.getActionsDAG(true, false); + auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); + auto required_columns = filter_actions->getRequiredColumns(); + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); + + + auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); + std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); + + Names required_columns_sorted = required_columns; + std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); + + Names filter_columns; + + + std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), + required_columns.begin(), required_columns.end(), + std::inserter(filter_columns, filter_columns.begin())); + + builder.addSimpleTransform([&](const Block & stream_header) { - - ASTPtr expr = row_policy_filter->expression; - - auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); - auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, local_context}; - - auto actions_dag = expression_analyzer.getActionsDAG(true, false); - auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - auto required_columns = filter_actions->getRequiredColumns(); - - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - - - auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); - std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); - - Names required_columns_sorted = required_columns; - std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); - - Names filter_columns; - - - std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), - required_columns.begin(), required_columns.end(), - std::inserter(filter_columns, filter_columns.begin())); - - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, filter_actions, filter_columns.front(), true /* remove fake column */); - }); - } + return std::make_shared(stream_header, filter_actions, filter_columns.front(), true /* remove fake column */); + }); } } From a447b3f9e0576b76895c8840da17191d68531f58 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sun, 25 Jun 2023 21:22:58 +0000 Subject: [PATCH 034/813] merge_row_policy: more tests --- .../02763_row_policy_storage_merge.reference | 254 ++++++++++-------- .../02763_row_policy_storage_merge.sql.j2 | 44 ++- 2 files changed, 182 insertions(+), 116 deletions(-) diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference index 9dcc5f449ab..444513c6c20 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference @@ -1,138 +1,180 @@ -1 -1 -1 -1 -2 -2 -2 -2 -3 -3 -3 -3 -4 -4 -4 -4 -1 -2 -3 -4 +1 11 +1 11 +1 11 +1 11 +2 12 +2 12 +2 12 +2 12 +3 13 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 +4 14 +1 11 +2 12 +3 13 +4 14 SETTINGS optimize_move_to_prewhere= 0 SELECT * FROM 02763_merge_log_1 -3 +3 13 SELECT * FROM merge(currentDatabase(), 02763_merge_log_1) -3 +3 13 SELECT * FROM merge(currentDatabase(), 02763_merge_log) -1 -2 -3 -3 -4 +1 11 +2 12 +3 13 +3 13 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>2 -3 -3 -4 +3 13 +3 13 +4 14 SELECT * FROM 02763_merge_merge_1 -4 -SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1 -4 +4 14 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1) +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge_merge) -1 -2 -3 -4 -4 +1 11 +2 12 +3 13 +4 14 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2 -3 -4 -4 +3 13 +4 14 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge) -1 -1 -2 -2 -3 -3 -3 -4 -4 -4 +1 11 +1 11 +2 12 +2 12 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge) WHERE x>2 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 +aaa 6 39 +aaa 6 39 +aaa 6 39 +aaa 8 42 +aaa 8 42 +aaa 8 42 3 3 3 4 4 4 -aaa 6 -aaa 6 -aaa 6 -aaa 8 -aaa 8 -aaa 8 -3 -3 -3 -4 -4 -4 +SELECT * FROM merge(...) LEFT JOIN merge(...) +3 13 13 +3 13 13 +4 14 14 +4 14 14 +SELECT * FROM merge(...) UNION ALL SELECT * FROM merge(...) +1 11 +1 11 +2 12 +2 12 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 +SELECT x, SUM(x) FROM (SELECT * FROM merge(...) UNION ALL ...) GROUP BY x +1 22 +2 24 +3 39 +4 42 SETTINGS optimize_move_to_prewhere= 1 SELECT * FROM 02763_merge_log_1 -3 +3 13 SELECT * FROM merge(currentDatabase(), 02763_merge_log_1) -3 +3 13 SELECT * FROM merge(currentDatabase(), 02763_merge_log) -1 -2 -3 -3 -4 +1 11 +2 12 +3 13 +3 13 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>2 -3 -3 -4 +3 13 +3 13 +4 14 SELECT * FROM 02763_merge_merge_1 -4 -SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1 -4 +4 14 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1) +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge_merge) -1 -2 -3 -4 -4 +1 11 +2 12 +3 13 +4 14 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2 -3 -4 -4 +3 13 +4 14 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge) -1 -1 -2 -2 -3 -3 -3 -4 -4 -4 +1 11 +1 11 +2 12 +2 12 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge) WHERE x>2 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 +aaa 6 39 +aaa 6 39 +aaa 6 39 +aaa 8 42 +aaa 8 42 +aaa 8 42 3 3 3 4 4 4 -aaa 6 -aaa 6 -aaa 6 -aaa 8 -aaa 8 -aaa 8 -3 -3 -3 -4 -4 -4 +SELECT * FROM merge(...) LEFT JOIN merge(...) +3 13 13 +3 13 13 +4 14 14 +4 14 14 +SELECT * FROM merge(...) UNION ALL SELECT * FROM merge(...) +1 11 +1 11 +2 12 +2 12 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 +SELECT x, SUM(x) FROM (SELECT * FROM merge(...) UNION ALL ...) GROUP BY x +1 22 +2 24 +3 39 +4 42 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index 33b02275d4a..b5094f927f4 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -6,16 +6,16 @@ DROP ROW POLICY IF EXISTS 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY IF EXISTS 02763_filter_2 ON 02763_merge_merge_1; -CREATE TABLE 02763_merge_log_1 (x UInt8) ENGINE = Log; -CREATE TABLE 02763_merge_log_2 (x UInt8) ENGINE = Log; +CREATE TABLE 02763_merge_log_1 (x UInt8, y UInt64) ENGINE = Log; +CREATE TABLE 02763_merge_log_2 (x UInt8, y UInt64) ENGINE = Log; -CREATE TABLE 02763_merge_merge_1 (x UInt8) ENGINE = MergeTree ORDER BY x; -CREATE TABLE 02763_merge_merge_2 (x UInt8) ENGINE = MergeTree ORDER BY x; +CREATE TABLE 02763_merge_merge_1 (x UInt8, y UInt64) ENGINE = MergeTree ORDER BY x; +CREATE TABLE 02763_merge_merge_2 (x UInt8, y UInt64) ENGINE = MergeTree ORDER BY x; -INSERT INTO 02763_merge_log_1 VALUES (1), (2), (3), (4); -INSERT INTO 02763_merge_log_2 VALUES (1), (2), (3), (4); -INSERT INTO 02763_merge_merge_1 VALUES (1), (2), (3), (4); -INSERT INTO 02763_merge_merge_2 VALUES (1), (2), (3), (4); +INSERT INTO 02763_merge_log_1 VALUES (1, 11), (2, 12), (3, 13), (4, 14); +INSERT INTO 02763_merge_log_2 VALUES (1, 11), (2, 12), (3, 13), (4, 14); +INSERT INTO 02763_merge_merge_1 VALUES (1, 11), (2, 12), (3, 13), (4, 14); +INSERT INTO 02763_merge_merge_2 VALUES (1, 11), (2, 12), (3, 13), (4, 14); SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x; @@ -41,7 +41,7 @@ CREATE ROW POLICY 02763_filter_2 ON 02763_merge_merge_1 USING x=4 AS permissive SELECT 'SELECT * FROM 02763_merge_merge_1'; SELECT * FROM 02763_merge_merge_1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; -SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1'; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge_1)'; SELECT * FROM merge(currentDatabase(), '02763_merge_merge_1') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge)'; SELECT * FROM merge(currentDatabase(), '02763_merge_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; @@ -54,9 +54,33 @@ SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x SETTINGS optimi SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge) WHERE x>2'; SELECT * FROM merge(currentDatabase(), '02763_merge') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; -SELECT 'aaa', x*2 as x_2 FROM merge(currentDatabase(), '02763_merge') WHERE x>2 ORDER BY x_2 SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'aaa', x*2 as x_2, y*3 as y_3 FROM merge(currentDatabase(), '02763_merge') WHERE x>2 ORDER BY x_2 SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT x FROM (SELECT * FROM merge(currentDatabase(), '02763_merge') WHERE x IN (3,4)) ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT * FROM merge(...) LEFT JOIN merge(...)'; +SELECT * FROM merge(currentDatabase(), '02763_merge.*1') as a +LEFT JOIN +merge(currentDatabase(), '02763_merge.*2') as b +USING (x) +ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + +SELECT 'SELECT * FROM merge(...) UNION ALL SELECT * FROM merge(...)'; +SELECT * FROM +( +SELECT * FROM merge(currentDatabase(), '02763_merge.*1') +UNION ALL +SELECT * FROM merge(currentDatabase(), '02763_merge.*2') +) +ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + +SELECT 'SELECT x, SUM(x) FROM (SELECT * FROM merge(...) UNION ALL ...) GROUP BY x'; +SELECT x, SUM(y) FROM +(SELECT * FROM merge(currentDatabase(), '02763_merge.*1') +UNION ALL +SELECT * FROM merge(currentDatabase(), '02763_merge.*2')) +GROUP BY x +ORDER BY x; + DROP ROW POLICY 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY 02763_filter_2 ON 02763_merge_merge_1; From f4c77c1f1827c639aefc8e12cac78080b8f380cc Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 11 Jul 2023 11:46:37 +0000 Subject: [PATCH 035/813] merge_row_policy: namesDifference, try to handle nonselected columns --- src/Storages/StorageMerge.cpp | 75 ++++++++++++------- .../02763_row_policy_storage_merge.reference | 54 ++++++++++++- .../02763_row_policy_storage_merge.sql.j2 | 23 +++++- 3 files changed, 118 insertions(+), 34 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 483f592f819..ce1fdece231 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -66,6 +66,26 @@ bool columnDefaultKindHasSameType(ColumnDefaultKind lhs, ColumnDefaultKind rhs) return false; } +std::string namesDifference(Names && outer_set, Names && inner_set) +{ + std::sort(outer_set.begin(), outer_set.end()); + + std::sort(inner_set.begin(), inner_set.end()); + + Names result; + + std::set_difference(outer_set.begin(), outer_set.end(), + inner_set.begin(), inner_set.end(), std::inserter(result, result.begin())); + + if (result.size() != 1) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot determine row level filter"); + } + + return result.front(); +} + } namespace DB @@ -682,29 +702,22 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( { ASTPtr expr = row_policy_filter->expression; - auto syntax_result = TreeRewriter(modified_context).analyze(expr, header.getNamesAndTypesList()); + auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto storage_columns = storage_metadata_snapshot->getColumns(); + auto needed_columns = storage_columns.getAllPhysical(); // header.getNamesAndTypesList() + + auto syntax_result = TreeRewriter(modified_context).analyze(expr, needed_columns); auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, modified_context}; - auto filter_dag_ptr = expression_analyzer.getActionsDAG(true, false); - auto filter_actions = std::make_shared(filter_dag_ptr, ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + auto actions_dag = expression_analyzer.getActionsDAG(true, false); + auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); auto required_columns = filter_actions->getRequiredColumns(); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "filter_actions_dag: {},<> {}, <> {}", filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); - std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); - - Names required_columns_sorted = required_columns; - std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); - - Names filter_columns; - - std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), - required_columns.begin(), required_columns.end(), - std::inserter(filter_columns, filter_columns.begin())); - - source_step_with_filter->addFilter(filter_dag_ptr, filter_columns.front()); + auto filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); + source_step_with_filter->addFilter(actions_dag, filter_column_name); } } } @@ -1059,33 +1072,39 @@ void ReadFromMerge::convertingSourceStream( { ASTPtr expr = row_policy_filter->expression; - auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); + auto storage_columns = metadata_snapshot->getColumns(); + auto needed_columns = storage_columns.getAllPhysical(); // header.getNamesAndTypesList() + + + auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns /* pipe_columns */); auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, local_context}; auto actions_dag = expression_analyzer.getActionsDAG(true, false); auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - auto required_columns = filter_actions->getRequiredColumns(); LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - auto fa_actions_columns_sorted = filter_actions->getSampleBlock().getNames(); - std::sort(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end()); - Names required_columns_sorted = required_columns; - std::sort(required_columns_sorted.begin(), required_columns_sorted.end()); + for (auto & colname : filter_actions->getSampleBlock().getNames()) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions->getSampleBlock().getNames(): {}", colname); + } - Names filter_columns; + for (auto & colname : filter_actions->getRequiredColumns()) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions->getRequiredColumns(): {}", colname); + } - std::set_difference(fa_actions_columns_sorted.begin(), fa_actions_columns_sorted.end(), - required_columns.begin(), required_columns.end(), - std::inserter(filter_columns, filter_columns.begin())); + + auto filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); + builder.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared(stream_header, filter_actions, filter_columns.front(), true /* remove fake column */); + return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove fake column */); }); } } diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference index 444513c6c20..55890a11783 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference @@ -1,3 +1,9 @@ +SELECT * FROM 02763_merge_log_1 ORDER BY x +1 11 +2 12 +3 13 +4 14 +SELECT * FROM merge(currentDatabase(), 02763_merge) ORDER BY x 1 11 1 11 1 11 @@ -14,10 +20,6 @@ 4 14 4 14 4 14 -1 11 -2 12 -3 13 -4 14 SETTINGS optimize_move_to_prewhere= 0 SELECT * FROM 02763_merge_log_1 3 13 @@ -98,6 +100,28 @@ SELECT x, SUM(x) FROM (SELECT * FROM merge(...) UNION ALL ...) GROUP BY x 2 24 3 39 4 42 +1 11 0 +2 12 0 +3 13 0 +4 14 1 +4 14 1 +SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>1 -- with y>12 +2 12 +3 13 +3 13 +4 14 +4 14 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>1 -- with y>12 +2 12 +3 13 +3 13 +4 14 +4 14 +2 12 0 +3 13 1 +3 13 1 +4 14 1 +4 14 1 SETTINGS optimize_move_to_prewhere= 1 SELECT * FROM 02763_merge_log_1 3 13 @@ -178,3 +202,25 @@ SELECT x, SUM(x) FROM (SELECT * FROM merge(...) UNION ALL ...) GROUP BY x 2 24 3 39 4 42 +1 11 0 +2 12 0 +3 13 0 +4 14 1 +4 14 1 +SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>1 -- with y>12 +2 12 +3 13 +3 13 +4 14 +4 14 +SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>1 -- with y>12 +2 12 +3 13 +3 13 +4 14 +4 14 +2 12 0 +3 13 1 +3 13 1 +4 14 1 +4 14 1 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index b5094f927f4..94872dfd7b6 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -17,10 +17,12 @@ INSERT INTO 02763_merge_log_2 VALUES (1, 11), (2, 12), (3, 13), (4, 14); INSERT INTO 02763_merge_merge_1 VALUES (1, 11), (2, 12), (3, 13), (4, 14); INSERT INTO 02763_merge_merge_2 VALUES (1, 11), (2, 12), (3, 13), (4, 14); -SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x; - +SELECT 'SELECT * FROM 02763_merge_log_1 ORDER BY x'; SELECT * FROM 02763_merge_log_1 ORDER BY x; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge) ORDER BY x'; +SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x; + {% for prew in [0 , 1] -%} @@ -81,7 +83,24 @@ SELECT * FROM merge(currentDatabase(), '02763_merge.*2')) GROUP BY x ORDER BY x; +SELECT *, x=4 FROM merge(currentDatabase(), '02763_merge_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + + +CREATE ROW POLICY 02763_filter_3 ON 02763_merge_log_1 USING y>12 AS permissive TO ALL; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>1 -- with y>12'; +SELECT * FROM merge(currentDatabase(), '02763_merge_log') WHERE x>1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + +CREATE ROW POLICY 02763_filter_4 ON 02763_merge_merge_1 USING y>12 AS permissive TO ALL; +SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>1 -- with y>12'; +SELECT * FROM merge(currentDatabase(), '02763_merge_merge') WHERE x>1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + +SELECT *, (x=4 OR y>12) FROM merge(currentDatabase(), '02763_merge_merge') WHERE x>1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + + DROP ROW POLICY 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY 02763_filter_2 ON 02763_merge_merge_1; +DROP ROW POLICY 02763_filter_3 ON 02763_merge_log_1; +DROP ROW POLICY 02763_filter_4 ON 02763_merge_merge_1; + {% endfor %} From b57d8bc4a91eddcc41469cc21f983237797fa272 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 11 Jul 2023 21:16:54 +0000 Subject: [PATCH 036/813] merge_row_policy: works again with adding missed columns --- src/Storages/StorageMerge.cpp | 119 +++++++++++++++++++++------------- 1 file changed, 73 insertions(+), 46 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index ce1fdece231..920604c876a 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -681,6 +682,29 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( StorageView * view = dynamic_cast(storage.get()); if (!view || allow_experimental_analyzer) { + + auto row_policy_filter = modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); + if (row_policy_filter) + { + ASTPtr expr = row_policy_filter->expression; + + RequiredSourceColumnsVisitor::Data columns_context; + RequiredSourceColumnsVisitor(columns_context).visit(expr); + + auto req_columns = columns_context.requiredColumns(); + for (const auto & req_column : req_columns) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "req.column: {}", req_column); + std::sort(real_column_names.begin(), real_column_names.end()); + + if (!std::binary_search(real_column_names.begin(), real_column_names.end(), req_column)) + { + real_column_names.push_back(req_column); + } + } + } + + storage->read(plan, real_column_names, storage_snapshot, @@ -694,13 +718,12 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (!plan.isInitialized()) return {}; - if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) - { - auto row_policy_filter = modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); - if (row_policy_filter) + if (row_policy_filter) + { + ASTPtr expr = row_policy_filter->expression; + if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) { - ASTPtr expr = row_policy_filter->expression; auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_columns = storage_metadata_snapshot->getColumns(); @@ -713,12 +736,13 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); auto required_columns = filter_actions->getRequiredColumns(); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "filter_actions_dag: {},<> {}, <> {}", + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "filter_actions_dag: {},<> {}, <> {}", filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); auto filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); source_step_with_filter->addFilter(actions_dag, filter_column_name); } + } } else @@ -1026,46 +1050,6 @@ void ReadFromMerge::convertingSourceStream( const String & database_name, const String & table_name) { - Block before_block_header = builder.getHeader(); - - auto storage_sample_block = metadata_snapshot->getSampleBlock(); - auto pipe_columns = builder.getHeader().getNamesAndTypesList(); - - for (const auto & alias : aliases) - { - pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type)); - ASTPtr expr = alias.expression; - auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); - auto expression_analyzer = ExpressionAnalyzer{alias.expression, syntax_result, local_context}; - - 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); - }); - } - - ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; - - if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns) - convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; - - auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().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 row_policy_filter = local_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); if (row_policy_filter) @@ -1107,6 +1091,49 @@ void ReadFromMerge::convertingSourceStream( return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove fake column */); }); } + + + + Block before_block_header = builder.getHeader(); + + auto storage_sample_block = metadata_snapshot->getSampleBlock(); + auto pipe_columns = builder.getHeader().getNamesAndTypesList(); + + for (const auto & alias : aliases) + { + pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type)); + ASTPtr expr = alias.expression; + auto syntax_result = TreeRewriter(local_context).analyze(expr, pipe_columns); + auto expression_analyzer = ExpressionAnalyzer{alias.expression, syntax_result, local_context}; + + 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); + }); + } + + ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; + + if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns) + convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; + + auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().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); + }); + + } bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) From b39a201cd0ea81adcbe97943913d961e80cca088 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 12 Jul 2023 14:42:32 +0000 Subject: [PATCH 037/813] merge_row_policy: refactored with ReadFromMerge::RowPolicyData --- src/Storages/StorageMerge.cpp | 246 ++++++++++-------- src/Storages/StorageMerge.h | 5 +- .../02763_row_policy_storage_merge.reference | 26 ++ .../02763_row_policy_storage_merge.sql.j2 | 3 +- 4 files changed, 173 insertions(+), 107 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 920604c876a..0b6968f78ce 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -67,26 +67,6 @@ bool columnDefaultKindHasSameType(ColumnDefaultKind lhs, ColumnDefaultKind rhs) return false; } -std::string namesDifference(Names && outer_set, Names && inner_set) -{ - std::sort(outer_set.begin(), outer_set.end()); - - std::sort(inner_set.begin(), inner_set.end()); - - Names result; - - std::set_difference(outer_set.begin(), outer_set.end(), - inner_set.begin(), inner_set.end(), std::inserter(result, result.begin())); - - if (result.size() != 1) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot determine row level filter"); - } - - return result.front(); -} - } namespace DB @@ -395,6 +375,28 @@ void StorageMerge::read( query_plan.addStep(std::move(step)); } +class ReadFromMerge::RowPolicyData +{ +public: + bool needCare() + { + return static_cast(row_policy_filter_ptr); + } + void init(RowPolicyFilterPtr, + const std::shared_ptr, + ContextPtr); + void extendNames(Names &); + void addStorageFilter(SourceStepWithFilter *); + void addFilterTransform(QueryPipelineBuilder &); +private: + static std::string namesDifference(Names && outer_set, Names && inner_set); + RowPolicyFilterPtr row_policy_filter_ptr; + std::string filter_column_name; + ActionsDAGPtr actions_dag; + ExpressionActionsPtr filter_actions; +}; + + ReadFromMerge::ReadFromMerge( Block common_header_, StorageListWithLocks selected_tables_, @@ -671,6 +673,8 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage_snapshot, modified_query_info); + RowPolicyData row_policy_data; + if (processed_stage <= storage_stage || (allow_experimental_analyzer && processed_stage == QueryProcessingStage::FetchColumns)) { /// If there are only virtual columns in query, you must request at least one other column. @@ -682,29 +686,15 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( StorageView * view = dynamic_cast(storage.get()); if (!view || allow_experimental_analyzer) { + row_policy_data.init(modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER), + storage, + modified_context); - auto row_policy_filter = modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); - if (row_policy_filter) + if (row_policy_data.needCare()) { - ASTPtr expr = row_policy_filter->expression; - - RequiredSourceColumnsVisitor::Data columns_context; - RequiredSourceColumnsVisitor(columns_context).visit(expr); - - auto req_columns = columns_context.requiredColumns(); - for (const auto & req_column : req_columns) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "req.column: {}", req_column); - std::sort(real_column_names.begin(), real_column_names.end()); - - if (!std::binary_search(real_column_names.begin(), real_column_names.end(), req_column)) - { - real_column_names.push_back(req_column); - } - } + row_policy_data.extendNames(real_column_names); } - storage->read(plan, real_column_names, storage_snapshot, @@ -714,35 +704,16 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( max_block_size, UInt32(streams_num)); - if (!plan.isInitialized()) return {}; - if (row_policy_filter) + if (row_policy_data.needCare()) { - ASTPtr expr = row_policy_filter->expression; if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) { - - auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto storage_columns = storage_metadata_snapshot->getColumns(); - auto needed_columns = storage_columns.getAllPhysical(); // header.getNamesAndTypesList() - - auto syntax_result = TreeRewriter(modified_context).analyze(expr, needed_columns); - auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, modified_context}; - - auto actions_dag = expression_analyzer.getActionsDAG(true, false); - auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - auto required_columns = filter_actions->getRequiredColumns(); - - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - - auto filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); - source_step_with_filter->addFilter(actions_dag, filter_column_name); + row_policy_data.addStorageFilter(source_step_with_filter); } - } } else @@ -864,12 +835,119 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( /// 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. - convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, *builder, processed_stage, database_name, table_name); + convertingSourceStream(header, + storage_snapshot->metadata, + aliases, + modified_context, + *builder, + processed_stage, + row_policy_data); } return builder; } +void ReadFromMerge::RowPolicyData::init(RowPolicyFilterPtr row_policy_filter_ptr_, + const std::shared_ptr storage, + ContextPtr local_context) +{ + + if (row_policy_filter_ptr_) + { + row_policy_filter_ptr = row_policy_filter_ptr_; + + ASTPtr expr = row_policy_filter_ptr->expression; + + auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto storage_columns = storage_metadata_snapshot->getColumns(); + auto needed_columns = storage_columns.getAllPhysical(); // header.getNamesAndTypesList() + + + auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns /* pipe_columns */); + auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; + + actions_dag = expression_analyzer.getActionsDAG(true, false); + filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); + filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); + } + +} + +void ReadFromMerge::RowPolicyData::extendNames(Names & names) +{ + assert(row_policy_filter_ptr); + ASTPtr expr = row_policy_filter_ptr->expression; + + RequiredSourceColumnsVisitor::Data columns_context; + RequiredSourceColumnsVisitor(columns_context).visit(expr); + + auto req_columns = columns_context.requiredColumns(); + for (const auto & req_column : req_columns) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), "req.column: {}", req_column); + std::sort(names.begin(), names.end()); + + if (!std::binary_search(names.begin(), names.end(), req_column)) + { + names.push_back(req_column); + } + } +} + +void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) +{ + assert(row_policy_filter_ptr); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); + step->addFilter(actions_dag, filter_column_name); +} + +void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & builder) +{ + assert(row_policy_filter_ptr); + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); + + + for (auto & colname : filter_actions->getSampleBlock().getNames()) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "filter_actions->getSampleBlock().getNames(): {}", colname); + } + + for (auto & colname : filter_actions->getRequiredColumns()) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "filter_actions->getRequiredColumns(): {}", colname); + } + + // auto filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); + + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove fake column */); + }); +} + +std::string ReadFromMerge::RowPolicyData::namesDifference(Names && outer_set, Names && inner_set) +{ + std::sort(outer_set.begin(), outer_set.end()); + + std::sort(inner_set.begin(), inner_set.end()); + + Names result; + + std::set_difference(outer_set.begin(), outer_set.end(), + inner_set.begin(), inner_set.end(), std::inserter(result, result.begin())); + + if (result.size() != 1) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot determine row level filter"); + } + + return result.front(); +} + StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( ContextPtr query_context, const ASTPtr & query /* = nullptr */, @@ -1047,53 +1125,13 @@ void ReadFromMerge::convertingSourceStream( ContextPtr local_context, QueryPipelineBuilder & builder, const QueryProcessingStage::Enum & processed_stage, - const String & database_name, - const String & table_name) + RowPolicyData & row_policy_data) { - auto row_policy_filter = local_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER); - - if (row_policy_filter) + if (row_policy_data.needCare()) { - ASTPtr expr = row_policy_filter->expression; - - auto storage_columns = metadata_snapshot->getColumns(); - auto needed_columns = storage_columns.getAllPhysical(); // header.getNamesAndTypesList() - - - auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns /* pipe_columns */); - auto expression_analyzer = ExpressionAnalyzer{row_policy_filter->expression, syntax_result, local_context}; - - auto actions_dag = expression_analyzer.getActionsDAG(true, false); - auto filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - - - - for (auto & colname : filter_actions->getSampleBlock().getNames()) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions->getSampleBlock().getNames(): {}", colname); - } - - for (auto & colname : filter_actions->getRequiredColumns()) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions->getRequiredColumns(): {}", colname); - } - - - - auto filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); - - - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove fake column */); - }); + row_policy_data.addFilterTransform(builder); } - - Block before_block_header = builder.getHeader(); auto storage_sample_block = metadata_snapshot->getSampleBlock(); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index fbe6dcec298..4dc7cb77c48 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -177,6 +177,8 @@ private: using Aliases = std::vector; + class RowPolicyData; + static SelectQueryInfo getModifiedQueryInfo(const SelectQueryInfo & query_info, const ContextPtr & modified_context, const StorageWithLockAndName & storage_with_lock_and_name, @@ -202,8 +204,7 @@ private: ContextPtr context, QueryPipelineBuilder & builder, const QueryProcessingStage::Enum & processed_stage, - const String & database_name, - const String & table_name); + RowPolicyData & row_policy_data); }; } diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference index 55890a11783..070520bb392 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference @@ -122,6 +122,19 @@ SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>1 -- with y>12 3 13 1 4 14 1 4 14 1 +SELECT y from merge(currentDatabase(), 02763_merge) +11 +11 +12 +12 +13 +13 +13 +13 +14 +14 +14 +14 SETTINGS optimize_move_to_prewhere= 1 SELECT * FROM 02763_merge_log_1 3 13 @@ -224,3 +237,16 @@ SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>1 -- with y>12 3 13 1 4 14 1 4 14 1 +SELECT y from merge(currentDatabase(), 02763_merge) +11 +11 +12 +12 +13 +13 +13 +13 +14 +14 +14 +14 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index 94872dfd7b6..3030be2539f 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -85,7 +85,6 @@ ORDER BY x; SELECT *, x=4 FROM merge(currentDatabase(), '02763_merge_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; - CREATE ROW POLICY 02763_filter_3 ON 02763_merge_log_1 USING y>12 AS permissive TO ALL; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_log) WHERE x>1 -- with y>12'; SELECT * FROM merge(currentDatabase(), '02763_merge_log') WHERE x>1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; @@ -96,6 +95,8 @@ SELECT * FROM merge(currentDatabase(), '02763_merge_merge') WHERE x>1 ORDER BY x SELECT *, (x=4 OR y>12) FROM merge(currentDatabase(), '02763_merge_merge') WHERE x>1 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT y from merge(currentDatabase(), 02763_merge)'; +SELECT y from merge(currentDatabase(), '02763_merge') ORDER BY y SETTINGS optimize_move_to_prewhere= {{prew}}; DROP ROW POLICY 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY 02763_filter_2 ON 02763_merge_merge_1; From f89a538f314f01ef8eeb056afb0c6d4b381e89c7 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 12 Jul 2023 15:11:59 +0000 Subject: [PATCH 038/813] merge_row_policy: stylecheck --- src/Storages/StorageMerge.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0b6968f78ce..76c7afc5b73 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -937,7 +937,7 @@ std::string ReadFromMerge::RowPolicyData::namesDifference(Names && outer_set, Na Names result; std::set_difference(outer_set.begin(), outer_set.end(), - inner_set.begin(), inner_set.end(), std::inserter(result, result.begin())); + inner_set.begin(), inner_set.end(), std::inserter(result, result.begin())); if (result.size() != 1) { From 6161116bf2ce1b271fa327181d5017e637b25424 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 12 Jul 2023 15:52:48 +0000 Subject: [PATCH 039/813] merge_row_policy: stylecheck again --- src/Storages/StorageMerge.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 76c7afc5b73..934a72684d2 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -81,6 +81,7 @@ namespace ErrorCodes extern const int SAMPLING_NOT_SUPPORTED; extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int LOGICAL_ERROR; } StorageMerge::StorageMerge( From 57f1e5627658f71e5e5dcfec17dadd5afeeaeac0 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 14 Jul 2023 13:39:43 +0000 Subject: [PATCH 040/813] merge_row_policy: cleanup, comments, new tests --- src/Storages/StorageMerge.cpp | 78 ++++++++++--------- .../02763_row_policy_storage_merge.reference | 54 +++++++++++++ .../02763_row_policy_storage_merge.sql.j2 | 25 ++++++ 3 files changed, 121 insertions(+), 36 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 934a72684d2..df353fa3158 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -268,7 +268,6 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( /// (see removeJoin()) /// /// And for this we need to return FetchColumns. - if (const auto * select = query_info.query->as(); select && hasJoin(*select)) return QueryProcessingStage::FetchColumns; @@ -315,6 +314,7 @@ void StorageMerge::read( * since there is no certainty that it works when one of table is MergeTree and other is not. */ auto modified_context = Context::createCopy(local_context); + modified_context->setSetting("optimize_move_to_prewhere", false); bool has_database_virtual_column = false; bool has_table_virtual_column = false; @@ -376,9 +376,13 @@ void StorageMerge::read( query_plan.addStep(std::move(step)); } +/// A transient object of this helper class is created +/// when processing a Merge table data source (subordinary table) +/// to guarantee that row policies are applied class ReadFromMerge::RowPolicyData { public: + /// Row policy requires extra filtering bool needCare() { return static_cast(row_policy_filter_ptr); @@ -386,18 +390,28 @@ public: void init(RowPolicyFilterPtr, const std::shared_ptr, ContextPtr); + + /// Add columns that needed for row policies to data stream + /// SELECT x from T if T has row policy y=42 + /// required y in data pipeline void extendNames(Names &); + + /// Use storage facilities to filter data + /// does not guarantee accuracy, but reduce number of rows void addStorageFilter(SourceStepWithFilter *); + + /// Create explicit filter transform to stop + /// rows that are not conform to row level policy void addFilterTransform(QueryPipelineBuilder &); + private: static std::string namesDifference(Names && outer_set, Names && inner_set); RowPolicyFilterPtr row_policy_filter_ptr; - std::string filter_column_name; + std::string filter_column_name; // complex filer, may contain logic operations ActionsDAGPtr actions_dag; ExpressionActionsPtr filter_actions; }; - ReadFromMerge::ReadFromMerge( Block common_header_, StorageListWithLocks selected_tables_, @@ -687,7 +701,11 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( StorageView * view = dynamic_cast(storage.get()); if (!view || allow_experimental_analyzer) { - row_policy_data.init(modified_context->getRowPolicyFilter(database_name, table_name, RowPolicyFilterType::SELECT_FILTER), + row_policy_data.init( + modified_context->getRowPolicyFilter( + database_name, + table_name, + RowPolicyFilterType::SELECT_FILTER), storage, modified_context); @@ -708,7 +726,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (!plan.isInitialized()) return {}; - if (row_policy_data.needCare()) { if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) @@ -742,14 +759,12 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( { size_t filters_dags_size = filter_dags.size(); for (size_t i = 0; i < filters_dags_size; ++i) - { read_from_merge_tree->addFilter(filter_dags[i], filter_nodes.nodes[i]); - } } + builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(modified_context), BuildQueryPipelineSettings::fromContext(modified_context)); - } else if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { @@ -835,6 +850,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( /// 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. + /// Besides this we add FilterTransform if it is needed to follow row level policies. convertingSourceStream(header, storage_snapshot->metadata, @@ -852,7 +868,6 @@ void ReadFromMerge::RowPolicyData::init(RowPolicyFilterPtr row_policy_filter_ptr const std::shared_ptr storage, ContextPtr local_context) { - if (row_policy_filter_ptr_) { row_policy_filter_ptr = row_policy_filter_ptr_; @@ -861,19 +876,20 @@ void ReadFromMerge::RowPolicyData::init(RowPolicyFilterPtr row_policy_filter_ptr auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_columns = storage_metadata_snapshot->getColumns(); - auto needed_columns = storage_columns.getAllPhysical(); // header.getNamesAndTypesList() + auto needed_columns = storage_columns.getAllPhysical(); - - auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns /* pipe_columns */); + auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns); auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(true, false); - filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); + filter_actions = std::make_shared(actions_dag, + ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); } - } +// Add columns that needed to evaluate row policies +// SELECT x from t if t has row policy void ReadFromMerge::RowPolicyData::extendNames(Names & names) { assert(row_policy_filter_ptr); @@ -885,7 +901,6 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) auto req_columns = columns_context.requiredColumns(); for (const auto & req_column : req_columns) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), "req.column: {}", req_column); std::sort(names.begin(), names.end()); if (!std::binary_search(names.begin(), names.end(), req_column)) @@ -898,30 +913,22 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) { assert(row_policy_filter_ptr); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::createSources"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - step->addFilter(actions_dag, filter_column_name); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::addStorageFilter"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), + filter_actions->getActionsDAG().dumpDAG(), + filter_actions->getSampleBlock().dumpStructure()); + + step->addFilter(actions_dag, filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & builder) { assert(row_policy_filter_ptr); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertinfSourceStream"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), filter_actions->getSampleBlock().dumpStructure()); - - - for (auto & colname : filter_actions->getSampleBlock().getNames()) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "filter_actions->getSampleBlock().getNames(): {}", colname); - } - - for (auto & colname : filter_actions->getRequiredColumns()) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), "filter_actions->getRequiredColumns(): {}", colname); - } - - // auto filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::addFilterTransform"), "filter_actions_dag: {},<> {}, <> {}", + filter_actions->getActionsDAG().dumpNames(), + filter_actions->getActionsDAG().dumpDAG(), + filter_actions->getSampleBlock().dumpStructure()); builder.addSimpleTransform([&](const Block & stream_header) { @@ -929,10 +936,10 @@ void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & bui }); } +/// Find out an item that in outer_set vector, but not in inner_set vector std::string ReadFromMerge::RowPolicyData::namesDifference(Names && outer_set, Names && inner_set) { std::sort(outer_set.begin(), outer_set.end()); - std::sort(inner_set.begin(), inner_set.end()); Names result; @@ -949,6 +956,7 @@ std::string ReadFromMerge::RowPolicyData::namesDifference(Names && outer_set, Na return result.front(); } + StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( ContextPtr query_context, const ASTPtr & query /* = nullptr */, @@ -1171,8 +1179,6 @@ void ReadFromMerge::convertingSourceStream( { return std::make_shared(stream_header, actions); }); - - } bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference index 070520bb392..9c91a1652c7 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference @@ -135,6 +135,33 @@ SELECT y from merge(currentDatabase(), 02763_merge) 14 14 14 +02763_merge_fancycols +SELECT * +SELECT x, lc +SELECT * +1 11 111 111 42 +1 11 111 111 42 +SELECT x, lc +1 111 +1 111 +SELECT x, lc, cnst +1 111 42 +1 111 42 +SELECT x, y from merge(currentDatabase(), 02763_merge +1 11 +1 11 +1 11 +1 11 +2 12 +2 12 +3 13 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 +4 14 SETTINGS optimize_move_to_prewhere= 1 SELECT * FROM 02763_merge_log_1 3 13 @@ -250,3 +277,30 @@ SELECT y from merge(currentDatabase(), 02763_merge) 14 14 14 +02763_merge_fancycols +SELECT * +SELECT x, lc +SELECT * +1 11 111 111 42 +1 11 111 111 42 +SELECT x, lc +1 111 +1 111 +SELECT x, lc, cnst +1 111 42 +1 111 42 +SELECT x, y from merge(currentDatabase(), 02763_merge +1 11 +1 11 +1 11 +1 11 +2 12 +2 12 +3 13 +3 13 +3 13 +3 13 +4 14 +4 14 +4 14 +4 14 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index 3030be2539f..ea2cf1fef00 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -98,10 +98,35 @@ SELECT *, (x=4 OR y>12) FROM merge(currentDatabase(), '02763_merge_merge') WHERE SELECT 'SELECT y from merge(currentDatabase(), 02763_merge)'; SELECT y from merge(currentDatabase(), '02763_merge') ORDER BY y SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT '02763_merge_fancycols'; +CREATE OR REPLACE TABLE 02763_merge_fancycols (x UInt8, y Nullable(UInt64), z String DEFAULT CONCAT(toString(x), toString(y)), lc LowCardinality(String) DEFAULT z, cnst UInt32 MATERIALIZED 42) ENGINE = MergeTree() ORDER BY tuple(); +INSERT INTO 02763_merge_fancycols (x, y) SELECT x, y from merge(currentDatabase(), '02763_merge'); + +CREATE ROW POLICY 02763_filter_5 ON 02763_merge_fancycols USING cnst<>42 AS permissive TO ALL; +SELECT 'SELECT *'; +SELECT * from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT 'SELECT x, lc'; +SELECT x, lc from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; + +CREATE ROW POLICY 02763_filter_6 ON 02763_merge_fancycols USING lc='111' AS permissive TO ALL; +SELECT 'SELECT *'; +SELECT * from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT 'SELECT x, lc'; +SELECT x, lc from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT 'SELECT x, lc, cnst'; +SELECT x, lc, cnst from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT 'SELECT x, y from merge(currentDatabase(), 02763_merge'; +SELECT x, y from merge(currentDatabase(), '02763_merge') ORDER BY x; + +DROP TABLE 02763_merge_fancycols; + DROP ROW POLICY 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY 02763_filter_2 ON 02763_merge_merge_1; DROP ROW POLICY 02763_filter_3 ON 02763_merge_log_1; DROP ROW POLICY 02763_filter_4 ON 02763_merge_merge_1; +DROP ROW POLICY 02763_filter_5 ON 02763_merge_fancycols; +DROP ROW POLICY 02763_filter_6 ON 02763_merge_fancycols; + {% endfor %} From 9a561cff8258a878271d289a3d74f1a014d2419b Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 17 Jul 2023 12:10:27 +0000 Subject: [PATCH 041/813] merge_row_policy: cleanup, verbose short messages in 00002_log ... --- src/Storages/StorageMerge.cpp | 13 ++++++------- src/Storages/StorageMerge.h | 4 ++-- .../00002_log_and_exception_messages_formatting.sql | 3 ++- .../02763_row_policy_storage_merge.sql.j2 | 7 ++++++- 4 files changed, 16 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index df353fa3158..5bbc6c9a03c 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -387,9 +387,8 @@ public: { return static_cast(row_policy_filter_ptr); } - void init(RowPolicyFilterPtr, - const std::shared_ptr, - ContextPtr); + + void init(RowPolicyFilterPtr, std::shared_ptr, ContextPtr); /// Add columns that needed for row policies to data stream /// SELECT x from T if T has row policy y=42 @@ -660,8 +659,8 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer QueryPipelineBuilderPtr ReadFromMerge::createSources( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & modified_query_info, - const QueryProcessingStage::Enum & processed_stage, - const UInt64 max_block_size, + QueryProcessingStage::Enum processed_stage, + UInt64 max_block_size, const Block & header, const Aliases & aliases, const StorageWithLockAndName & storage_with_lock, @@ -865,7 +864,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( } void ReadFromMerge::RowPolicyData::init(RowPolicyFilterPtr row_policy_filter_ptr_, - const std::shared_ptr storage, + std::shared_ptr storage, ContextPtr local_context) { if (row_policy_filter_ptr_) @@ -1133,7 +1132,7 @@ void ReadFromMerge::convertingSourceStream( const Aliases & aliases, ContextPtr local_context, QueryPipelineBuilder & builder, - const QueryProcessingStage::Enum & processed_stage, + QueryProcessingStage::Enum processed_stage, RowPolicyData & row_policy_data) { if (row_policy_data.needCare()) diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 4dc7cb77c48..1a5aca24e4c 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -187,7 +187,7 @@ private: QueryPipelineBuilderPtr createSources( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - const QueryProcessingStage::Enum & processed_stage, + QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, const Block & header, const Aliases & aliases, @@ -203,7 +203,7 @@ private: const Aliases & aliases, ContextPtr context, QueryPipelineBuilder & builder, - const QueryProcessingStage::Enum & processed_stage, + QueryProcessingStage::Enum processed_stage, RowPolicyData & row_policy_data); }; diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index f4ec9b79a4c..19b68873a10 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -52,7 +52,8 @@ create temporary table known_short_messages (s String) as select * from (select select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_string), 1) from logs where length(message_format_string) < 10 and message_format_string not in known_short_messages; -- Same as above. Feel free to update the threshold or remove this query if really necessary -select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 3) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; +-- select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 3) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; +select message_format_string from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; -- Unlike above, here we look at length of the formatted message, not format string. Most short format strings are fine because they end up decorated with context from outer or inner exceptions, e.g.: -- "Expected end of line" -> "Code: 117. DB::Exception: Expected end of line: (in file/uri /var/lib/clickhouse/user_files/data_02118): (at row 1)" diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index ea2cf1fef00..9b3197cc34f 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -2,8 +2,13 @@ DROP TABLE IF EXISTS 02763_merge_log_1; DROP TABLE IF EXISTS 02763_merge_log_2; DROP TABLE IF EXISTS 02763_merge_merge_1; DROP TABLE IF EXISTS 02763_merge_merge_2; +DROP TABLE IF EXISTS 02763_merge_fancycols; DROP ROW POLICY IF EXISTS 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY IF EXISTS 02763_filter_2 ON 02763_merge_merge_1; +DROP ROW POLICY IF EXISTS 02763_filter_3 ON 02763_merge_log_1; +DROP ROW POLICY IF EXISTS 02763_filter_4 ON 02763_merge_merge_1; +DROP ROW POLICY IF EXISTS 02763_filter_5 ON 02763_merge_fancycols; +DROP ROW POLICY IF EXISTS 02763_filter_6 ON 02763_merge_fancycols; CREATE TABLE 02763_merge_log_1 (x UInt8, y UInt64) ENGINE = Log; @@ -99,7 +104,7 @@ SELECT 'SELECT y from merge(currentDatabase(), 02763_merge)'; SELECT y from merge(currentDatabase(), '02763_merge') ORDER BY y SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT '02763_merge_fancycols'; -CREATE OR REPLACE TABLE 02763_merge_fancycols (x UInt8, y Nullable(UInt64), z String DEFAULT CONCAT(toString(x), toString(y)), lc LowCardinality(String) DEFAULT z, cnst UInt32 MATERIALIZED 42) ENGINE = MergeTree() ORDER BY tuple(); +CREATE TABLE 02763_merge_fancycols (x UInt8, y Nullable(UInt64), z String DEFAULT CONCAT(toString(x), toString(y)), lc LowCardinality(String) DEFAULT z, cnst UInt32 MATERIALIZED 42) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO 02763_merge_fancycols (x, y) SELECT x, y from merge(currentDatabase(), '02763_merge'); CREATE ROW POLICY 02763_filter_5 ON 02763_merge_fancycols USING cnst<>42 AS permissive TO ALL; From da3419068d5da01f5f9daca1b7e00ce8a55685a8 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 19 Jul 2023 21:32:52 +0000 Subject: [PATCH 042/813] merge_row_policy: more verbosity for 00002_log_and_exception_... --- .../00002_log_and_exception_messages_formatting.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 19b68873a10..30814b791ed 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -53,7 +53,8 @@ select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_st -- Same as above. Feel free to update the threshold or remove this query if really necessary -- select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 3) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; -select message_format_string from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; +select 'from logs', message_format_string from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; +select 'from system.text_log', message_format_string from system.text_log where length(message_format_string) < 16 and message_format_string not in known_short_messages and now() - toIntervalMinute(120) < event_time; -- Unlike above, here we look at length of the formatted message, not format string. Most short format strings are fine because they end up decorated with context from outer or inner exceptions, e.g.: -- "Expected end of line" -> "Code: 117. DB::Exception: Expected end of line: (in file/uri /var/lib/clickhouse/user_files/data_02118): (at row 1)" From ea48679185c1de20ea492c9e5afaf0d873b7f051 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 11 Sep 2023 15:07:10 +0000 Subject: [PATCH 043/813] merge_row_policy: revert debug, 00002_log... test is fixed in master --- .../00002_log_and_exception_messages_formatting.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 30814b791ed..f4ec9b79a4c 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -52,9 +52,7 @@ create temporary table known_short_messages (s String) as select * from (select select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_string), 1) from logs where length(message_format_string) < 10 and message_format_string not in known_short_messages; -- Same as above. Feel free to update the threshold or remove this query if really necessary --- select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 3) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; -select 'from logs', message_format_string from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; -select 'from system.text_log', message_format_string from system.text_log where length(message_format_string) < 16 and message_format_string not in known_short_messages and now() - toIntervalMinute(120) < event_time; +select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 3) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; -- Unlike above, here we look at length of the formatted message, not format string. Most short format strings are fine because they end up decorated with context from outer or inner exceptions, e.g.: -- "Expected end of line" -> "Code: 117. DB::Exception: Expected end of line: (in file/uri /var/lib/clickhouse/user_files/data_02118): (at row 1)" From 3480243fa3b07a2df8638da527b8ef4fa9a378e9 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 4 Oct 2023 08:31:12 +0000 Subject: [PATCH 044/813] merge_row_policy: per code review --- src/Storages/StorageMerge.cpp | 26 ++++++++++++++++---------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5bbc6c9a03c..354e17ad5ba 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -383,7 +383,7 @@ class ReadFromMerge::RowPolicyData { public: /// Row policy requires extra filtering - bool needCare() + bool hasRowPolicy() { return static_cast(row_policy_filter_ptr); } @@ -708,7 +708,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage, modified_context); - if (row_policy_data.needCare()) + if (row_policy_data.hasRowPolicy()) { row_policy_data.extendNames(real_column_names); } @@ -725,7 +725,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (!plan.isInitialized()) return {}; - if (row_policy_data.needCare()) + if (row_policy_data.hasRowPolicy()) { if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) { @@ -880,7 +880,7 @@ void ReadFromMerge::RowPolicyData::init(RowPolicyFilterPtr row_policy_filter_ptr auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns); auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; - actions_dag = expression_analyzer.getActionsDAG(true, false); + actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); @@ -897,16 +897,22 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) RequiredSourceColumnsVisitor::Data columns_context; RequiredSourceColumnsVisitor(columns_context).visit(expr); - auto req_columns = columns_context.requiredColumns(); + const auto req_columns = columns_context.requiredColumns(); + + std::sort(names.begin(), names.end()); + NameSet added_names; + for (const auto & req_column : req_columns) { - std::sort(names.begin(), names.end()); - if (!std::binary_search(names.begin(), names.end(), req_column)) { - names.push_back(req_column); + added_names.insert(req_column); } } + if (!added_names.empty()) + { + std::copy(added_names.begin(), added_names.end(), std::back_inserter(names)); + } } void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) @@ -931,7 +937,7 @@ void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & bui builder.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove fake column */); + return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove filter column */); }); } @@ -1135,7 +1141,7 @@ void ReadFromMerge::convertingSourceStream( QueryProcessingStage::Enum processed_stage, RowPolicyData & row_policy_data) { - if (row_policy_data.needCare()) + if (row_policy_data.hasRowPolicy()) { row_policy_data.addFilterTransform(builder); } From c7a3c74cde8d6dd68b23fb318827d0eaadf8f292 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Oct 2023 15:20:31 +0200 Subject: [PATCH 045/813] Better --- src/Core/Settings.h | 2 +- src/Interpreters/Cache/FileCache.cpp | 68 ++++++++++++++++++++------ src/Interpreters/Cache/FileCache.h | 3 +- src/Interpreters/Cache/FileCache_fwd.h | 2 +- 4 files changed, 58 insertions(+), 17 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b9aa678a3c5..485b96b2341 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -719,7 +719,7 @@ class IColumn; M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \ M(UInt64, filesystem_cache_max_download_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be downloaded by a single query", 0) \ M(Bool, throw_on_error_from_cache_on_write_operations, false, "Ignore error from cache when caching on write operations (INSERT, merges)", 0) \ - M(UInt64, filesystem_cache_getorset_batch_size, 100, "A batch size for holding file segments for a single read range", 0) \ + M(UInt64, filesystem_cache_getorset_batch_size, 20, "A batch size for holding file segments for a single read range", 0) \ \ M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \ M(Bool, enable_filesystem_read_prefetches_log, false, "Log to system.filesystem prefetch_log during query. Should be used only for testing or debugging, not recommended to be turned on by default", 0) \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index bef1f3086df..576aab31adc 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -147,7 +147,7 @@ CacheGuard::Lock FileCache::lockCache() const return cache_guard.lock(); } -FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment::Range & range) const +FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment::Range & range, size_t file_segments_limit) const { /// Given range = [left, right] and non-overlapping ordered set of file segments, /// find list [segment1, ..., segmentN] of segments which intersect with given range. @@ -166,6 +166,9 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: FileSegments result; auto add_to_result = [&](const FileSegmentMetadata & file_segment_metadata) { + if (file_segments_limit && result.size() == file_segments_limit) + return false; + FileSegmentPtr file_segment; if (!file_segment_metadata.evicting()) { @@ -181,6 +184,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: } result.push_back(file_segment); + return true; }; auto segment_it = file_segments.lower_bound(range.left); @@ -197,7 +201,8 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: if (file_segment_metadata.file_segment->range().right < range.left) return {}; - add_to_result(file_segment_metadata); + if (!add_to_result(file_segment_metadata)) + return result; } else /// segment_it <-- segmment{k} { @@ -213,7 +218,8 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: /// [___________ /// ^ /// range.left - add_to_result(prev_file_segment_metadata); + if (!add_to_result(prev_file_segment_metadata)) + return result; } } @@ -229,7 +235,8 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: if (range.right < file_segment_metadata.file_segment->range().left) break; - add_to_result(file_segment_metadata); + if (!add_to_result(file_segment_metadata)) + return result; ++segment_it; } } @@ -273,6 +280,7 @@ void FileCache::fillHolesWithEmptyFileSegments( LockedKey & locked_key, FileSegments & file_segments, const FileSegment::Range & range, + size_t file_segments_limit, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & settings) { @@ -338,6 +346,9 @@ void FileCache::fillHolesWithEmptyFileSegments( ++it; } + if (file_segments.size() >= file_segments_limit) + return; + if (current_pos <= range.right) { /// ________] -- requested range @@ -374,7 +385,7 @@ FileSegmentsHolderPtr FileCache::set( auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY); FileSegment::Range range(offset, offset + size - 1); - auto file_segments = getImpl(*locked_key, range); + auto file_segments = getImpl(*locked_key, range, /* file_segments_limit */0); if (!file_segments.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Having intersection with already existing cache"); @@ -416,19 +427,46 @@ FileCache::getOrSet( auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY); /// Get all segments which intersect with the given range. - auto file_segments = getImpl(*locked_key, range); + auto file_segments = getImpl(*locked_key, range, file_segments_limit); + + bool limit_reached = false; if (file_segments.empty()) { file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, settings); + + while (!file_segments.empty() && file_segments.front()->range().right < offset) + file_segments.pop_front(); } else { - fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, /* fill_with_detached */false, settings); - } + limit_reached = file_segments_limit && file_segments.size() >= file_segments_limit; - while (!file_segments.empty() && file_segments.front()->range().right < offset) - file_segments.pop_front(); + /// A while loop for the case if we set a limit to n, but all these n file segments are removed + /// as they turned out redundant because of the alignment of offset to aligned_offset. + while (true) + { + size_t last_offset = file_segments.back()->range().right; + + while (!file_segments.empty() && file_segments.front()->range().right < offset) + file_segments.pop_front(); + + if (!file_segments.empty()) + break; + + if (!limit_reached) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of file segments"); + + range.left = std::min(offset, last_offset + 1); + file_segments = getImpl(*locked_key, range, file_segments_limit); + } + + range.left = std::min(offset, file_segments.front()->range().left); + if (limit_reached) + range.right = file_segments.back()->range().right; + + fillHolesWithEmptyFileSegments( + *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, settings); + } while (!file_segments.empty() && file_segments.back()->range().left >= offset + size) file_segments.pop_back(); @@ -439,7 +477,9 @@ FileCache::getOrSet( file_segments.pop_back(); } - chassert(!file_segments.empty()); + if (file_segments.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of file segments for offset {}, size {} (file size: {})", offset, size, file_size); + return std::make_unique(std::move(file_segments)); } @@ -455,11 +495,11 @@ FileSegmentsHolderPtr FileCache::get(const Key & key, size_t offset, size_t size FileSegment::Range range(offset, offset + size - 1); /// Get all segments which intersect with the given range. - auto file_segments = getImpl(*locked_key, range); + auto file_segments = getImpl(*locked_key, range, file_segments_limit); if (!file_segments.empty()) { fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, /* fill_with_detached */true, CreateFileSegmentSettings{}); + *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */true, CreateFileSegmentSettings{}); if (file_segments_limit) { diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 1a1a25cd9c1..14f27a69a68 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -215,7 +215,7 @@ private: void loadMetadataImpl(); void loadMetadataForKeys(const std::filesystem::path & keys_dir); - FileSegments getImpl(const LockedKey & locked_key, const FileSegment::Range & range) const; + FileSegments getImpl(const LockedKey & locked_key, const FileSegment::Range & range, size_t file_segments_limit) const; FileSegments splitRangeIntoFileSegments( LockedKey & locked_key, @@ -228,6 +228,7 @@ private: LockedKey & locked_key, FileSegments & file_segments, const FileSegment::Range & range, + size_t file_segments_limit, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & settings); diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 1f61617668e..3e7150ad253 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -4,7 +4,7 @@ namespace DB { -static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 16 * 1024 * 1024; /// 16Mi +static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi static constexpr int FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT = 4 * 1024 * 1024; /// 4Mi static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 2; static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 1; From d7c5caef927d4e0ca2091f483e824a1fbddc3909 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 7 Oct 2023 15:23:32 +0200 Subject: [PATCH 046/813] Better --- src/Interpreters/Cache/FileCache.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 576aab31adc..39cef20829e 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -456,7 +456,8 @@ FileCache::getOrSet( if (!limit_reached) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of file segments"); - range.left = std::min(offset, last_offset + 1); + range.left = last_offset + 1; + chassert(offset >= range.left); file_segments = getImpl(*locked_key, range, file_segments_limit); } From f31a6f2c0db3e1012394c1e7454c6ccbb2f5cf21 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 12 Oct 2023 14:49:16 +0000 Subject: [PATCH 047/813] Support only Full Sort --- src/Processors/QueryPlan/SortingStep.cpp | 8 +- src/Processors/QueryPlan/SortingStep.h | 17 ++- .../02884_parallel_window_functions.reference | 100 +++++++++++++++ .../02884_parallel_window_functions.sql | 119 ++++++++++++++++++ 4 files changed, 234 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02884_parallel_window_functions.reference create mode 100644 tests/queries/0_stateless/02884_parallel_window_functions.sql diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index ec32f6f6a28..e94e818a4aa 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -193,7 +193,7 @@ void SortingStep::scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline) for (size_t i = 0; i < threads; ++i) { size_t output_it = i; - auto resize = std::make_shared(ports[output_it]->getHeader(), streams, 1); + auto resize = std::make_shared(stream_header, streams, 1); auto & inputs = resize->getInputs(); for (auto input_it = inputs.begin(); input_it != inputs.end(); output_it += threads, ++input_it) @@ -367,11 +367,7 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build if (type == Type::FinishSorting) { bool need_finish_sorting = (prefix_description.size() < result_description.size()); - - if (partition_by_description.empty()) - mergingSorted(pipeline, prefix_description, (need_finish_sorting ? 0 : limit)); - - scatterByPartitionIfNeeded(pipeline); + mergingSorted(pipeline, prefix_description, (need_finish_sorting ? 0 : limit)); if (need_finish_sorting) { diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index ede91852aff..bdfc96f7931 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -97,12 +97,21 @@ private: void scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline); void updateOutputStream() override; - static void - mergeSorting(QueryPipelineBuilder & pipeline, const Settings & sort_settings, const SortDescription & result_sort_desc, UInt64 limit_); + static void mergeSorting( + QueryPipelineBuilder & pipeline, + const Settings & sort_settings, + const SortDescription & result_sort_desc, + UInt64 limit_); - void mergingSorted(QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, UInt64 limit_); + void mergingSorted( + QueryPipelineBuilder & pipeline, + const SortDescription & result_sort_desc, + UInt64 limit_); void finishSorting( - QueryPipelineBuilder & pipeline, const SortDescription & input_sort_desc, const SortDescription & result_sort_desc, UInt64 limit_); + QueryPipelineBuilder & pipeline, + const SortDescription & input_sort_desc, + const SortDescription & result_sort_desc, + UInt64 limit_); void fullSort( QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.reference b/tests/queries/0_stateless/02884_parallel_window_functions.reference new file mode 100644 index 00000000000..cab6195b625 --- /dev/null +++ b/tests/queries/0_stateless/02884_parallel_window_functions.reference @@ -0,0 +1,100 @@ +1 +-- { echoOn } + +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10; +0 2 0 +1 2 0 +2 2 0 +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10 +SETTINGS max_threads = 1; +0 2 0 +1 2 0 +2 2 0 +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 0 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 1 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 2 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 3 + GROUP BY + ac, + nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10; +0 2 0 +1 2 0 +2 2 0 diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.sql b/tests/queries/0_stateless/02884_parallel_window_functions.sql new file mode 100644 index 00000000000..5e71fadb3ff --- /dev/null +++ b/tests/queries/0_stateless/02884_parallel_window_functions.sql @@ -0,0 +1,119 @@ +CREATE TABLE window_funtion_threading +Engine = MergeTree +ORDER BY (ac, nw) +AS SELECT + toUInt64(toFloat32(number % 2) % 20000000) as ac, + toFloat32(1) as wg, + toUInt16(toFloat32(number % 3) % 400) as nw +FROM numbers_mt(10000000); + +SELECT count() FROM (EXPLAIN PIPELINE SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10) where explain ilike '%ScatterByPartitionTransform%'; + +-- { echoOn } + +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10; + +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10 +SETTINGS max_threads = 1; + +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 0 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 1 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 2 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 3 + GROUP BY + ac, + nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10; From 09072097ec33231cd0df95dc91cfd1317f662da2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 14:32:47 +0200 Subject: [PATCH 048/813] Better --- src/Interpreters/Cache/FileCache.cpp | 82 ++++++++++---------- src/Interpreters/Cache/FileCacheSettings.cpp | 3 + src/Interpreters/Cache/FileSegment.h | 2 + 3 files changed, 47 insertions(+), 40 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index bbb5fd71b8f..6d507413bab 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -237,6 +237,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: if (!add_to_result(file_segment_metadata)) return result; + ++segment_it; } } @@ -418,66 +419,67 @@ FileCache::getOrSet( assertInitialized(); + const auto end_offset = offset + size - 1; const auto aligned_offset = roundDownToMultiple(offset, boundary_alignment); - const auto aligned_end = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size); - const auto aligned_size = aligned_end - aligned_offset; - - FileSegment::Range range(aligned_offset, aligned_offset + aligned_size - 1); + const auto aligned_end_offset = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size) - 1; + chassert(aligned_offset <= offset); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY); /// Get all segments which intersect with the given range. + FileSegment::Range range(offset, end_offset); auto file_segments = getImpl(*locked_key, range, file_segments_limit); - bool limit_reached = false; + if (aligned_offset < offset && (file_segments.empty() || offset < file_segments.front()->range().left)) + { + auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? offset - 1 : file_segments.front()->range().left - 1); + auto prefix_file_segments = getImpl(*locked_key, prefix_range, /* file_segments_limit */0); + + while (!prefix_file_segments.empty() && prefix_file_segments.front()->range().right < offset) + prefix_file_segments.pop_front(); + + if (!prefix_file_segments.empty()) + { + file_segments.splice(file_segments.begin(), prefix_file_segments); + range.left = file_segments.front()->range().left; + } + } + + if (end_offset < aligned_end_offset && (file_segments.empty() || file_segments.back()->range().right < end_offset)) + { + auto suffix_range = FileSegment::Range(end_offset, aligned_end_offset); + /// Get only 1 file segment. + auto suffix_file_segments = getImpl(*locked_key, suffix_range, /* file_segments_limit */1); + + if (!suffix_file_segments.empty()) + range.right = suffix_file_segments.front()->range().left - 1; + } + if (file_segments.empty()) { file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, settings); - - while (!file_segments.empty() && file_segments.front()->range().right < offset) - file_segments.pop_front(); } else { - limit_reached = file_segments_limit && file_segments.size() >= file_segments_limit; - - /// A while loop for the case if we set a limit to n, but all these n file segments are removed - /// as they turned out redundant because of the alignment of offset to aligned_offset. - while (true) - { - size_t last_offset = file_segments.back()->range().right; - - while (!file_segments.empty() && file_segments.front()->range().right < offset) - file_segments.pop_front(); - - if (!file_segments.empty()) - break; - - if (!limit_reached) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of file segments"); - - range.left = last_offset + 1; - chassert(offset >= range.left); - file_segments = getImpl(*locked_key, range, file_segments_limit); - } - - range.left = std::min(offset, file_segments.front()->range().left); - if (limit_reached) - range.right = file_segments.back()->range().right; + chassert(file_segments.front()->range().right >= offset); + chassert(file_segments.back()->range().left <= end_offset); fillHolesWithEmptyFileSegments( *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, settings); + + if (!file_segments.front()->range().contains(offset)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " + "(end offset: {}, aligned offset: {}, aligned end offset: {})", + file_segments.front()->range().toString(), offset, end_offset, aligned_offset, aligned_end_offset); + } + + chassert(file_segments_limit ? file_segments.back()->range().left <= end_offset : file_segments.back()->range().contains(end_offset)); } - while (!file_segments.empty() && file_segments.back()->range().left >= offset + size) + while (file_segments_limit && file_segments.size() > file_segments_limit) file_segments.pop_back(); - if (file_segments_limit) - { - while (file_segments.size() > file_segments_limit) - file_segments.pop_back(); - } - if (file_segments.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of file segments for offset {}, size {} (file size: {})", offset, size, file_size); diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 6f2f8c4b778..de21555c050 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -47,6 +47,9 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & if (config.has(config_prefix + ".boundary_alignment")) boundary_alignment = parseWithSizeSuffix(config.getString(config_prefix + ".boundary_alignment")); + if (boundary_alignment > max_file_segment_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `boundary_alignment` cannot exceed `max_file_segment_size`"); + if (config.has(config_prefix + ".background_download_threads")) background_download_threads = config.getUInt(config_prefix + ".background_download_threads"); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 03d534f906d..04c82d71b5e 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -136,6 +136,8 @@ public: size_t size() const { return right - left + 1; } String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); } + + bool contains(size_t offset) const { return left <= offset && offset <= right; } }; static String getCallerId(); From ea95a49e864f3e49de467b53b29dc72a719e4f33 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 16 Oct 2023 21:22:43 +0000 Subject: [PATCH 049/813] merge_row_policy: some changes per code review --- src/Storages/StorageMerge.cpp | 76 ++++++++++++++--------------------- src/Storages/StorageMerge.h | 3 +- 2 files changed, 32 insertions(+), 47 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 354e17ad5ba..33b850f0aa0 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -382,13 +382,7 @@ void StorageMerge::read( class ReadFromMerge::RowPolicyData { public: - /// Row policy requires extra filtering - bool hasRowPolicy() - { - return static_cast(row_policy_filter_ptr); - } - - void init(RowPolicyFilterPtr, std::shared_ptr, ContextPtr); + RowPolicyData(RowPolicyFilterPtr, std::shared_ptr, ContextPtr); /// Add columns that needed for row policies to data stream /// SELECT x from T if T has row policy y=42 @@ -687,7 +681,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage_snapshot, modified_query_info); - RowPolicyData row_policy_data; + std::optional row_policy_data; if (processed_stage <= storage_stage || (allow_experimental_analyzer && processed_stage == QueryProcessingStage::FetchColumns)) { @@ -700,19 +694,17 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( StorageView * view = dynamic_cast(storage.get()); if (!view || allow_experimental_analyzer) { - row_policy_data.init( - modified_context->getRowPolicyFilter( + auto row_policy_filter_ptr = modified_context->getRowPolicyFilter( database_name, table_name, - RowPolicyFilterType::SELECT_FILTER), - storage, - modified_context); - - if (row_policy_data.hasRowPolicy()) + RowPolicyFilterType::SELECT_FILTER); + if (row_policy_filter_ptr) { - row_policy_data.extendNames(real_column_names); + row_policy_data.emplace(row_policy_filter_ptr, storage, modified_context); + row_policy_data->extendNames(real_column_names); } + storage->read(plan, real_column_names, storage_snapshot, @@ -725,11 +717,11 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (!plan.isInitialized()) return {}; - if (row_policy_data.hasRowPolicy()) + if (row_policy_data) { if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) { - row_policy_data.addStorageFilter(source_step_with_filter); + row_policy_data->addStorageFilter(source_step_with_filter); } } } @@ -847,6 +839,11 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( }); } + if (row_policy_data) + { + row_policy_data->addFilterTransform(*builder); + } + /// 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. /// Besides this we add FilterTransform if it is needed to follow row level policies. @@ -856,42 +853,38 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( aliases, modified_context, *builder, - processed_stage, - row_policy_data); + processed_stage); } return builder; } -void ReadFromMerge::RowPolicyData::init(RowPolicyFilterPtr row_policy_filter_ptr_, +ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter_ptr_, std::shared_ptr storage, ContextPtr local_context) + : row_policy_filter_ptr(row_policy_filter_ptr_) { - if (row_policy_filter_ptr_) - { - row_policy_filter_ptr = row_policy_filter_ptr_; + assert(row_policy_filter_ptr_); - ASTPtr expr = row_policy_filter_ptr->expression; + ASTPtr expr = row_policy_filter_ptr->expression; - auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto storage_columns = storage_metadata_snapshot->getColumns(); - auto needed_columns = storage_columns.getAllPhysical(); + auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto storage_columns = storage_metadata_snapshot->getColumns(); + auto needed_columns = storage_columns.getAllPhysical(); - auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns); - auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; + auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns); + auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; - actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(actions_dag, - ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); - } + actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); + filter_actions = std::make_shared(actions_dag, + ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); + filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); } // Add columns that needed to evaluate row policies // SELECT x from t if t has row policy void ReadFromMerge::RowPolicyData::extendNames(Names & names) { - assert(row_policy_filter_ptr); ASTPtr expr = row_policy_filter_ptr->expression; RequiredSourceColumnsVisitor::Data columns_context; @@ -899,12 +892,11 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const auto req_columns = columns_context.requiredColumns(); - std::sort(names.begin(), names.end()); NameSet added_names; for (const auto & req_column : req_columns) { - if (!std::binary_search(names.begin(), names.end(), req_column)) + if (std::find(names.begin(), names.end(), req_column) == names.end()) { added_names.insert(req_column); } @@ -1138,14 +1130,8 @@ void ReadFromMerge::convertingSourceStream( const Aliases & aliases, ContextPtr local_context, QueryPipelineBuilder & builder, - QueryProcessingStage::Enum processed_stage, - RowPolicyData & row_policy_data) + QueryProcessingStage::Enum processed_stage) { - if (row_policy_data.hasRowPolicy()) - { - row_policy_data.addFilterTransform(builder); - } - Block before_block_header = builder.getHeader(); auto storage_sample_block = metadata_snapshot->getSampleBlock(); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 1a5aca24e4c..04c5a70651f 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -203,8 +203,7 @@ private: const Aliases & aliases, ContextPtr context, QueryPipelineBuilder & builder, - QueryProcessingStage::Enum processed_stage, - RowPolicyData & row_policy_data); + QueryProcessingStage::Enum processed_stage); }; } From 7f991bf6e7372c1520a278fad9f7bc8d7dd83bc3 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 16 Oct 2023 21:49:16 +0000 Subject: [PATCH 050/813] merge_row_policy: a comment --- src/Storages/StorageMerge.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 33b850f0aa0..95760bbeb42 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -881,8 +881,8 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); } -// Add columns that needed to evaluate row policies -// SELECT x from t if t has row policy +// Add columns that needed _only_ to evaluate row policies +// SELECT x from t if t has row policy that is based on y void ReadFromMerge::RowPolicyData::extendNames(Names & names) { ASTPtr expr = row_policy_filter_ptr->expression; From 786183ee2447936fe9277df4b2554ed8a9d9cc47 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 16 Oct 2023 22:16:53 +0000 Subject: [PATCH 051/813] merge_row_policy: get rid of RequiredSourceColumnsVisitor --- src/Storages/StorageMerge.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 95760bbeb42..61f2132cfeb 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -885,16 +885,9 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter // SELECT x from t if t has row policy that is based on y void ReadFromMerge::RowPolicyData::extendNames(Names & names) { - ASTPtr expr = row_policy_filter_ptr->expression; - - RequiredSourceColumnsVisitor::Data columns_context; - RequiredSourceColumnsVisitor(columns_context).visit(expr); - - const auto req_columns = columns_context.requiredColumns(); - NameSet added_names; - for (const auto & req_column : req_columns) + for (const auto & req_column : filter_actions->getRequiredColumns()) { if (std::find(names.begin(), names.end(), req_column) == names.end()) { From 7515853ad4b3e910f20df99038d706ef77ab2819 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 11:43:51 +0200 Subject: [PATCH 052/813] Fix build --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/Interpreters/Cache/FileCache.cpp | 4 ++-- src/Interpreters/Cache/FileCache.h | 9 +++++++-- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 3c16d3d9ae2..27d0b6706a6 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -127,7 +127,7 @@ bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch() else { CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); - file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), settings.filesystem_cache_getorset_batch_size, create_settings); + file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_getorset_batch_size); } return !file_segments->empty(); } diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 6d507413bab..82a724523e7 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -412,8 +412,8 @@ FileCache::getOrSet( size_t offset, size_t size, size_t file_size, - size_t file_segments_limit, - const CreateFileSegmentSettings & settings) + const CreateFileSegmentSettings & settings, + size_t file_segments_limit) { ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheGetOrSetMicroseconds); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 2bf7b9281d5..d85f50cf34c 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -84,8 +84,13 @@ public: * As long as pointers to returned file segments are held * it is guaranteed that these file segments are not removed from cache. */ - FileSegmentsHolderPtr - getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, size_t file_segments_limit, const CreateFileSegmentSettings & settings); + FileSegmentsHolderPtr getOrSet( + const Key & key, + size_t offset, + size_t size, + size_t file_size, + const CreateFileSegmentSettings & settings, + size_t file_segments_limit = 0); /** * Segments in returned list are ordered in ascending order and represent a full contiguous From d837aa675f5ec56434aa7f58332fc4b922b1b9ba Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 13:14:28 +0200 Subject: [PATCH 053/813] Fix --- src/Interpreters/Cache/FileCache.cpp | 29 +++++++++++++++++++++------- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 82a724523e7..ba4998bfa00 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -435,13 +435,26 @@ FileCache::getOrSet( auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? offset - 1 : file_segments.front()->range().left - 1); auto prefix_file_segments = getImpl(*locked_key, prefix_range, /* file_segments_limit */0); - while (!prefix_file_segments.empty() && prefix_file_segments.front()->range().right < offset) - prefix_file_segments.pop_front(); - - if (!prefix_file_segments.empty()) + if (prefix_file_segments.empty()) { - file_segments.splice(file_segments.begin(), prefix_file_segments); - range.left = file_segments.front()->range().left; + range.left = aligned_offset; + } + else + { + size_t last_right_offset = prefix_file_segments.back()->range().right; + + while (!prefix_file_segments.empty() && prefix_file_segments.front()->range().right < offset) + prefix_file_segments.pop_front(); + + if (prefix_file_segments.empty()) + { + range.left = last_right_offset + 1; + } + else + { + file_segments.splice(file_segments.begin(), prefix_file_segments); + range.left = file_segments.front()->range().left; + } } } @@ -451,7 +464,9 @@ FileCache::getOrSet( /// Get only 1 file segment. auto suffix_file_segments = getImpl(*locked_key, suffix_range, /* file_segments_limit */1); - if (!suffix_file_segments.empty()) + if (suffix_file_segments.empty()) + range.right = aligned_end_offset; + else range.right = suffix_file_segments.front()->range().left - 1; } From 5d8b1cea910b3beb531e1e6122d2596f7197eae8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 19:19:19 +0200 Subject: [PATCH 054/813] Fix --- src/Interpreters/Cache/FileCache.cpp | 2 +- .../0_stateless/02503_cache_on_write_with_small_segment_size.sh | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index ba4998bfa00..3349dcc8d3f 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -347,7 +347,7 @@ void FileCache::fillHolesWithEmptyFileSegments( ++it; } - if (file_segments.size() >= file_segments_limit) + if (file_segments_limit && file_segments.size() >= file_segments_limit) return; if (current_pos <= range.right) diff --git a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh index 63f912c6bff..4f3fd0e54f6 100755 --- a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh +++ b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh @@ -22,6 +22,7 @@ SETTINGS min_bytes_for_wide_part = 0, type = cache, max_size = '128Mi', max_file_segment_size = '10Ki', + boundary_alignment = '5Ki', path = '${CLICKHOUSE_TEST_UNIQUE_NAME}', cache_on_write_operations = 1, enable_filesystem_query_cache_limit = 1, From 5ca43e51c1dfce7d13b0f192327b8c16e524243c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 17 Oct 2023 20:45:00 +0000 Subject: [PATCH 055/813] Remove unused code --- src/Processors/QueryPlan/WindowStep.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Processors/QueryPlan/WindowStep.cpp b/src/Processors/QueryPlan/WindowStep.cpp index d104cfe2975..12f78ac014d 100644 --- a/src/Processors/QueryPlan/WindowStep.cpp +++ b/src/Processors/QueryPlan/WindowStep.cpp @@ -7,11 +7,6 @@ #include #include -#include "Columns/ColumnConst.h" -#include "DataTypes/DataTypesNumber.h" -#include "Functions/FunctionFactory.h" -#include "Processors/Transforms/ScatterByPartitionTransform.h" - namespace DB { From 22bab4bcc3b13741f19e0b8dd4afa23b4f1cca44 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 11:26:44 +0200 Subject: [PATCH 056/813] Fix configs --- tests/integration/test_filesystem_cache/test.py | 2 +- .../configs/config.d/storage_configuration.xml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_filesystem_cache/test.py b/tests/integration/test_filesystem_cache/test.py index be7b12946a7..3a6a1ef76eb 100644 --- a/tests/integration/test_filesystem_cache/test.py +++ b/tests/integration/test_filesystem_cache/test.py @@ -46,7 +46,7 @@ def test_parallel_cache_loading_on_startup(cluster, node_name): path = 'paralel_loading_test', disk = 'hdd_blob', max_file_segment_size = '1Ki', - boundary_alignemt = '1Ki', + boundary_alignment = '1Ki', max_size = '1Gi', max_elements = 10000000, load_metadata_threads = 30); diff --git a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml index b527c74e8de..3064003e6c8 100644 --- a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml @@ -12,6 +12,7 @@ /tiny_local_cache/ 10M 1M + 1M 1 0 From 7aa57516c199f45548aea308c4ce2ee1d814e73e Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 11:31:10 +0200 Subject: [PATCH 057/813] Update tests config --- tests/config/users.d/s3_cache.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/users.d/s3_cache.xml b/tests/config/users.d/s3_cache.xml index 69b24ecbbc4..4740f37a90c 100644 --- a/tests/config/users.d/s3_cache.xml +++ b/tests/config/users.d/s3_cache.xml @@ -3,6 +3,7 @@ 1 1 + 10 From c792d952716f333b198bf014d99fc1dceb6a062b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 14:41:06 +0200 Subject: [PATCH 058/813] Update config --- tests/config/config.d/s3_storage_policy_by_default.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/config.d/s3_storage_policy_by_default.xml b/tests/config/config.d/s3_storage_policy_by_default.xml index dd93a317a77..e161c2ee01a 100644 --- a/tests/config/config.d/s3_storage_policy_by_default.xml +++ b/tests/config/config.d/s3_storage_policy_by_default.xml @@ -12,6 +12,7 @@ 1Gi cached_s3/ s3 + 10 From 89272e0925c91ed659b51741c58ddc364e149792 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 19 Oct 2023 11:23:56 +0200 Subject: [PATCH 059/813] Fix upgrade check, randomize more settings --- docker/test/upgrade/run.sh | 2 ++ tests/clickhouse-test | 3 +++ tests/config/config.d/s3_storage_policy_by_default.xml | 1 - tests/config/install.sh | 1 + tests/config/users.d/s3_cache_new.xml | 7 +++++++ 5 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/config/users.d/s3_cache_new.xml diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index c69d90b9af0..3580f8e5021 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -78,6 +78,7 @@ remove_keeper_config "create_if_not_exists" "[01]" rm /etc/clickhouse-server/config.d/merge_tree.xml rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml rm /etc/clickhouse-server/users.d/nonconst_timezone.xml +rm /etc/clickhouse-server/users.d/s3_cache_new.xml start stop @@ -114,6 +115,7 @@ sudo chgrp clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_defau rm /etc/clickhouse-server/config.d/merge_tree.xml rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml rm /etc/clickhouse-server/users.d/nonconst_timezone.xml +rm /etc/clickhouse-server/users.d/s3_cache_new.xml start diff --git a/tests/clickhouse-test b/tests/clickhouse-test index cab7d7e79ff..c0c2d482703 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -577,6 +577,9 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), + "filesystem_cache_getorset_batch_size": lambda: random.randint(0, 3, 10, 50), + "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint(0, 1), + "throw_on_error_from_cache_on_write_operations": lambda: random.randint(0, 1), "remote_filesystem_read_prefetch": lambda: random.randint(0, 1), "allow_prefetched_read_pool_for_remote_filesystem": lambda: random.randint( 0, 1 diff --git a/tests/config/config.d/s3_storage_policy_by_default.xml b/tests/config/config.d/s3_storage_policy_by_default.xml index e161c2ee01a..dd93a317a77 100644 --- a/tests/config/config.d/s3_storage_policy_by_default.xml +++ b/tests/config/config.d/s3_storage_policy_by_default.xml @@ -12,7 +12,6 @@ 1Gi cached_s3/ s3 - 10 diff --git a/tests/config/install.sh b/tests/config/install.sh index 9e3b235515d..d76949fadc7 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -151,6 +151,7 @@ if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then ln -sf $SRC_PATH/config.d/storage_conf.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/users.d/ + ln -sf $SRC_PATH/users.d/s3_cache_new.xml $DEST_SERVER_PATH/users.d/ fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then diff --git a/tests/config/users.d/s3_cache_new.xml b/tests/config/users.d/s3_cache_new.xml new file mode 100644 index 00000000000..638b7267960 --- /dev/null +++ b/tests/config/users.d/s3_cache_new.xml @@ -0,0 +1,7 @@ + + + + 10 + + + From 7a096904ed001be79b88d2ef50d0c7c460a933bc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Oct 2023 15:40:32 +0200 Subject: [PATCH 060/813] clickhouse-test: export product function for jinja2 Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index cab7d7e79ff..c09974b780d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -8,6 +8,7 @@ import enum from queue import Full import shutil +import itertools import sys import os import os.path @@ -1605,6 +1606,8 @@ class TestSuite: if USE_JINJA else None ) + if j2env is not None: + j2env.globals.update(product=itertools.product) for test_name in os.listdir(self.suite_path): if not is_test_from_dir(self.suite_path, test_name): From 6da75a47c5eb0c68c6b39b22a08ca9923181f808 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Oct 2023 14:32:50 +0200 Subject: [PATCH 061/813] Fix "Cannot find column X in source stream" for Distributed queries with LIMIT BY Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterSelectQuery.cpp | 15 +++++- .../02899_distributed_limit_by.reference | 52 +++++++++++++++++++ .../02899_distributed_limit_by.sql.j2 | 26 ++++++++++ .../02900_limit_by_query_stage.reference | 3 ++ .../0_stateless/02900_limit_by_query_stage.sh | 9 ++++ 5 files changed, 104 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02899_distributed_limit_by.reference create mode 100644 tests/queries/0_stateless/02899_distributed_limit_by.sql.j2 create mode 100644 tests/queries/0_stateless/02900_limit_by_query_stage.reference create mode 100755 tests/queries/0_stateless/02900_limit_by_query_stage.sh diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0300cc4c9b4..4c51e1d6396 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -993,6 +993,9 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.before_window) return analysis_result.before_window->getResultColumns(); + // NOTE: should not handle before_limit_by specially since + // WithMergeableState does not process LIMIT BY + return analysis_result.before_order_by->getResultColumns(); } @@ -1036,6 +1039,12 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.before_window) return analysis_result.before_window->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_order_by->getResultColumns(); } @@ -1482,7 +1491,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

= 0 limit 1 by dummy + dummy + 0 as l settings {{ settings }}; +select dummy from (select dummy + dummy + 0 as l, dummy from remote('127.{1,1}', system.one) where dummy + dummy >= 0 limit 1 by l) settings {{ settings }}; +{% endfor %} diff --git a/tests/queries/0_stateless/02900_limit_by_query_stage.reference b/tests/queries/0_stateless/02900_limit_by_query_stage.reference new file mode 100644 index 00000000000..b01fb1ca5b0 --- /dev/null +++ b/tests/queries/0_stateless/02900_limit_by_query_stage.reference @@ -0,0 +1,3 @@ +0 0 +0 0 +0 0 diff --git a/tests/queries/0_stateless/02900_limit_by_query_stage.sh b/tests/queries/0_stateless/02900_limit_by_query_stage.sh new file mode 100755 index 00000000000..d34d0d81bcd --- /dev/null +++ b/tests/queries/0_stateless/02900_limit_by_query_stage.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --stage with_mergeable_state --query 'SELECT dummy FROM system.one WHERE (dummy + dummy) >= 0 LIMIT 1 BY (dummy + dummy) + 0 AS l' +$CLICKHOUSE_CLIENT --stage with_mergeable_state_after_aggregation --query 'SELECT dummy FROM system.one WHERE (dummy + dummy) >= 0 LIMIT 1 BY (dummy + dummy) + 0 AS l' +$CLICKHOUSE_CLIENT --stage with_mergeable_state_after_aggregation_and_limit --query 'SELECT dummy FROM system.one WHERE (dummy + dummy) >= 0 LIMIT 1 BY (dummy + dummy) + 0 AS l' From b902cf51bb8ae71066ea6f46c174608440b04d44 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Oct 2023 18:41:49 +0200 Subject: [PATCH 062/813] Update 01952_optimize_distributed_group_by_sharding_key reference Signed-off-by: Azat Khuzhin --- ...ze_distributed_group_by_sharding_key.reference | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index 9016e731106..6adb2382a6f 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -19,10 +19,8 @@ explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 Expression (Projection) LimitBy Union - Expression (Before LIMIT BY) - LimitBy - Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))) - ReadFromStorage (SystemNumbers) + Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))) + ReadFromStorage (SystemNumbers) Expression ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized @@ -58,11 +56,10 @@ Expression (Projection) Expression (Before LIMIT BY) Sorting (Merge sorted streams for ORDER BY, without aggregation) Union - LimitBy - Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) [lifted up part])) - Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) - ReadFromStorage (SystemNumbers) + Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) [lifted up part]) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized Expression (Projection) From b13adbbeab1d5d0c91562f187ad5f4d651316311 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 20 Oct 2023 11:48:27 +0200 Subject: [PATCH 063/813] Fix style check --- tests/clickhouse-test | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c0c2d482703..36ac409a4cb 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -577,8 +577,10 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), - "filesystem_cache_getorset_batch_size": lambda: random.randint(0, 3, 10, 50), - "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint(0, 1), + "filesystem_cache_getorset_batch_size": lambda: random.choice([0, 3, 10, 50]), + "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint( + 0, 1 + ), "throw_on_error_from_cache_on_write_operations": lambda: random.randint(0, 1), "remote_filesystem_read_prefetch": lambda: random.randint(0, 1), "allow_prefetched_read_pool_for_remote_filesystem": lambda: random.randint( From 8609125f7a244b91a64e75242939e64f824d2ce6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 20 Oct 2023 12:21:05 +0200 Subject: [PATCH 064/813] fuzzer/generate-test-j2: export missing product function Signed-off-by: Azat Khuzhin --- docker/test/fuzzer/generate-test-j2.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fuzzer/generate-test-j2.py b/docker/test/fuzzer/generate-test-j2.py index 11525163ed8..6fd37d6bd02 100755 --- a/docker/test/fuzzer/generate-test-j2.py +++ b/docker/test/fuzzer/generate-test-j2.py @@ -3,6 +3,7 @@ from argparse import ArgumentParser import os import jinja2 +import itertools def removesuffix(text, suffix): @@ -47,6 +48,7 @@ def main(args): loader=jinja2.FileSystemLoader(suite_dir), keep_trailing_newline=True, ) + j2env.globals.update(product=itertools.product) test_names = os.listdir(suite_dir) for test_name in test_names: From 0d27150948a54a9bf8513622111a4feb52476bbb Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 24 Oct 2023 13:09:57 +0000 Subject: [PATCH 065/813] merge_row_policy: cleanup --- src/Storages/StorageMerge.cpp | 88 +++++++++++++---------------------- 1 file changed, 32 insertions(+), 56 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 61f2132cfeb..7df8b8cc6c1 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -46,7 +45,6 @@ #include #include -#include namespace { @@ -384,23 +382,23 @@ class ReadFromMerge::RowPolicyData public: RowPolicyData(RowPolicyFilterPtr, std::shared_ptr, ContextPtr); - /// Add columns that needed for row policies to data stream - /// SELECT x from T if T has row policy y=42 - /// required y in data pipeline + /// Add to data stream columns that are needed only for row policies + /// SELECT x from T if T has row policy y=42 + /// required y in data pipeline void extendNames(Names &); /// Use storage facilities to filter data - /// does not guarantee accuracy, but reduce number of rows + /// optimization + /// does not guarantee accuracy, but reduces number of rows void addStorageFilter(SourceStepWithFilter *); - /// Create explicit filter transform to stop + /// Create explicit filter transform to exclude /// rows that are not conform to row level policy void addFilterTransform(QueryPipelineBuilder &); private: - static std::string namesDifference(Names && outer_set, Names && inner_set); RowPolicyFilterPtr row_policy_filter_ptr; - std::string filter_column_name; // complex filer, may contain logic operations + std::string filter_column_name; // complex filter, may contain logic operations ActionsDAGPtr actions_dag; ExpressionActionsPtr filter_actions; }; @@ -704,7 +702,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( row_policy_data->extendNames(real_column_names); } - storage->read(plan, real_column_names, storage_snapshot, @@ -713,17 +710,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( processed_stage, max_block_size, UInt32(streams_num)); - - if (!plan.isInitialized()) - return {}; - - if (row_policy_data) - { - if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) - { - row_policy_data->addStorageFilter(source_step_with_filter); - } - } } else { @@ -741,9 +727,17 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( view->getInMemoryMetadataPtr(), SelectQueryOptions(processed_stage)); interpreter.buildQueryPlan(plan); + } - if (!plan.isInitialized()) - return {}; + if (!plan.isInitialized()) + return {}; + + if (row_policy_data) + { + if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) + { + row_policy_data->addStorageFilter(source_step_with_filter); + } } if (auto * read_from_merge_tree = typeid_cast(plan.getRootNode()->step.get())) @@ -846,14 +840,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( /// 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. - /// Besides this we add FilterTransform if it is needed to follow row level policies. - - convertingSourceStream(header, - storage_snapshot->metadata, - aliases, - modified_context, - *builder, - processed_stage); + convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, *builder, processed_stage); } return builder; @@ -878,18 +865,28 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); filter_actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - filter_column_name = namesDifference(filter_actions->getSampleBlock().getNames(), filter_actions->getRequiredColumns()); + const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); + const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); + + NamesAndTypesList added, deleted; + sample_block_columns.getDifference(required_columns, added, deleted); + if (!deleted.empty() || added.size() != 1) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot determine row level filter"); + } + + filter_column_name = added.getNames().front(); } -// Add columns that needed _only_ to evaluate row policies -// SELECT x from t if t has row policy that is based on y void ReadFromMerge::RowPolicyData::extendNames(Names & names) { + std::sort(names.begin(), names.end()); NameSet added_names; for (const auto & req_column : filter_actions->getRequiredColumns()) { - if (std::find(names.begin(), names.end(), req_column) == names.end()) + if (!std::binary_search(names.begin(), names.end(), req_column)) { added_names.insert(req_column); } @@ -926,27 +923,6 @@ void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & bui }); } -/// Find out an item that in outer_set vector, but not in inner_set vector -std::string ReadFromMerge::RowPolicyData::namesDifference(Names && outer_set, Names && inner_set) -{ - std::sort(outer_set.begin(), outer_set.end()); - std::sort(inner_set.begin(), inner_set.end()); - - Names result; - - std::set_difference(outer_set.begin(), outer_set.end(), - inner_set.begin(), inner_set.end(), std::inserter(result, result.begin())); - - if (result.size() != 1) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot determine row level filter"); - } - - return result.front(); -} - - StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( ContextPtr query_context, const ASTPtr & query /* = nullptr */, From 2c055480d622d0ccc05b65c9c0252b36b66f7eca Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 24 Oct 2023 14:52:47 +0000 Subject: [PATCH 066/813] Remove unnecessary flag --- src/Processors/Sources/RemoteSource.cpp | 6 ++---- src/Processors/Sources/RemoteSource.h | 1 - 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 74ab3649068..6ca5e611713 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -42,7 +42,7 @@ void RemoteSource::setStorageLimits(const std::shared_ptr RemoteSource::tryGenerate() { /// onCancel() will do the cancel if the query was sent. - if (was_query_canceled) + if (isCancelled()) return {}; if (!was_query_sent) @@ -169,7 +169,6 @@ std::optional RemoteSource::tryGenerate() void RemoteSource::onCancel() { - was_query_canceled = true; query_executor->cancel(); } @@ -177,7 +176,6 @@ void RemoteSource::onUpdatePorts() { if (getPort().isFinished()) { - was_query_canceled = true; query_executor->finish(); } } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index da39b5d0046..dbfa0156331 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -39,7 +39,6 @@ protected: void onCancel() override; private: - std::atomic was_query_canceled = false; bool was_query_sent = false; bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; From 2516c2ea284d85d0cdeaca33f781e54f1813d806 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 24 Oct 2023 20:46:44 +0000 Subject: [PATCH 067/813] Fix clickhouse-local exit on bad send_logs_level setting --- src/Client/ClientBase.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a350654cdda..1382d3830f5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1810,7 +1810,12 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin { const auto * logs_level_field = set_query->changes.tryGet(std::string_view{"send_logs_level"}); if (logs_level_field) - updateLoggerLevel(logs_level_field->safeGet()); + { + auto logs_level = logs_level_field->safeGet(); + /// Check that setting value is correct before updating logger level. + SettingFieldLogsLevelTraits::fromString(logs_level); + updateLoggerLevel(logs_level); + } } if (const auto * create_user_query = parsed_query->as()) From 2606b60e6649915d71dbd70b2ee7a80dc371a6d7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 25 Oct 2023 04:17:35 +0000 Subject: [PATCH 068/813] Fix REPLICA_ALREADY_EXISTS for ReplicatedMergeTree --- src/Storages/StorageReplicatedMergeTree.cpp | 84 ++++++++++++++++++++- 1 file changed, 81 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4de7ecb7fd9..91b15f96297 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -861,6 +861,84 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada LOG_DEBUG(log, "Creating replica {}", replica_path); + const String local_metadata = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); + const String local_columns = metadata_snapshot->getColumns().toString(); + const String local_metadata_version = toString(metadata_snapshot->getMetadataVersion()); + + /// It is possible for the replica to fail after creating ZK nodes without saving local metadata. + /// Because of that we need to check whether the replica exists and is newly created. + /// For this we check that all nodes exist, the metadata of the table is the same, and other nodes are not modified. + + std::vector paths_exists = { + replica_path, + replica_path + "/host", + replica_path + "/log_pointer", + replica_path + "/queue", + replica_path + "/parts", + replica_path + "/flags", + replica_path + "/is_lost", + replica_path + "/metadata", + replica_path + "/columns", + replica_path + "/metadata_version", + replica_path + "/mutation_pointer", + replica_path + "/min_unprocessed_insert_time", + replica_path + "/max_processed_insert_time", + replica_path + "/mutation_pointer" + }; + + auto response_exists = zookeeper->tryGet(paths_exists); + size_t response_num = 0; + + if (response_exists[response_num++].error == Coordination::Error::ZOK) + { + bool all_nodes_exist = true; + + for (size_t i = 0; i < response_exists.size(); ++i) + { + if (response_exists[i].error != Coordination::Error::ZOK) + { + all_nodes_exist = false; + break; + } + } + + if (all_nodes_exist) + { + const auto & zk_host = response_exists[response_num++].data; + const auto & zk_log_pointer = response_exists[response_num++].data; + const auto & zk_queue = response_exists[response_num++].data; + const auto & zk_parts = response_exists[response_num++].data; + const auto & zk_flags = response_exists[response_num++].data; + const auto & zk_is_lost = response_exists[response_num++].data; + const auto & zk_metadata = response_exists[response_num++].data; + const auto & zk_columns = response_exists[response_num++].data; + const auto & zk_metadata_version = response_exists[response_num++].data; + const auto & zk_min_unprocessed_insert_time = response_exists[response_num++].data; + const auto & zk_max_processed_insert_time = response_exists[response_num++].data; + const auto & zk_mutation_pointer = response_exists[response_num++].data; + + if (zk_host.empty() && + zk_log_pointer.empty() && + zk_queue.empty() && + zk_parts.empty() && + zk_flags.empty() && + (zk_is_lost == "0" || zk_is_lost == "1") && + zk_metadata == local_metadata && + zk_columns == local_columns && + zk_metadata_version == local_metadata_version && + zk_mutation_pointer.empty() && + zk_min_unprocessed_insert_time.empty() && + zk_max_processed_insert_time.empty() && + zk_mutation_pointer.empty()) + { + LOG_DEBUG(log, "Empty replica {} exists, will use it", replica_path); + return; + } + } + + throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, "Replica {} already exists", replica_path); + } + Coordination::Error code; do @@ -892,11 +970,11 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/is_lost", is_lost_value, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata", ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata", local_metadata, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", metadata_snapshot->getColumns().toString(), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", local_columns, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", toString(metadata_snapshot->getMetadataVersion()), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", local_metadata_version, zkutil::CreateMode::Persistent)); /// The following 3 nodes were added in version 1.1.xxx, so we create them here, not in createNewZooKeeperNodes() From efbcac4e600430262e4999c2a799e92dc6e5c4c8 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 25 Oct 2023 10:21:06 +0200 Subject: [PATCH 069/813] Bug fix explain ast with parameterized view --- src/Interpreters/executeQuery.cpp | 6 ++++++ .../02903_parameterized_view_explain_ast.reference | 12 ++++++++++++ .../02903_parameterized_view_explain_ast.sql | 3 +++ 3 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02903_parameterized_view_explain_ast.reference create mode 100644 tests/queries/0_stateless/02903_parameterized_view_explain_ast.sql diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index decda4c62f9..557f80e8d70 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -730,6 +730,12 @@ static std::tuple executeQueryImpl( bool is_create_parameterized_view = false; if (const auto * create_query = ast->as()) is_create_parameterized_view = create_query->isParameterizedView(); + else if (const auto * explain_query = ast->as()) + { + assert(explain_query->children.size() => 1); + if (const auto * create_of_explain_query = explain_query->children[0]->as()) + is_create_parameterized_view = create_of_explain_query->isParameterizedView(); + } /// Replace ASTQueryParameter with ASTLiteral for prepared statements. /// Even if we don't have parameters in query_context, check that AST doesn't have unknown parameters diff --git a/tests/queries/0_stateless/02903_parameterized_view_explain_ast.reference b/tests/queries/0_stateless/02903_parameterized_view_explain_ast.reference new file mode 100644 index 00000000000..6ee8d0c3d23 --- /dev/null +++ b/tests/queries/0_stateless/02903_parameterized_view_explain_ast.reference @@ -0,0 +1,12 @@ +CreateQuery numbers_pv (children 2) + Identifier numbers_pv + SelectWithUnionQuery (children 1) + ExpressionList (children 1) + SelectQuery (children 3) + ExpressionList (children 1) + Asterisk + TablesInSelectQuery (children 1) + TablesInSelectQueryElement (children 1) + TableExpression (children 1) + TableIdentifier numbers + QueryParameter amount:UInt8 diff --git a/tests/queries/0_stateless/02903_parameterized_view_explain_ast.sql b/tests/queries/0_stateless/02903_parameterized_view_explain_ast.sql new file mode 100644 index 00000000000..6af6dab2f4e --- /dev/null +++ b/tests/queries/0_stateless/02903_parameterized_view_explain_ast.sql @@ -0,0 +1,3 @@ +EXPLAIN AST +CREATE VIEW numbers_pv AS +SELECT * FROM numbers LIMIT {amount:UInt8}; \ No newline at end of file From d6c63e07ce97932bdfcf7baa5ab76bd9c1fee85b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 25 Oct 2023 10:23:22 +0200 Subject: [PATCH 070/813] Fixed expression --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 557f80e8d70..27fbce5311c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -732,7 +732,7 @@ static std::tuple executeQueryImpl( is_create_parameterized_view = create_query->isParameterizedView(); else if (const auto * explain_query = ast->as()) { - assert(explain_query->children.size() => 1); + assert(explain_query->children.size() >= 1); if (const auto * create_of_explain_query = explain_query->children[0]->as()) is_create_parameterized_view = create_of_explain_query->isParameterizedView(); } From 3c690337eccba39587c7cf4f6206d951c64414de Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 26 Oct 2023 03:17:30 +0000 Subject: [PATCH 071/813] Improvements --- src/Core/Settings.h | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 8 ++ src/Storages/StorageReplicatedMergeTree.cpp | 96 ++++++++++--------- ...ated_merge_tree_creation_failure.reference | 4 + ..._replicated_merge_tree_creation_failure.sh | 38 ++++++++ 5 files changed, 102 insertions(+), 46 deletions(-) create mode 100644 tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference create mode 100755 tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f2b55fbcd0b..955ad815e00 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -607,6 +607,8 @@ class IColumn; 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, create_replicated_merge_tree_fault_injection, false, "If true, the creation of table will be aborted after creating metadata in ZooKeeper", 0) \ + \ M(Bool, use_query_cache, false, "Enable the query cache", 0) \ M(Bool, enable_writes_to_query_cache, true, "Enable storing results of SELECT queries in the query cache", 0) \ M(Bool, enable_reads_from_query_cache, true, "Enable reading results of SELECT queries from the query cache", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a0635f18214..3da6c9fa264 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -104,6 +104,7 @@ namespace ErrorCodes extern const int UNKNOWN_STORAGE; extern const int SYNTAX_ERROR; extern const int SUPPORT_IS_DISABLED; + extern const int ABORTED; } namespace fs = std::filesystem; @@ -1442,6 +1443,13 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, "ATTACH ... FROM ... query is not supported for {} table engine, " "because such tables do not store any data on disk. Use CREATE instead.", res->getName()); + if (getContext()->getSettingsRef().create_replicated_merge_tree_fault_injection) + { + bool is_replicated_storage = typeid_cast(res.get()) != nullptr; + if (is_replicated_storage) + throw Exception(ErrorCodes::ABORTED, "Shutdown is called for 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/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 91b15f96297..0781684b7b7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -19,6 +19,8 @@ #include #include +#include + #include #include @@ -834,6 +836,9 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_uuid", toString(ServerUUID::get()), + zkutil::CreateMode::Persistent)); + Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) @@ -864,13 +869,13 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada const String local_metadata = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); const String local_columns = metadata_snapshot->getColumns().toString(); const String local_metadata_version = toString(metadata_snapshot->getMetadataVersion()); + const String creator_uuid = toString(ServerUUID::get()); /// It is possible for the replica to fail after creating ZK nodes without saving local metadata. /// Because of that we need to check whether the replica exists and is newly created. /// For this we check that all nodes exist, the metadata of the table is the same, and other nodes are not modified. std::vector paths_exists = { - replica_path, replica_path + "/host", replica_path + "/log_pointer", replica_path + "/queue", @@ -880,63 +885,59 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada replica_path + "/metadata", replica_path + "/columns", replica_path + "/metadata_version", - replica_path + "/mutation_pointer", replica_path + "/min_unprocessed_insert_time", replica_path + "/max_processed_insert_time", - replica_path + "/mutation_pointer" + replica_path + "/mutation_pointer", + replica_path + "/creator_uuid" }; auto response_exists = zookeeper->tryGet(paths_exists); - size_t response_num = 0; + bool all_nodes_exist = true; - if (response_exists[response_num++].error == Coordination::Error::ZOK) + for (size_t i = 0; i < response_exists.size(); ++i) { - bool all_nodes_exist = true; - - for (size_t i = 0; i < response_exists.size(); ++i) + if (response_exists[i].error != Coordination::Error::ZOK) { - if (response_exists[i].error != Coordination::Error::ZOK) - { - all_nodes_exist = false; - break; - } + all_nodes_exist = false; + break; } + } - if (all_nodes_exist) + if (all_nodes_exist) + { + size_t response_num = 0; + + const auto & zk_host = response_exists[response_num++].data; + const auto & zk_log_pointer = response_exists[response_num++].data; + const auto & zk_queue = response_exists[response_num++].data; + const auto & zk_parts = response_exists[response_num++].data; + const auto & zk_flags = response_exists[response_num++].data; + const auto & zk_is_lost = response_exists[response_num++].data; + const auto & zk_metadata = response_exists[response_num++].data; + const auto & zk_columns = response_exists[response_num++].data; + const auto & zk_metadata_version = response_exists[response_num++].data; + const auto & zk_min_unprocessed_insert_time = response_exists[response_num++].data; + const auto & zk_max_processed_insert_time = response_exists[response_num++].data; + const auto & zk_mutation_pointer = response_exists[response_num++].data; + const auto & zk_creator_uuid = response_exists[response_num++].data; + + if (zk_host.empty() && + zk_log_pointer.empty() && + zk_queue.empty() && + zk_parts.empty() && + zk_flags.empty() && + (zk_is_lost == "0" || zk_is_lost == "1") && + zk_metadata == local_metadata && + zk_columns == local_columns && + zk_metadata_version == local_metadata_version && + zk_min_unprocessed_insert_time.empty() && + zk_max_processed_insert_time.empty() && + zk_mutation_pointer.empty() && + zk_creator_uuid == creator_uuid) { - const auto & zk_host = response_exists[response_num++].data; - const auto & zk_log_pointer = response_exists[response_num++].data; - const auto & zk_queue = response_exists[response_num++].data; - const auto & zk_parts = response_exists[response_num++].data; - const auto & zk_flags = response_exists[response_num++].data; - const auto & zk_is_lost = response_exists[response_num++].data; - const auto & zk_metadata = response_exists[response_num++].data; - const auto & zk_columns = response_exists[response_num++].data; - const auto & zk_metadata_version = response_exists[response_num++].data; - const auto & zk_min_unprocessed_insert_time = response_exists[response_num++].data; - const auto & zk_max_processed_insert_time = response_exists[response_num++].data; - const auto & zk_mutation_pointer = response_exists[response_num++].data; - - if (zk_host.empty() && - zk_log_pointer.empty() && - zk_queue.empty() && - zk_parts.empty() && - zk_flags.empty() && - (zk_is_lost == "0" || zk_is_lost == "1") && - zk_metadata == local_metadata && - zk_columns == local_columns && - zk_metadata_version == local_metadata_version && - zk_mutation_pointer.empty() && - zk_min_unprocessed_insert_time.empty() && - zk_max_processed_insert_time.empty() && - zk_mutation_pointer.empty()) - { - LOG_DEBUG(log, "Empty replica {} exists, will use it", replica_path); - return; - } + LOG_DEBUG(log, "Empty replica {} exists, will use it", replica_path); + return; } - - throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, "Replica {} already exists", replica_path); } Coordination::Error code; @@ -985,6 +986,9 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_uuid", creator_uuid, + zkutil::CreateMode::Persistent)); + /// Check version of /replicas to see if there are any replicas created at the same moment of time. ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", "last added replica: " + replica_name, replicas_stat.version)); diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference new file mode 100644 index 00000000000..487b1165348 --- /dev/null +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference @@ -0,0 +1,4 @@ +2 +2 +2 +2 diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh new file mode 100755 index 00000000000..0c5705c7a35 --- /dev/null +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash +# Tags: zookeeper + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC" + +#### 1 - There is only one replica + +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" + +# We will see that the replica is empty and throw the same ABORT exception as before +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" + +# We will succeed +${CLICKHOUSE_CLIENT} \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC" + +#### 2 - There are two replicas + +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r2') ORDER BY date" 2>&1 | grep -c "ABORT" + +# We will succeed +${CLICKHOUSE_CLIENT} \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC" \ No newline at end of file From 940d099e84d92eaaacaa96682c5a94b26f7a782c Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 27 Oct 2023 16:50:34 -0700 Subject: [PATCH 072/813] Set correct max_block_size value in docs --- docs/en/operations/settings/settings.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ccf290c8e20..60eda45ab22 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -731,11 +731,13 @@ Default value: LZ4. ## max_block_size {#setting-max_block_size} -In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. The `max_block_size` setting is a recommendation for what size of the block (in a count of rows) to load from tables. The block size shouldn’t be too small, so that the expenditures on each block are still noticeable, but not too large so that the query with LIMIT that is completed after the first block is processed quickly. The goal is to avoid consuming too much memory when extracting a large number of columns in multiple threads and to preserve at least some cache locality. +In ClickHouse, data is processed by blocks, which are sets of column parts. The internal processing cycles for a single block are efficient but there are noticeable costs when processing each block. -Default value: 65,536. +The `max_block_size` setting indicates the recommended maximum number of rows to include in a single block when loading data from tables. Blocks the size of `max_block_size` are not always loaded from the table: if ClickHouse determines that less data needs to be retrieved, a smaller block is processed. -Blocks the size of `max_block_size` are not always loaded from the table. If it is obvious that less data needs to be retrieved, a smaller block is processed. +The block size should not be too small to avoid noticeable costs when processing each block. It should also not be too large to ensure that queries with a LIMIT clause execute quickly after processing the first block. When setting `max_block_size`, the goal should be to avoid consuming too much memory when extracting a large number of columns in multiple threads and to preserve at least some cache locality. + +Default value: `65,409` ## preferred_block_size_bytes {#preferred-block-size-bytes} From 70e3dd808cc3f087504892d18a9e61eb6f948151 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 02:07:24 +0100 Subject: [PATCH 073/813] Granular code coverage with introspection --- CMakeLists.txt | 9 -- base/base/CMakeLists.txt | 2 + base/base/coverage.cpp | 106 ++++++++++++++++++- base/base/coverage.h | 6 ++ base/glibc-compatibility/memcpy/memcpy.cpp | 1 + base/glibc-compatibility/memcpy/memcpy.h | 2 +- cmake/sanitize.cmake | 18 ++++ contrib/CMakeLists.txt | 9 -- contrib/google-protobuf-cmake/CMakeLists.txt | 32 ------ contrib/libcxx-cmake/CMakeLists.txt | 2 - programs/CMakeLists.txt | 2 + src/CMakeLists.txt | 5 +- src/Functions/coverage.cpp | 91 ++++++++++++++++ src/Interpreters/InterpreterSystemQuery.cpp | 8 ++ src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserSystemQuery.cpp | 16 +-- 16 files changed, 244 insertions(+), 66 deletions(-) create mode 100644 src/Functions/coverage.cpp diff --git a/CMakeLists.txt b/CMakeLists.txt index a5b94efefc5..d259b105a0a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -286,9 +286,6 @@ set (CMAKE_C_STANDARD 11) set (CMAKE_C_EXTENSIONS ON) # required by most contribs written in C set (CMAKE_C_STANDARD_REQUIRED ON) -# Compiler-specific coverage flags e.g. -fcoverage-mapping -option(WITH_COVERAGE "Profile the resulting binary/binaries" OFF) - if (COMPILER_CLANG) # Enable C++14 sized global deallocation functions. It should be enabled by setting -std=c++14 but I'm not sure. # See https://reviews.llvm.org/D112921 @@ -304,12 +301,6 @@ if (COMPILER_CLANG) set(BRANCHES_WITHIN_32B_BOUNDARIES "-mbranches-within-32B-boundaries") set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}") endif() - - if (WITH_COVERAGE) - set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-instr-generate -fcoverage-mapping") - # If we want to disable coverage for specific translation units - set(WITHOUT_COVERAGE "-fno-profile-instr-generate -fno-coverage-mapping") - endif() endif () set (COMPILER_FLAGS "${COMPILER_FLAGS}") diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 8ab3c8a0711..f9bf413a6c8 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -1,3 +1,5 @@ +add_compile_options($<$,$>:${COVERAGE_FLAGS}>) + if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") endif () diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index 1027638be3d..60eb6fcac72 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -1,11 +1,15 @@ #include "coverage.h" -#if WITH_COVERAGE - #pragma GCC diagnostic ignored "-Wreserved-identifier" -# include -# include + +/// WITH_COVERAGE enables the default implementation of code coverage, +/// that dumps a map to the filesystem. + +#if WITH_COVERAGE + +#include +#include # if defined(__clang__) @@ -31,3 +35,97 @@ void dumpCoverageReportIfPossible() #endif } + + +/// SANITIZE_COVERAGE enables code instrumentation, +/// but leaves the callbacks implementation to us, +/// which we use to calculate coverage on a per-test basis +/// and to write it to system tables. + +#if defined(SANITIZE_COVERAGE) + +namespace +{ + bool initialized = false; + + uint32_t * guards_start = nullptr; + uint32_t * guards_end = nullptr; + size_t coverage_array_size = 0; + + uintptr_t * coverage_array = nullptr; +} + +extern "C" +{ + +/// This is called at least once for every DSO for initialization. +/// But we will use it only for the main DSO. +void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop) +{ + if (initialized) + return; + initialized = true; + + /// The function can be called multiple times, but we need to initialize only once. + if (start == stop || *start) + return; + + guards_start = start; + guards_end = stop; + coverage_array_size = stop - start; + + /// Note: we will leak this. + coverage_array = static_cast(malloc(sizeof(uintptr_t) * coverage_array_size)); + + resetCoverage(); +} + +/// This is called at every basic block / edge, etc. +void __sanitizer_cov_trace_pc_guard(uint32_t * guard) +{ + /// Duplicate the guard check. + if (!*guard) + return; + *guard = 0; + + /// If you set *guard to 0 this code will not be called again for this edge. + /// Now we can get the PC and do whatever you want: + /// - store it somewhere or symbolize it and print right away. + /// The values of `*guard` are as you set them in + /// __sanitizer_cov_trace_pc_guard_init and so you can make them consecutive + /// and use them to dereference an array or a bit vector. + void * pc = __builtin_return_address(0); + + coverage_array[guard - guards_start] = reinterpret_cast(pc); +} + +} + +__attribute__((no_sanitize("coverage"))) std::span getCoverage() +{ + return {coverage_array, coverage_array_size}; +} + +__attribute__((no_sanitize("coverage"))) void resetCoverage() +{ + memset(coverage_array, 0, coverage_array_size * sizeof(*coverage_array)); + + /// The guard defines whether the __sanitizer_cov_trace_pc_guard should be called. + /// For example, you can unset it after first invocation to prevent excessive work. + /// Initially set all the guards to 1 to enable callbacks. + for (uint32_t * x = guards_start; x < guards_end; ++x) + *x = 1; +} + +#else + +std::span getCoverage() +{ + return {}; +} + +void resetCoverage() +{ +} + +#endif diff --git a/base/base/coverage.h b/base/base/coverage.h index 4a57528b0ce..b6664bec223 100644 --- a/base/base/coverage.h +++ b/base/base/coverage.h @@ -1,5 +1,8 @@ #pragma once +#include +#include + /// Flush coverage report to file, depending on coverage system /// proposed by compiler (llvm for clang and gcov for gcc). /// @@ -7,3 +10,6 @@ /// Thread safe (use exclusive lock). /// Idempotent, may be called multiple times. void dumpCoverageReportIfPossible(); + +std::span getCoverage(); +void resetCoverage(); diff --git a/base/glibc-compatibility/memcpy/memcpy.cpp b/base/glibc-compatibility/memcpy/memcpy.cpp index ec43a2c3649..8bab35934d3 100644 --- a/base/glibc-compatibility/memcpy/memcpy.cpp +++ b/base/glibc-compatibility/memcpy/memcpy.cpp @@ -1,5 +1,6 @@ #include "memcpy.h" +__attribute__((no_sanitize("coverage"))) extern "C" void * memcpy(void * __restrict dst, const void * __restrict src, size_t size) { return inline_memcpy(dst, src, size); diff --git a/base/glibc-compatibility/memcpy/memcpy.h b/base/glibc-compatibility/memcpy/memcpy.h index 0930dfb5c67..86439dda061 100644 --- a/base/glibc-compatibility/memcpy/memcpy.h +++ b/base/glibc-compatibility/memcpy/memcpy.h @@ -93,7 +93,7 @@ * See https://habr.com/en/company/yandex/blog/457612/ */ - +__attribute__((no_sanitize("coverage"))) static inline void * inline_memcpy(void * __restrict dst_, const void * __restrict src_, size_t size) { /// We will use pointer arithmetic, so char pointer will be used. diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index f17283774eb..0c901f1aa36 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -58,3 +58,21 @@ if (SANITIZE) message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}") endif () endif() + +# Default coverage instrumentation (dumping the coverage map on exit) +option(WITH_COVERAGE "Instrumentation for code coverage with default implementation" OFF) + +if (WITH_COVERAGE) + message (INFORMATION "Enabled instrumentation for code coverage") + set(COVERAGE_FLAGS "-fprofile-instr-generate -fcoverage-mapping") +endif() + +option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) + +if (SANITIZE_COVERAGE) + message (INFORMATION "Enabled instrumentation for code coverage") + add_definitions(-DSANITIZE_COVERAGE=1) + set (COVERAGE_FLAGS "-fsanitize-coverage=trace-pc-guard") +endif() + +set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 390b0241e7d..fa97e59eefc 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -3,15 +3,6 @@ set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") -if (WITH_COVERAGE) - set (WITHOUT_COVERAGE_LIST ${WITHOUT_COVERAGE}) - separate_arguments(WITHOUT_COVERAGE_LIST) - # disable coverage for contib files and build with optimisations - if (COMPILER_CLANG) - add_compile_options(-O3 -DNDEBUG -finline-functions -finline-hint-functions ${WITHOUT_COVERAGE_LIST}) - endif() -endif() - if (SANITIZE STREQUAL "undefined") # 3rd-party libraries usually not intended to work with UBSan. add_compile_options(-fno-sanitize=undefined) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 268f0fbe0e4..fbb7d6ea018 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -278,38 +278,6 @@ else () COMMAND_ECHO STDOUT) endif () -# add_custom_command ( -# OUTPUT ${PROTOC_BUILD_DIR} -# COMMAND mkdir -p ${PROTOC_BUILD_DIR}) -# -# add_custom_command ( -# OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" -# -# COMMAND ${CMAKE_COMMAND} -# -G"${CMAKE_GENERATOR}" -# -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" -# -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" -# -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" -# -Dprotobuf_BUILD_TESTS=0 -# -Dprotobuf_BUILD_CONFORMANCE=0 -# -Dprotobuf_BUILD_EXAMPLES=0 -# -Dprotobuf_BUILD_PROTOC_BINARIES=1 -# "${protobuf_source_dir}/cmake" -# -# DEPENDS "${PROTOC_BUILD_DIR}" -# WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" -# COMMENT "Configuring 'protoc' for host architecture." -# USES_TERMINAL) -# -# add_custom_command ( -# OUTPUT "${PROTOC_BUILD_DIR}/protoc" -# COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" -# DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" -# COMMENT "Building 'protoc' for host architecture." -# USES_TERMINAL) -# -# add_custom_target (protoc-host DEPENDS "${PROTOC_BUILD_DIR}/protoc") - add_executable(protoc IMPORTED GLOBAL) set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc") add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc") diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index b7e59e2c9a3..c77d5d8319e 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -1,5 +1,3 @@ -include(CheckCXXCompilerFlag) - set(LIBCXX_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm-project/libcxx") set(SRCS diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index eb4a898d472..fce6894ed11 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -1,3 +1,5 @@ +add_compile_options($<$,$>:${COVERAGE_FLAGS}>) + if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") endif () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d2985665db3..f88a6cff6c0 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -1,3 +1,5 @@ +add_compile_options($<$,$>:${COVERAGE_FLAGS}>) + if (USE_INCLUDE_WHAT_YOU_USE) set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH}) endif () @@ -282,7 +284,8 @@ set_source_files_properties( Common/Elf.cpp Common/Dwarf.cpp Common/SymbolIndex.cpp - PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE}") + Common/ThreadFuzzer.cpp + PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE_FLAGS}") target_link_libraries (clickhouse_common_io PRIVATE diff --git a/src/Functions/coverage.cpp b/src/Functions/coverage.cpp new file mode 100644 index 00000000000..1825e6aa826 --- /dev/null +++ b/src/Functions/coverage.cpp @@ -0,0 +1,91 @@ +#if defined(SANITIZE_COVERAGE) + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace +{ + +/** If ClickHouse is build with coverage instrumentation, returns an array + * of currently accumulated unique code addresses. + */ +class FunctionCoverage : public IFunction +{ +public: + static constexpr auto name = "coverage"; + + String getName() const override + { + return name; + } + + explicit FunctionCoverage() + { + } + + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override + { + return false; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isDeterministic() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(std::make_shared()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + auto coverage_table = getCoverage(); + + auto column_addresses = ColumnUInt64::create(); + auto & data = column_addresses->getData(); + + for (auto ptr : coverage_table) + if (ptr) + data.push_back(ptr); + + auto column_array = ColumnArray::create( + std::move(column_addresses), + ColumnArray::ColumnOffsets::create(1, data.size())); + + return ColumnConst::create(std::move(column_array), input_rows_count); + } +}; + +} + +REGISTER_FUNCTION(Coverage) +{ + factory.registerFunction(); +} + +} + +#endif diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 07a1ae7d170..4e1d32bd3cb 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -60,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -690,6 +691,12 @@ BlockIO InterpreterSystemQuery::execute() FailPointInjection::disableFailPoint(query.fail_point_name); break; } + case Type::RESET_COVERAGE: + { + getContext()->checkAccess(AccessType::SYSTEM); + resetCoverage(); + break; + } default: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown type of SYSTEM query"); } @@ -1299,6 +1306,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::START_THREAD_FUZZER: case Type::ENABLE_FAILPOINT: case Type::DISABLE_FAILPOINT: + case Type::RESET_COVERAGE: case Type::UNKNOWN: case Type::END: break; } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index cc06e0fdcb5..5f7ba5be330 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -86,6 +86,7 @@ public: START_PULLING_REPLICATION_LOG, STOP_CLEANUP, START_CLEANUP, + RESET_COVERAGE, END }; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index a26fdc1396b..f0fc38d6adb 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -453,14 +453,14 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & } case Type::DROP_FORMAT_SCHEMA_CACHE: { - if (ParserKeyword{"FOR"}.ignore(pos, expected)) - { - if (ParserKeyword{"Protobuf"}.ignore(pos, expected)) - res->schema_cache_format = "Protobuf"; - else - return false; - } - break; + if (ParserKeyword{"FOR"}.ignore(pos, expected)) + { + if (ParserKeyword{"Protobuf"}.ignore(pos, expected)) + res->schema_cache_format = "Protobuf"; + else + return false; + } + break; } case Type::UNFREEZE: { From 3142921bb4dcb8b7169f7d32a05110c9a5baa351 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 11:15:11 +0100 Subject: [PATCH 074/813] Taming query profiler --- src/Common/ProfileEvents.cpp | 1 + src/Common/QueryProfiler.cpp | 12 ++++++++++++ 2 files changed, 13 insertions(+) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f4d7242f70a..aadbc8b2471 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -316,6 +316,7 @@ The server successfully detected this situation and will download merged part fr \ M(CannotWriteToWriteBufferDiscard, "Number of stack traces dropped by query profiler or signal handler because pipe is full or cannot write to pipe.") \ M(QueryProfilerSignalOverruns, "Number of times we drop processing of a query profiler signal due to overrun plus the number of signals that OS has not delivered due to overrun.") \ + M(QueryProfilerConcurrencyOverruns, "Number of times we drop processing of a query profiler signal due to too many concurrent query profilers in other threads, which may indicate overload.") \ M(QueryProfilerRuns, "Number of times QueryProfiler had been run.") \ \ M(CreatedLogEntryForMerge, "Successfully created log entry to merge parts in ReplicatedMergeTree.") \ diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index dc9f3610513..c656e7f992f 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -22,6 +22,7 @@ namespace CurrentMetrics namespace ProfileEvents { extern const Event QueryProfilerSignalOverruns; + extern const Event QueryProfilerConcurrencyOverruns; extern const Event QueryProfilerRuns; } @@ -40,8 +41,19 @@ namespace /// to ignore delivered signals after timer_delete(). thread_local bool signal_handler_disarmed = true; + /// Don't permit too many threads be busy inside profiler, + /// which could slow down the system in some environments. + std::atomic concurrent_invocations = 0; + void writeTraceInfo(TraceType trace_type, int /* sig */, siginfo_t * info, void * context) { + SCOPE_EXIT({ concurrent_invocations.fetch_sub(1, std::memory_order_relaxed); }); + if (concurrent_invocations.fetch_add(1, std::memory_order_relaxed) > 100) + { + ProfileEvents::incrementNoTrace(ProfileEvents::QueryProfilerConcurrencyOverruns); + return; + } + auto saved_errno = errno; /// We must restore previous value of errno in signal handler. #if defined(OS_LINUX) From e15815ee974dafada9ec0de9996d2d29eb26e6cb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 11:32:45 +0100 Subject: [PATCH 075/813] Add a test --- ...ry_profiler_concurrency_overruns.reference | 1 + ...907_query_profiler_concurrency_overruns.sh | 20 +++++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.reference create mode 100755 tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh diff --git a/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.reference b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.reference new file mode 100644 index 00000000000..8f75b7cccf2 --- /dev/null +++ b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.reference @@ -0,0 +1 @@ +1000000000 1 1 diff --git a/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh new file mode 100755 index 00000000000..f3f37704e23 --- /dev/null +++ b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash +# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-fasttest, no-cpu-aarch64 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# The check is probablistic, so make sure that it passes at least sometimes: + +while true +do + ${CLICKHOUSE_CLIENT} -n --query=" + SELECT count() FROM numbers_mt(1000000000) SETTINGS + query_profiler_real_time_period_ns = 1000000, + query_profiler_cpu_time_period_ns = 1000000, + max_threads = 1000; + SELECT anyIf(value, event = 'QueryProfilerRuns') > 0, anyIf(value, event = 'QueryProfilerConcurrencyOverruns') > 0 FROM system.events; + " | tr '\t\n' ' ' | grep '1000000000 1 1' && break + sleep 1 +done From 56de2333f9c69097e57ec2134f6270271f1d5b3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 16:55:47 +0100 Subject: [PATCH 076/813] Add warning --- programs/server/Server.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index af460ccc7d9..854168a2041 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -666,6 +666,10 @@ try global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif +#if defined(SANITIZE_COVERAGE) || WITH_COVERAGE + global_context->addWarningMessage("Server was built with code coverage. It will work slowly."); +#endif + const size_t physical_server_memory = getMemoryAmount(); LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", From 8e0f48738710f2715f04006db9ddfb0d76c0a865 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 17:21:45 +0100 Subject: [PATCH 077/813] Initial support in clickhouse-test --- cmake/sanitize.cmake | 8 +++++++- tests/clickhouse-test | 15 +++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 0c901f1aa36..7d25a85ef62 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -71,7 +71,13 @@ option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbac if (SANITIZE_COVERAGE) message (INFORMATION "Enabled instrumentation for code coverage") - add_definitions(-DSANITIZE_COVERAGE=1) + + # We set this define for whole build to indicate that at least some parts are compiled with coverage. + # And to expose it in system.build_options. + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -DSANITIZE_COVERAGE=1") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -DSANITIZE_COVERAGE=1") + + # But the actual coverage will be enabled on per-library basis: for ClickHouse code, but not for 3rd-party. set (COVERAGE_FLAGS "-fsanitize-coverage=trace-pc-guard") endif() diff --git a/tests/clickhouse-test b/tests/clickhouse-test index cab7d7e79ff..2a4ed865dd5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1173,6 +1173,16 @@ class TestCase: description_full += result.reason.value description_full += result.description + + if BuildFlags.SANITIZE_COVERAGE in args.build_flags: + coverage = clickhouse_execute( + args, + f"SELECT length(coverage())", + retry_error_codes=True, + ).decode() + + description_full += f" Coverage: {coverage}" + description_full += "\n" if result.status == TestStatus.FAIL and self.testcase_args: @@ -1872,6 +1882,7 @@ class BuildFlags: UNDEFINED = "ubsan" MEMORY = "msan" DEBUG = "debug" + SANITIZE_COVERAGE = "sanitize-coverage" RELEASE = "release" ORDINARY_DATABASE = "ordinary-database" POLYMORPHIC_PARTS = "polymorphic-parts" @@ -1891,6 +1902,8 @@ def collect_build_flags(args): result.append(BuildFlags.UNDEFINED) elif b"-fsanitize=memory" in value: result.append(BuildFlags.MEMORY) + elif b"-DSANITIZE_COVERAGE=1" in value: + result.append(BuildFlags.SANITIZE_COVERAGE) value = clickhouse_execute( args, "SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'" @@ -2072,6 +2085,8 @@ def reportCoverageFor(args, what, query, permissive=False): return True +# This is high-level coverage on per-component basis (functions, data types, etc.) +# Don't be confused with the code coverage. def reportCoverage(args): clickhouse_execute(args, "SYSTEM FLUSH LOGS") From 4288cb3b7895df917f982e03d0d0b55029ecc5cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 17:43:01 +0100 Subject: [PATCH 078/813] Make clickhouse-test to calculate coverage on a per-test basis --- tests/clickhouse-test | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2a4ed865dd5..e827a596ada 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1175,6 +1175,12 @@ class TestCase: description_full += result.description if BuildFlags.SANITIZE_COVERAGE in args.build_flags: + clickhouse_execute( + args, + f"INSERT INTO system.coverage SELECT '{self.case}', coverage()", + retry_error_codes=True, + ) + coverage = clickhouse_execute( args, f"SELECT length(coverage())", @@ -1241,6 +1247,14 @@ class TestCase: + pattern ) + # We want to calculate per-test code coverage. That's why we reset it before each test. + if BuildFlags.SANITIZE_COVERAGE in args.build_flags: + clickhouse_execute( + args, + "SYSTEM RESET COVERAGE", + retry_error_codes=True, + ) + command = pattern.format(**params) proc = Popen(command, shell=True, env=os.environ) @@ -2349,6 +2363,18 @@ def main(args): print(f"Failed to create databases for tests: {e}") server_died.set() + if BuildFlags.SANITIZE_COVERAGE in args.build_flags: + clickhouse_execute( + args, + """ + CREATE TABLE IF NOT EXISTS system.coverage + ( + test_name String, + coverage Array(UInt64) + ) ENGINE = MergeTree ORDER BY test_name; + """, + ) + total_tests_run = 0 for suite in sorted(os.listdir(base_dir), key=suite_key_func): From 8e6a7fdff09430378a6b13e87ded874524327e3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 23:43:36 +0100 Subject: [PATCH 079/813] Fix f-string --- CMakeLists.txt | 7 ++++--- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- tests/clickhouse-test | 8 +++++++- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d259b105a0a..d19bb521c70 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -187,9 +187,10 @@ if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") endif () endif() -if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" - OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" - OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") +if (NOT (SANITIZE_COVERAGE OR WITH_COVERAGE) + AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" + OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" + OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL")) set (OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT ON) else() set (OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT OFF) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 069ed20c730..705972da8f4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8795,7 +8795,7 @@ void StorageReplicatedMergeTree::createTableSharedID() const else if (code == Coordination::Error::ZNONODE) /// table completely dropped, we can choose any id we want { id = toString(UUIDHelpers::Nil); - LOG_DEBUG(log, "Table was completely drop, we can use anything as ID (will use {})", id); + LOG_DEBUG(log, "Table was completely dropped, and we can use anything as ID (will use {})", id); } else if (code != Coordination::Error::ZOK) { diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e827a596ada..e5659e8fca4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1183,7 +1183,7 @@ class TestCase: coverage = clickhouse_execute( args, - f"SELECT length(coverage())", + "SELECT length(coverage())", retry_error_codes=True, ).decode() @@ -2375,6 +2375,12 @@ def main(args): """, ) + # Coverage collected at the system startup before running any tests: + clickhouse_execute( + args, + "INSERT INTO system.coverage SELECT '', coverage()", + ) + total_tests_run = 0 for suite in sorted(os.listdir(base_dir), key=suite_key_func): From f85e9138da6990fa95ec1c757cdf6207e6040ddc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 02:31:54 +0300 Subject: [PATCH 080/813] Update 02907_query_profiler_concurrency_overruns.sh --- .../0_stateless/02907_query_profiler_concurrency_overruns.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh index f3f37704e23..c43889d78b2 100755 --- a/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh +++ b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh @@ -10,10 +10,11 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) while true do ${CLICKHOUSE_CLIENT} -n --query=" - SELECT count() FROM numbers_mt(1000000000) SETTINGS + SELECT count() FROM numbers_mt(1000000) SETTINGS query_profiler_real_time_period_ns = 1000000, query_profiler_cpu_time_period_ns = 1000000, - max_threads = 1000; + max_threads = 1000, + max_block_size = 100; SELECT anyIf(value, event = 'QueryProfilerRuns') > 0, anyIf(value, event = 'QueryProfilerConcurrencyOverruns') > 0 FROM system.events; " | tr '\t\n' ' ' | grep '1000000000 1 1' && break sleep 1 From c544a0221010d812c68fb805aee3eed1a252b50c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 02:32:22 +0300 Subject: [PATCH 081/813] Update 02907_query_profiler_concurrency_overruns.sh --- .../0_stateless/02907_query_profiler_concurrency_overruns.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh index c43889d78b2..7c5e4209124 100755 --- a/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh +++ b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.sh @@ -10,12 +10,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) while true do ${CLICKHOUSE_CLIENT} -n --query=" - SELECT count() FROM numbers_mt(1000000) SETTINGS + SELECT count() FROM zeros_mt(1000000) SETTINGS query_profiler_real_time_period_ns = 1000000, query_profiler_cpu_time_period_ns = 1000000, max_threads = 1000, max_block_size = 100; SELECT anyIf(value, event = 'QueryProfilerRuns') > 0, anyIf(value, event = 'QueryProfilerConcurrencyOverruns') > 0 FROM system.events; - " | tr '\t\n' ' ' | grep '1000000000 1 1' && break + " | tr '\t\n' ' ' | grep '1000000 1 1' && break sleep 1 done From e6644c17736a07ad32dcf0a848dec05a94a3505a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 02:32:32 +0300 Subject: [PATCH 082/813] Update 02907_query_profiler_concurrency_overruns.reference --- .../02907_query_profiler_concurrency_overruns.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.reference b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.reference index 8f75b7cccf2..45d53fbec54 100644 --- a/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.reference +++ b/tests/queries/0_stateless/02907_query_profiler_concurrency_overruns.reference @@ -1 +1 @@ -1000000000 1 1 +1000000 1 1 From ea6cb1ad0c95f194519c863bb29302e8829669a3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 01:04:50 +0100 Subject: [PATCH 083/813] Maybe better --- tests/clickhouse-test | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e5659e8fca4..36846a4aeb1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1177,7 +1177,7 @@ class TestCase: if BuildFlags.SANITIZE_COVERAGE in args.build_flags: clickhouse_execute( args, - f"INSERT INTO system.coverage SELECT '{self.case}', coverage()", + f"INSERT INTO system.coverage SELECT now(), '{self.case}', coverage()", retry_error_codes=True, ) @@ -2369,6 +2369,7 @@ def main(args): """ CREATE TABLE IF NOT EXISTS system.coverage ( + time DateTime, test_name String, coverage Array(UInt64) ) ENGINE = MergeTree ORDER BY test_name; @@ -2378,7 +2379,7 @@ def main(args): # Coverage collected at the system startup before running any tests: clickhouse_execute( args, - "INSERT INTO system.coverage SELECT '', coverage()", + "INSERT INTO system.coverage SELECT now(), '', coverage()", ) total_tests_run = 0 From ccf5003442eff0b60cafad3faa489fc2c7ff1aa0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 03:20:04 +0100 Subject: [PATCH 084/813] Maybe smaller binary --- src/Functions/geometryConverters.h | 5 +++++ src/Functions/polygonArea.cpp | 4 ++++ src/Functions/polygonConvexHull.cpp | 5 ++++- src/Functions/polygonPerimeter.cpp | 5 +++++ src/Functions/polygonsDistance.cpp | 5 ++++- src/Functions/polygonsEquals.cpp | 7 ++++++- src/Functions/polygonsIntersection.cpp | 5 ++++- src/Functions/polygonsSymDifference.cpp | 4 ++++ src/Functions/polygonsUnion.cpp | 10 ++++------ src/Functions/polygonsWithin.cpp | 3 +++ src/Functions/readWkt.cpp | 4 ++++ src/Functions/svg.cpp | 5 +++++ src/Functions/wkt.cpp | 6 ++++++ 13 files changed, 58 insertions(+), 10 deletions(-) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 97162fa9dd0..dba984b4184 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -28,6 +28,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + template using Ring = boost::geometry::model::ring; @@ -371,3 +374,5 @@ static void callOnTwoGeometryDataTypes(DataTypePtr left_type, DataTypePtr right_ } } + +} diff --git a/src/Functions/polygonArea.cpp b/src/Functions/polygonArea.cpp index e49a4eb9fb3..1c4ef9f79a3 100644 --- a/src/Functions/polygonArea.cpp +++ b/src/Functions/polygonArea.cpp @@ -26,6 +26,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + template class FunctionPolygonArea : public IFunction { @@ -99,6 +102,7 @@ const char * FunctionPolygonArea::name = "polygonAreaCartesian"; template <> const char * FunctionPolygonArea::name = "polygonAreaSpherical"; +} REGISTER_FUNCTION(PolygonArea) { diff --git a/src/Functions/polygonConvexHull.cpp b/src/Functions/polygonConvexHull.cpp index d7fca45bd1e..921c0700ca7 100644 --- a/src/Functions/polygonConvexHull.cpp +++ b/src/Functions/polygonConvexHull.cpp @@ -25,6 +25,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + template class FunctionPolygonConvexHull : public IFunction { @@ -94,10 +97,10 @@ public: } }; - template <> const char * FunctionPolygonConvexHull::name = "polygonConvexHullCartesian"; +} REGISTER_FUNCTION(PolygonConvexHull) { diff --git a/src/Functions/polygonPerimeter.cpp b/src/Functions/polygonPerimeter.cpp index 2d89d4e4f5b..85645118f84 100644 --- a/src/Functions/polygonPerimeter.cpp +++ b/src/Functions/polygonPerimeter.cpp @@ -17,13 +17,17 @@ #include #include + namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ template class FunctionPolygonPerimeter : public IFunction @@ -97,6 +101,7 @@ const char * FunctionPolygonPerimeter::name = "polygonPerimeterC template <> const char * FunctionPolygonPerimeter::name = "polygonPerimeterSpherical"; +} REGISTER_FUNCTION(PolygonPerimeter) { diff --git a/src/Functions/polygonsDistance.cpp b/src/Functions/polygonsDistance.cpp index d2c58105eae..d6c7d799b5e 100644 --- a/src/Functions/polygonsDistance.cpp +++ b/src/Functions/polygonsDistance.cpp @@ -27,6 +27,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + template class FunctionPolygonsDistance : public IFunction { @@ -108,6 +111,7 @@ const char * FunctionPolygonsDistance::name = "polygonsDistanceC template <> const char * FunctionPolygonsDistance::name = "polygonsDistanceSpherical"; +} REGISTER_FUNCTION(PolygonsDistance) { @@ -115,5 +119,4 @@ REGISTER_FUNCTION(PolygonsDistance) factory.registerFunction>(); } - } diff --git a/src/Functions/polygonsEquals.cpp b/src/Functions/polygonsEquals.cpp index 3c80ae1e4c5..bdc4f18042c 100644 --- a/src/Functions/polygonsEquals.cpp +++ b/src/Functions/polygonsEquals.cpp @@ -19,13 +19,18 @@ #include #include + namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + template class FunctionPolygonsEquals : public IFunction { @@ -103,10 +108,10 @@ public: } }; - template <> const char * FunctionPolygonsEquals::name = "polygonsEqualsCartesian"; +} REGISTER_FUNCTION(PolygonsEquals) { diff --git a/src/Functions/polygonsIntersection.cpp b/src/Functions/polygonsIntersection.cpp index 84e5fe0d4b7..5777f438a19 100644 --- a/src/Functions/polygonsIntersection.cpp +++ b/src/Functions/polygonsIntersection.cpp @@ -26,6 +26,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + template class FunctionPolygonsIntersection : public IFunction { @@ -107,13 +110,13 @@ public: } }; - template <> const char * FunctionPolygonsIntersection::name = "polygonsIntersectionCartesian"; template <> const char * FunctionPolygonsIntersection::name = "polygonsIntersectionSpherical"; +} REGISTER_FUNCTION(PolygonsIntersection) { diff --git a/src/Functions/polygonsSymDifference.cpp b/src/Functions/polygonsSymDifference.cpp index ceb39547427..785a8f76ba6 100644 --- a/src/Functions/polygonsSymDifference.cpp +++ b/src/Functions/polygonsSymDifference.cpp @@ -25,6 +25,8 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ template class FunctionPolygonsSymDifference : public IFunction @@ -109,6 +111,8 @@ const char * FunctionPolygonsSymDifference::name = "polygonsSymD template <> const char * FunctionPolygonsSymDifference::name = "polygonsSymDifferenceSpherical"; +} + REGISTER_FUNCTION(PolygonsSymDifference) { factory.registerFunction>(); diff --git a/src/Functions/polygonsUnion.cpp b/src/Functions/polygonsUnion.cpp index 4a604d0f810..a31d223ea8c 100644 --- a/src/Functions/polygonsUnion.cpp +++ b/src/Functions/polygonsUnion.cpp @@ -3,19 +3,14 @@ #include #include -#include #include -#include #include -#include -#include #include -#include #include -#include + namespace DB { @@ -25,6 +20,8 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ template class FunctionPolygonsUnion : public IFunction @@ -112,6 +109,7 @@ const char * FunctionPolygonsUnion::name = "polygonsUnionCartesi template <> const char * FunctionPolygonsUnion::name = "polygonsUnionSpherical"; +} REGISTER_FUNCTION(PolygonsUnion) { diff --git a/src/Functions/polygonsWithin.cpp b/src/Functions/polygonsWithin.cpp index 1b094f42060..bf4db1cf9f8 100644 --- a/src/Functions/polygonsWithin.cpp +++ b/src/Functions/polygonsWithin.cpp @@ -27,6 +27,8 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ template class FunctionPolygonsWithin : public IFunction @@ -111,6 +113,7 @@ const char * FunctionPolygonsWithin::name = "polygonsWithinCarte template <> const char * FunctionPolygonsWithin::name = "polygonsWithinSpherical"; +} REGISTER_FUNCTION(PolygonsWithin) { diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index ec20cdf3723..8dff297bcb1 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -16,6 +16,8 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ template class FunctionReadWKT : public IFunction @@ -95,6 +97,8 @@ struct ReadWKTMultiPolygonNameHolder static constexpr const char * name = "readWKTMultiPolygon"; }; +} + REGISTER_FUNCTION(ReadWKT) { factory.registerFunction, ReadWKTPointNameHolder>>(); diff --git a/src/Functions/svg.cpp b/src/Functions/svg.cpp index f8f85216b3f..550fe29cfc4 100644 --- a/src/Functions/svg.cpp +++ b/src/Functions/svg.cpp @@ -16,6 +16,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + class FunctionSvg : public IFunction { public: @@ -94,6 +97,8 @@ public: } }; +} + REGISTER_FUNCTION(Svg) { factory.registerFunction(); diff --git a/src/Functions/wkt.cpp b/src/Functions/wkt.cpp index fc9ef75a1e2..afcfabd0bf4 100644 --- a/src/Functions/wkt.cpp +++ b/src/Functions/wkt.cpp @@ -9,6 +9,9 @@ namespace DB { +namespace +{ + class FunctionWkt : public IFunction { public: @@ -52,6 +55,7 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { std::stringstream str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + str.exceptions(std::ios::failbit); str << boost::geometry::wkt(figures[i]); std::string serialized = str.str(); res_column->insertData(serialized.c_str(), serialized.size()); @@ -68,6 +72,8 @@ public: } }; +} + REGISTER_FUNCTION(Wkt) { factory.registerFunction(); From aaca32f6a7f5c84ce9be36f2d7864c3f80a56b4f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 03:22:49 +0100 Subject: [PATCH 085/813] Add function coverageAll --- base/base/coverage.cpp | 46 +++++++++++++++++++++++++++++++++----- base/base/coverage.h | 12 +++++++++- cmake/sanitize.cmake | 4 ++-- src/Functions/coverage.cpp | 27 ++++++++++++---------- 4 files changed, 68 insertions(+), 21 deletions(-) diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index 60eb6fcac72..4af6a279af9 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -46,13 +46,17 @@ void dumpCoverageReportIfPossible() namespace { - bool initialized = false; + bool pc_guards_initialized = false; + bool pc_table_initialized = false; uint32_t * guards_start = nullptr; uint32_t * guards_end = nullptr; - size_t coverage_array_size = 0; uintptr_t * coverage_array = nullptr; + size_t coverage_array_size = 0; + + uintptr_t * all_addresses_array = nullptr; + size_t all_addresses_array_size = 0; } extern "C" @@ -62,9 +66,9 @@ extern "C" /// But we will use it only for the main DSO. void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop) { - if (initialized) + if (pc_guards_initialized) return; - initialized = true; + pc_guards_initialized = true; /// The function can be called multiple times, but we need to initialize only once. if (start == stop || *start) @@ -80,6 +84,23 @@ void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop) resetCoverage(); } +/// This is called at least once for every DSO for initialization +/// and provides information about all instrumented addresses. +void __sanitizer_cov_pcs_init(const uintptr_t * pcs_begin, const uintptr_t * pcs_end) +{ + if (pc_table_initialized) + return; + pc_table_initialized = true; + + all_addresses_array = static_cast(malloc(sizeof(uintptr_t) * coverage_array_size)); + all_addresses_array_size = pcs_end - pcs_begin; + + /// They are not a real pointers, but also contain a flag in the most significant bit, + /// in which we are not interested for now. Reset it. + for (size_t i = 0; i < all_addresses_array_size; ++i) + all_addresses_array[i] = pcs_begin[i] & 0x7FFFFFFFFFFFFFFFULL; +} + /// This is called at every basic block / edge, etc. void __sanitizer_cov_trace_pc_guard(uint32_t * guard) { @@ -101,12 +122,20 @@ void __sanitizer_cov_trace_pc_guard(uint32_t * guard) } -__attribute__((no_sanitize("coverage"))) std::span getCoverage() +__attribute__((no_sanitize("coverage"))) +std::span getCoverage() { return {coverage_array, coverage_array_size}; } -__attribute__((no_sanitize("coverage"))) void resetCoverage() +__attribute__((no_sanitize("coverage"))) +std::span getAllInstrumentedAddresses() +{ + return {all_addresses_array, all_addresses_array_size}; +} + +__attribute__((no_sanitize("coverage"))) +void resetCoverage() { memset(coverage_array, 0, coverage_array_size * sizeof(*coverage_array)); @@ -124,6 +153,11 @@ std::span getCoverage() return {}; } +std::span getAllInstrumentedAddresses() +{ + return {}; +} + void resetCoverage() { } diff --git a/base/base/coverage.h b/base/base/coverage.h index b6664bec223..f75ed2d3553 100644 --- a/base/base/coverage.h +++ b/base/base/coverage.h @@ -11,5 +11,15 @@ /// Idempotent, may be called multiple times. void dumpCoverageReportIfPossible(); -std::span getCoverage(); +/// This is effective if SANITIZE_COVERAGE is enabled at build time. +/// Get accumulated unique program addresses of the instrumented parts of the code, +/// seen so far after program startup or after previous reset. +/// The returned span will be represented as a sparse map, containing mostly zeros, which you should filter away. +std::span getCoverage(); + +/// Get all instrumented addresses that could be in the coverage. +std::span getAllInstrumentedAddresses(); + +/// Reset the accumulated coverage. +/// This is useful to compare coverage of different tests, including differential coverage. void resetCoverage(); diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 7d25a85ef62..3f7a8498059 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -78,7 +78,7 @@ if (SANITIZE_COVERAGE) set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -DSANITIZE_COVERAGE=1") # But the actual coverage will be enabled on per-library basis: for ClickHouse code, but not for 3rd-party. - set (COVERAGE_FLAGS "-fsanitize-coverage=trace-pc-guard") + set (COVERAGE_FLAGS "-fsanitize-coverage=trace-pc-guard,pc-table") endif() -set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard") +set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table") diff --git a/src/Functions/coverage.cpp b/src/Functions/coverage.cpp index 1825e6aa826..8a62469fa54 100644 --- a/src/Functions/coverage.cpp +++ b/src/Functions/coverage.cpp @@ -18,28 +18,30 @@ namespace DB namespace { +enum class Kind +{ + Current, + All +}; + /** If ClickHouse is build with coverage instrumentation, returns an array - * of currently accumulated unique code addresses. + * of currently accumulated (`coverage`) / all possible (`coverageAll`) unique code addresses. */ class FunctionCoverage : public IFunction { -public: - static constexpr auto name = "coverage"; +private: + Kind kind; +public: String getName() const override { - return name; + return kind == Kind::Current ? "coverage" : "coverageAll"; } - explicit FunctionCoverage() + explicit FunctionCoverage(Kind kind_) : kind(kind_) { } - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; @@ -62,7 +64,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { - auto coverage_table = getCoverage(); + auto coverage_table = kind == Kind::Current ? getCoverage() : getAllInstrumentedAddresses(); auto column_addresses = ColumnUInt64::create(); auto & data = column_addresses->getData(); @@ -83,7 +85,8 @@ public: REGISTER_FUNCTION(Coverage) { - factory.registerFunction(); + factory.registerFunction("coverage", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::Current)); }); + factory.registerFunction("coverageAll", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::All)); }); } } From ad4bde6b8bcafcb9c87454cbeeb0448533279e07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 03:54:11 +0100 Subject: [PATCH 086/813] Fix build --- base/base/coverage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index 4af6a279af9..d70c3bcd82b 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -148,7 +148,7 @@ void resetCoverage() #else -std::span getCoverage() +std::span getCoverage() { return {}; } From f8e209ebd26f278ed582adf0aab8f786be8bb591 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 30 Oct 2023 13:45:18 +0300 Subject: [PATCH 087/813] WindowTransform decrease amount of virtual function calls --- src/Processors/Transforms/WindowTransform.cpp | 19 ++++++++++++------- src/Processors/Transforms/WindowTransform.h | 3 +++ 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 9565a073f48..df6246510bd 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -257,6 +257,7 @@ WindowTransform::WindowTransform(const Block & input_header_, window_description.frame = *custom_default_frame; } + workspace.is_aggregate_function_state = workspace.aggregate_function->isState(); workspace.aggregate_function_state.reset( aggregate_function->sizeOfData(), aggregate_function->alignOfData()); @@ -957,10 +958,7 @@ void WindowTransform::updateAggregationState() auto * columns = ws.argument_columns.data(); // Removing arena.get() from the loop makes it faster somehow... auto * arena_ptr = arena.get(); - for (auto row = first_row; row < past_the_end_row; ++row) - { - a->add(buf, columns, row, arena_ptr); - } + a->addBatchSinglePlaceFromInterval(first_row, past_the_end_row, buf, columns, arena_ptr); } } } @@ -987,9 +985,16 @@ void WindowTransform::writeOutCurrentRow() // FIXME does it also allocate the result on the arena? // We'll have to pass it out with blocks then... - /// We should use insertMergeResultInto to insert result into ColumnAggregateFunction - /// correctly if result contains AggregateFunction's states - a->insertMergeResultInto(buf, *result_column, arena.get()); + if (ws.is_aggregate_function_state) + { + /// We should use insertMergeResultInto to insert result into ColumnAggregateFunction + /// correctly if result contains AggregateFunction's states + a->insertMergeResultInto(buf, *result_column, arena.get()); + } + else + { + a->insertResultInto(buf, *result_column, arena.get()); + } } } } diff --git a/src/Processors/Transforms/WindowTransform.h b/src/Processors/Transforms/WindowTransform.h index de3e82d15ee..347c2516230 100644 --- a/src/Processors/Transforms/WindowTransform.h +++ b/src/Processors/Transforms/WindowTransform.h @@ -26,6 +26,9 @@ struct WindowFunctionWorkspace { AggregateFunctionPtr aggregate_function; + // Cached value of aggregate function isState virtual method + bool is_aggregate_function_state = false; + // This field is set for pure window functions. When set, we ignore the // window_function.aggregate_function, and work through this interface // instead. From 136f9841540eee70917815728765f51fe916fb11 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Sep 2023 15:49:25 +0000 Subject: [PATCH 088/813] Update arrow to release-13.0.0 --- contrib/arrow | 2 +- contrib/arrow-cmake/CMakeLists.txt | 51 ++++++++---------- .../02735_parquet_encoder.reference | 8 +-- .../02884_parquet_new_encodings.reference | 1 + .../02884_parquet_new_encodings.sh | 9 ++++ .../delta_lenght_byte_array_encoding.parquet | Bin 0 -> 2795 bytes 6 files changed, 37 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/02884_parquet_new_encodings.reference create mode 100755 tests/queries/0_stateless/02884_parquet_new_encodings.sh create mode 100644 tests/queries/0_stateless/data_parquet/delta_lenght_byte_array_encoding.parquet diff --git a/contrib/arrow b/contrib/arrow index 1d93838f69a..9d9c464ce68 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 1d93838f69a802639ca144ea5704a98e2481810d +Subproject commit 9d9c464ce6883f52aaca9f913eec4cd50006c767 diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 02e809c560f..c45d75bb3f2 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -230,6 +230,8 @@ set(ARROW_SRCS "${LIBRARY_DIR}/array/builder_nested.cc" "${LIBRARY_DIR}/array/builder_primitive.cc" "${LIBRARY_DIR}/array/builder_union.cc" + "${LIBRARY_DIR}/array/builder_run_end.cc" + "${LIBRARY_DIR}/array/array_run_end.cc" "${LIBRARY_DIR}/array/concatenate.cc" "${LIBRARY_DIR}/array/data.cc" "${LIBRARY_DIR}/array/diff.cc" @@ -309,9 +311,12 @@ set(ARROW_SRCS "${LIBRARY_DIR}/util/debug.cc" "${LIBRARY_DIR}/util/tracing.cc" "${LIBRARY_DIR}/util/atfork_internal.cc" + "${LIBRARY_DIR}/util/crc32.cc" + "${LIBRARY_DIR}/util/hashing.cc" + "${LIBRARY_DIR}/util/ree_util.cc" + "${LIBRARY_DIR}/util/union_util.cc" "${LIBRARY_DIR}/vendored/base64.cpp" "${LIBRARY_DIR}/vendored/datetime/tz.cpp" - "${LIBRARY_DIR}/vendored/musl/strptime.c" "${LIBRARY_DIR}/vendored/uriparser/UriCommon.c" "${LIBRARY_DIR}/vendored/uriparser/UriCompare.c" @@ -328,39 +333,20 @@ set(ARROW_SRCS "${LIBRARY_DIR}/vendored/uriparser/UriRecompose.c" "${LIBRARY_DIR}/vendored/uriparser/UriResolve.c" "${LIBRARY_DIR}/vendored/uriparser/UriShorten.c" + "${LIBRARY_DIR}/vendored/double-conversion/bignum.cc" + "${LIBRARY_DIR}/vendored/double-conversion/bignum-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/cached-powers.cc" + "${LIBRARY_DIR}/vendored/double-conversion/double-to-string.cc" + "${LIBRARY_DIR}/vendored/double-conversion/fast-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/fixed-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/string-to-double.cc" + "${LIBRARY_DIR}/vendored/double-conversion/strtod.cc" "${LIBRARY_DIR}/compute/api_aggregate.cc" "${LIBRARY_DIR}/compute/api_scalar.cc" "${LIBRARY_DIR}/compute/api_vector.cc" "${LIBRARY_DIR}/compute/cast.cc" "${LIBRARY_DIR}/compute/exec.cc" - "${LIBRARY_DIR}/compute/exec/accumulation_queue.cc" - "${LIBRARY_DIR}/compute/exec/accumulation_queue.h" - "${LIBRARY_DIR}/compute/exec/aggregate.cc" - "${LIBRARY_DIR}/compute/exec/aggregate_node.cc" - "${LIBRARY_DIR}/compute/exec/asof_join_node.cc" - "${LIBRARY_DIR}/compute/exec/bloom_filter.cc" - "${LIBRARY_DIR}/compute/exec/exec_plan.cc" - "${LIBRARY_DIR}/compute/exec/expression.cc" - "${LIBRARY_DIR}/compute/exec/filter_node.cc" - "${LIBRARY_DIR}/compute/exec/hash_join.cc" - "${LIBRARY_DIR}/compute/exec/hash_join_dict.cc" - "${LIBRARY_DIR}/compute/exec/hash_join_node.cc" - "${LIBRARY_DIR}/compute/exec/key_hash.cc" - "${LIBRARY_DIR}/compute/exec/key_map.cc" - "${LIBRARY_DIR}/compute/exec/map_node.cc" - "${LIBRARY_DIR}/compute/exec/options.cc" - "${LIBRARY_DIR}/compute/exec/order_by_impl.cc" - "${LIBRARY_DIR}/compute/exec/partition_util.cc" - "${LIBRARY_DIR}/compute/exec/project_node.cc" - "${LIBRARY_DIR}/compute/exec/query_context.cc" - "${LIBRARY_DIR}/compute/exec/sink_node.cc" - "${LIBRARY_DIR}/compute/exec/source_node.cc" - "${LIBRARY_DIR}/compute/exec/swiss_join.cc" - "${LIBRARY_DIR}/compute/exec/task_util.cc" - "${LIBRARY_DIR}/compute/exec/tpch_node.cc" - "${LIBRARY_DIR}/compute/exec/union_node.cc" - "${LIBRARY_DIR}/compute/exec/util.cc" "${LIBRARY_DIR}/compute/function.cc" "${LIBRARY_DIR}/compute/function_internal.cc" "${LIBRARY_DIR}/compute/kernel.cc" @@ -403,8 +389,13 @@ set(ARROW_SRCS "${LIBRARY_DIR}/compute/kernels/vector_select_k.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection.cc" "${LIBRARY_DIR}/compute/kernels/vector_sort.cc" + "${LIBRARY_DIR}/compute/kernels/vector_selection_internal.cc" + "${LIBRARY_DIR}/compute/kernels/vector_selection_filter_internal.cc" + "${LIBRARY_DIR}/compute/kernels/vector_selection_take_internal.cc" "${LIBRARY_DIR}/compute/light_array.cc" "${LIBRARY_DIR}/compute/registry.cc" + "${LIBRARY_DIR}/compute/expression.cc" + "${LIBRARY_DIR}/compute/ordering.cc" "${LIBRARY_DIR}/compute/row/compare_internal.cc" "${LIBRARY_DIR}/compute/row/encode_internal.cc" "${LIBRARY_DIR}/compute/row/grouper.cc" @@ -488,10 +479,10 @@ set(PARQUET_SRCS "${LIBRARY_DIR}/exception.cc" "${LIBRARY_DIR}/file_reader.cc" "${LIBRARY_DIR}/file_writer.cc" + "${LIBRARY_DIR}/page_index.cc" "${LIBRARY_DIR}/level_conversion.cc" "${LIBRARY_DIR}/level_comparison.cc" "${LIBRARY_DIR}/metadata.cc" - "${LIBRARY_DIR}/murmur3.cc" "${LIBRARY_DIR}/platform.cc" "${LIBRARY_DIR}/printer.cc" "${LIBRARY_DIR}/properties.cc" @@ -500,6 +491,8 @@ set(PARQUET_SRCS "${LIBRARY_DIR}/stream_reader.cc" "${LIBRARY_DIR}/stream_writer.cc" "${LIBRARY_DIR}/types.cc" + "${LIBRARY_DIR}/bloom_filter_reader.cc" + "${LIBRARY_DIR}/xxhasher.cc" "${GEN_LIBRARY_DIR}/parquet_constants.cpp" "${GEN_LIBRARY_DIR}/parquet_types.cpp" diff --git a/tests/queries/0_stateless/02735_parquet_encoder.reference b/tests/queries/0_stateless/02735_parquet_encoder.reference index a7ee82bc67f..143fde3093f 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.reference +++ b/tests/queries/0_stateless/02735_parquet_encoder.reference @@ -36,11 +36,11 @@ ipv6 Nullable(FixedString(16)) 1 1000000 1 3914219105369203805 4 1000000 1 -(1000000,0,NULL,'100','299') -(1000000,0,NULL,'0','-1294970296') -(1000000,0,NULL,'-2147483296','2147481000') +(1000000,NULL,NULL,'100','299') +(1000000,NULL,NULL,'0','-1294970296') +(1000000,NULL,NULL,'-2147483296','2147481000') (100000,900000,NULL,'100009','999999') -[(2,0,NULL,'','[]')] +[(2,NULL,NULL,'','[]')] 1 1 0 1 5090915589685802007 diff --git a/tests/queries/0_stateless/02884_parquet_new_encodings.reference b/tests/queries/0_stateless/02884_parquet_new_encodings.reference new file mode 100644 index 00000000000..1034f208e18 --- /dev/null +++ b/tests/queries/0_stateless/02884_parquet_new_encodings.reference @@ -0,0 +1 @@ +SWEEP SWETT 00459 \N ('20221206100111','+0100') ('20221206100111','+0100') ('20221206100111','+0100') 3 11 T \N diff --git a/tests/queries/0_stateless/02884_parquet_new_encodings.sh b/tests/queries/0_stateless/02884_parquet_new_encodings.sh new file mode 100755 index 00000000000..01114c2e4f4 --- /dev/null +++ b/tests/queries/0_stateless/02884_parquet_new_encodings.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_parquet/delta_lenght_byte_array_encoding.parquet')" + diff --git a/tests/queries/0_stateless/data_parquet/delta_lenght_byte_array_encoding.parquet b/tests/queries/0_stateless/data_parquet/delta_lenght_byte_array_encoding.parquet new file mode 100644 index 0000000000000000000000000000000000000000..cf785d97dc7032fdec8e51c8d3749c685ffc7a4d GIT binary patch literal 2795 zcmdUxO=ufO6vt;*@>-EMvg3G$9V!wiu?Q_qB+(RlDJeMM7EwfElH8V+GIAFrt+HAL zdNYR7QcOwO9C|V-h2o~A1w${zgnW>Da6<@%LUL*_t`8w8K9tZmv%BM2$Lr=0DAme% znD^dq{`)?5Re$$n1sm9c>*>W`zcy=F#|ZvRoW*DyAzho(vReMsnG+}Kk>4K+Pu7fd zznPiH@8PBUH}WZdbr1`N=nhm?bkW3a;}a{!4M_IHJ~&=#2P3MwB2V4Kp*L0a`*ah}wf8 z?19LjQ445@`{^N|kgP1Vwr!|LQBy#MwcR>vTgZRL$i?-cmRu_Ob8%t794{5K25Hn} z3X_>Ck{qr>&Jit=vvReh$>pYory~1`jvLW(F(Vx|GIIA$R_e~(X_A9#V=a+^-zdRs)Z$w$lKWFAe-k;5*%Zi}`kQ5ZG|+OG3kwVot-lRqXd5 z*Ktkk3(W6BZQ|q}s{OG4_i$esAC&73|OvO@a~8NuAV7x%Z0jKr*q85P4R>6XdTPt|BF1 ztA%9DStaI+QHWXElZ@H1!DRT7ro?|ciV68`TYPH&3C0weDs8Ha70y`5;{%HB!>JAW zTw@AogEUGjx*}|qL$BuX=@)`-Z)kb5iy4Lah%@I#63IH8-tp;5H&p5~PF)^JDov-R zO1+a+q+T8a=%;2<>TWZXF80ch!oAA5znDp}yKz(H&KnB$I>%nz8_CVmNo3I7M^aR% zMrx(ikORNXUTGkSsmoBIOIVA27+wl;1OVw94rtut%I{}@Zq3wr0I2z_StL) f-VCmHwCDKnE(qS3_2@dhJ6`aM77T>y@H6)xIfFfV literal 0 HcmV?d00001 From bf9567aac4506102232f9cd1200e8ebae11d2c8a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 25 Sep 2023 18:26:29 +0000 Subject: [PATCH 089/813] Fix typo in filename --- .../0_stateless/02884_parquet_new_encodings.sh | 2 +- ...uet => delta_length_byte_array_encoding.parquet} | Bin 2 files changed, 1 insertion(+), 1 deletion(-) rename tests/queries/0_stateless/data_parquet/{delta_lenght_byte_array_encoding.parquet => delta_length_byte_array_encoding.parquet} (100%) diff --git a/tests/queries/0_stateless/02884_parquet_new_encodings.sh b/tests/queries/0_stateless/02884_parquet_new_encodings.sh index 01114c2e4f4..496ed126e23 100755 --- a/tests/queries/0_stateless/02884_parquet_new_encodings.sh +++ b/tests/queries/0_stateless/02884_parquet_new_encodings.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_parquet/delta_lenght_byte_array_encoding.parquet')" +$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_parquet/delta_length_byte_array_encoding.parquet')" diff --git a/tests/queries/0_stateless/data_parquet/delta_lenght_byte_array_encoding.parquet b/tests/queries/0_stateless/data_parquet/delta_length_byte_array_encoding.parquet similarity index 100% rename from tests/queries/0_stateless/data_parquet/delta_lenght_byte_array_encoding.parquet rename to tests/queries/0_stateless/data_parquet/delta_length_byte_array_encoding.parquet From ad67b6c2ea8d9733d94f71b9ba9adcd2dfdf7f15 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 30 Oct 2023 19:33:06 +0800 Subject: [PATCH 090/813] allow tuple field pruning --- .../Impl/NativeORCBlockInputFormat.cpp | 150 +++++++++++++++++- 1 file changed, 143 insertions(+), 7 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index b346ef3d232..0af4428b5f0 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -679,6 +679,47 @@ buildORCSearchArgument(const KeyCondition & key_condition, const Block & header, } +static std::string toDotColumnPath(const std::vector & columns) +{ + if (columns.empty()) + return {}; + + std::ostringstream column_stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::copy(columns.begin(), columns.end(), std::ostream_iterator(column_stream, ".")); + std::string column_path = column_stream.str(); + return column_path.substr(0, column_path.length() - 1); +} + +static void buildORCTypeNameIdMap( + const orc::Type * orc_type, + std::vector & columns, + bool case_insensitive_column_matching, + std::map & id_type_map, + std::map & name_id_map) +{ + id_type_map[orc_type->getColumnId()] = orc_type; + if (orc::STRUCT == orc_type->getKind()) + { + for (size_t i = 0; i < orc_type->getSubtypeCount(); ++i) + { + const std::string & field_name = orc_type->getFieldName(i); + columns.push_back(field_name); + auto column_path = toDotColumnPath(columns); + if (case_insensitive_column_matching) + boost::to_lower(column_path); + name_id_map[column_path] = orc_type->getSubtype(i)->getColumnId(); + buildORCTypeNameIdMap(orc_type->getSubtype(i), columns, case_insensitive_column_matching, id_type_map, name_id_map); + columns.pop_back(); + } + } + else + { + // other non-primitive type + for (size_t j = 0; j < orc_type->getSubtypeCount(); ++j) + buildORCTypeNameIdMap(orc_type->getSubtype(j), columns, case_insensitive_column_matching, id_type_map, name_id_map); + } +} + static void getFileReaderAndSchema( ReadBuffer & in, std::unique_ptr & file_reader, @@ -706,6 +747,76 @@ static void getFileReaderAndSchema( } } +static void updateIncludeIndices( + DataTypePtr type, const orc::Type * orc_type, bool case_insensitive_column_matching, std::unordered_set & column_indices) +{ + /// Primitive types + if (orc_type->getSubtypeCount() == 0) + { + column_indices.insert(orc_type->getColumnId()); + return; + } + + auto non_nullable_type = removeNullable(type); + switch (orc_type->getKind()) + { + case orc::LIST: { + const auto * array_type = typeid_cast(non_nullable_type.get()); + if (array_type) + { + updateIncludeIndices( + array_type->getNestedType(), orc_type->getSubtype(0), case_insensitive_column_matching, column_indices); + } + return; + } + case orc::MAP: { + const auto * map_type = typeid_cast(non_nullable_type.get()); + if (map_type) + { + updateIncludeIndices(map_type->getKeyType(), orc_type->getSubtype(0), case_insensitive_column_matching, column_indices); + updateIncludeIndices(map_type->getValueType(), orc_type->getSubtype(1), case_insensitive_column_matching, column_indices); + } + return; + } + case orc::STRUCT: { + const auto * tuple_type = typeid_cast(non_nullable_type.get()); + if (tuple_type) + { + if (tuple_type->haveExplicitNames()) + { + const auto & names = tuple_type->getElementNames(); + for (size_t tuple_i = 0; tuple_i < names.size(); ++tuple_i) + { + const auto & name = names[tuple_i]; + for (size_t struct_i = 0; struct_i < orc_type->getSubtypeCount(); ++struct_i) + { + if (boost::equals(orc_type->getFieldName(struct_i), name) + || (case_insensitive_column_matching && boost::iequals(orc_type->getFieldName(struct_i), name))) + { + updateIncludeIndices( + tuple_type->getElement(tuple_i), + orc_type->getSubtype(struct_i), + case_insensitive_column_matching, + column_indices); + break; + } + } + } + } + else + { + for (size_t i = 0; i < tuple_type->getElements().size() && i < orc_type->getSubtypeCount(); ++i) + updateIncludeIndices( + tuple_type->getElement(i), orc_type->getSubtype(i), case_insensitive_column_matching, column_indices); + } + } + return; + } + default: + return; + } +} + NativeORCBlockInputFormat::NativeORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) : IInputFormat(std::move(header_), &in_), format_settings(format_settings_), skip_stripes(format_settings.orc.skip_stripes) { @@ -727,15 +838,39 @@ void NativeORCBlockInputFormat::prepareFileReader() format_settings.null_as_default, format_settings.orc.case_insensitive_column_matching); - const bool ignore_case = format_settings.orc.case_insensitive_column_matching; - std::unordered_set nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case); - for (size_t i = 0; i < schema.columns(); ++i) + const bool ignore_case = format_settings.orc.case_insensitive_column_matching; + std::vector columns; + std::map id_type_map; + std::map name_id_map; + buildORCTypeNameIdMap(&file_reader->getType(), columns, ignore_case, id_type_map, name_id_map); + + // std::cout << "subtypes:" << file_reader->getType().getSubtypeCount() << std::endl; + // std::cout << "id type map" << std::endl; + // for (const auto & [k, v]: id_type_map) + // std::cout << "id:" << k << ", type:" << v->toString() << std::endl; + // std::cout << "name id map" << std::endl; + // for (const auto & [k, v]: name_id_map) + // std::cout << "name:" << k << ", id:" << v << std::endl; + + const auto & header = getPort().getHeader(); + std::unordered_set column_indices; + for (const auto & column : header) { - const auto & name = schema.getByPosition(i).name; - if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name)) - include_indices.push_back(static_cast(i)); + auto name = column.name; + if (ignore_case) + boost::to_lower(name); + + if (name_id_map.contains(name)) + { + auto id = name_id_map[name]; + if (id_type_map.contains(id)) + { + updateIncludeIndices(column.type, id_type_map[id], ignore_case, column_indices); + } + } } + include_indices.assign(column_indices.begin(), column_indices.end()); if (format_settings.orc.filter_push_down && key_condition && !sarg) { @@ -816,6 +951,7 @@ Chunk NativeORCBlockInputFormat::generate() Chunk res; size_t num_rows = batch->numElements; const auto & schema = stripe_reader->getSelectedType(); + // std::cout << "output schema:" << schema.toString() << std::endl; orc_column_to_ch_column->orcTableToCHChunk(res, &schema, batch.get(), num_rows, &block_missing_values); approx_bytes_read_for_chunk = num_rows * current_stripe_info->getLength() / current_stripe_info->getNumberOfRows(); @@ -1376,8 +1512,8 @@ static ColumnWithTypeAndName readColumnFromORCColumn( Columns tuple_elements; DataTypes tuple_types; std::vector tuple_names; - const auto * tuple_type_hint = type_hint ? typeid_cast(type_hint.get()) : nullptr; + const auto * tuple_type_hint = type_hint ? typeid_cast(type_hint.get()) : nullptr; const auto * orc_struct_column = dynamic_cast(orc_column); for (size_t i = 0; i < orc_type->getSubtypeCount(); ++i) { From 8954b806b49b00f3c2c5a53b04ca80c31eb2e68c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 30 Oct 2023 20:08:09 +0800 Subject: [PATCH 091/813] add uts --- .../02906_orc_tuple_field_prune.reference | 108 ++++++++++++++++++ .../02906_orc_tuple_field_prune.sql | 38 ++++++ 2 files changed, 146 insertions(+) create mode 100644 tests/queries/0_stateless/02906_orc_tuple_field_prune.reference create mode 100644 tests/queries/0_stateless/02906_orc_tuple_field_prune.sql diff --git a/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference b/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference new file mode 100644 index 00000000000..dfdd38f5e8e --- /dev/null +++ b/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference @@ -0,0 +1,108 @@ +int64_column Nullable(Int64) +string_column Nullable(String) +float64_column Nullable(Float64) +tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)) +array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))) +map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))) +-- { echoOn } +-- Test primitive types +select int64_column, string_column, float64_column from file('02906.orc') where int64_column % 15 = 0; +0 0 0 +15 15 15 +30 30 30 +45 45 45 +60 60 60 +75 75 75 +90 90 90 +-- Test tuple type with names +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))') where int64_column % 15 = 0; +(NULL,NULL,NULL) +('15',15,15) +(NULL,NULL,NULL) +('45',45,45) +(NULL,NULL,NULL) +('75',75,75) +(NULL,NULL,NULL) +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(c Nullable(Int64))') where int64_column % 15 = 0; +(NULL) +(15) +(NULL) +(45) +(NULL) +(75) +(NULL) +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(c Nullable(Int64), d Nullable(String))') where int64_column % 15 = 0; +(NULL,NULL) +(15,NULL) +(NULL,NULL) +(45,NULL) +(NULL,NULL) +(75,NULL) +(NULL,NULL) +-- Test tuple type without names +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(Nullable(String), Nullable(Float64), Nullable(Int64))') where int64_column % 15 = 0; +(NULL,NULL,NULL) +('15',15,15) +(NULL,NULL,NULL) +('45',45,45) +(NULL,NULL,NULL) +('75',75,75) +(NULL,NULL,NULL) +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(Nullable(String), Nullable(Float64))') where int64_column % 15 = 0; +(NULL,NULL) +('15',15) +(NULL,NULL) +('45',45) +(NULL,NULL) +('75',75) +(NULL,NULL) +-- Test tuple nested in array +select array_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') where int64_column % 15 = 0; +[(NULL,NULL,NULL)] +[('15',15,15)] +[(NULL,NULL,NULL)] +[('45',45,45)] +[(NULL,NULL,NULL)] +[('75',75,75)] +[(NULL,NULL,NULL)] +select array_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(b Nullable(Float64), c Nullable(Int64)))') where int64_column % 15 = 0; +[(NULL,NULL)] +[(15,15)] +[(NULL,NULL)] +[(45,45)] +[(NULL,NULL)] +[(75,75)] +[(NULL,NULL)] +select array_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(b Nullable(Float64), c Nullable(Int64), d Nullable(String)))') where int64_column % 15 = 0; +[(NULL,NULL,NULL)] +[(15,15,NULL)] +[(NULL,NULL,NULL)] +[(45,45,NULL)] +[(NULL,NULL,NULL)] +[(75,75,NULL)] +[(NULL,NULL,NULL)] +-- Test tuple nested in map +select map_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') where int64_column % 15 = 0; +{'0':(NULL,NULL,NULL)} +{'15':('15',15,15)} +{'30':(NULL,NULL,NULL)} +{'45':('45',45,45)} +{'60':(NULL,NULL,NULL)} +{'75':('75',75,75)} +{'90':(NULL,NULL,NULL)} +select map_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(b Nullable(Float64), c Nullable(Int64)))') where int64_column % 15 = 0; +{'0':(NULL,NULL)} +{'15':(15,15)} +{'30':(NULL,NULL)} +{'45':(45,45)} +{'60':(NULL,NULL)} +{'75':(75,75)} +{'90':(NULL,NULL)} +select map_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(b Nullable(Float64), c Nullable(Int64), d Nullable(String)))') where int64_column % 15 = 0; +{'0':(NULL,NULL,NULL)} +{'15':(15,15,NULL)} +{'30':(NULL,NULL,NULL)} +{'45':(45,45,NULL)} +{'60':(NULL,NULL,NULL)} +{'75':(75,75,NULL)} +{'90':(NULL,NULL,NULL)} diff --git a/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql b/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql new file mode 100644 index 00000000000..a7f2c31d3e1 --- /dev/null +++ b/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql @@ -0,0 +1,38 @@ +set engine_file_truncate_on_insert = 1; +set flatten_nested = 0; + +insert into function file('02906.orc') +select + number::Int64 as int64_column, + number::String as string_column, + number::Float64 as float64_column, + cast(if(number % 10 = 0, tuple(null, null, null), tuple(number::String, number::Float64, number::Int64)) as Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))) as tuple_column, + cast(if(number % 10 = 0, array(tuple(null, null, null)), array(tuple(number::String, number::Float64, number::Int64))) as Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))) as array_tuple_column, + cast(if(number % 10 = 0, map(number::String, tuple(null, null, null)), map(number::String, tuple(number::String, number::Float64, number::Int64))) as Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))) as map_tuple_column + from numbers(100); + +desc file('02906.orc'); + +-- { echoOn } +-- Test primitive types +select int64_column, string_column, float64_column from file('02906.orc') where int64_column % 15 = 0; + +-- Test tuple type with names +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))') where int64_column % 15 = 0; +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(c Nullable(Int64))') where int64_column % 15 = 0; +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(c Nullable(Int64), d Nullable(String))') where int64_column % 15 = 0; + +-- Test tuple type without names +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(Nullable(String), Nullable(Float64), Nullable(Int64))') where int64_column % 15 = 0; +select tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(Nullable(String), Nullable(Float64))') where int64_column % 15 = 0; + +-- Test tuple nested in array +select array_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') where int64_column % 15 = 0; +select array_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(b Nullable(Float64), c Nullable(Int64)))') where int64_column % 15 = 0; +select array_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(b Nullable(Float64), c Nullable(Int64), d Nullable(String)))') where int64_column % 15 = 0; + +-- Test tuple nested in map +select map_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') where int64_column % 15 = 0; +select map_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(b Nullable(Float64), c Nullable(Int64)))') where int64_column % 15 = 0; +select map_tuple_column from file('02906.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(b Nullable(Float64), c Nullable(Int64), d Nullable(String)))') where int64_column % 15 = 0; +-- { echoOff } From 38f24c04558e4528ca1d9cf92ba170c400177569 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 30 Oct 2023 20:29:43 +0800 Subject: [PATCH 092/813] add performance tests --- tests/performance/orc_tuple_field_prune.xml | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 tests/performance/orc_tuple_field_prune.xml diff --git a/tests/performance/orc_tuple_field_prune.xml b/tests/performance/orc_tuple_field_prune.xml new file mode 100644 index 00000000000..2bcd15c8635 --- /dev/null +++ b/tests/performance/orc_tuple_field_prune.xml @@ -0,0 +1,17 @@ + + + 1 + 10000 + 0 + + + + insert into function file('test_orc_tfp.orc') select * from generateRandom('int64_column Nullable(Int64), tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 + + + DROP TABLE IF EXISTS test_orc_tfp + + select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(c Nullable(Int64))') format Null + select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(c Nullable(Int64)))') format Null + select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(c Nullable(Int64)))') format Null + \ No newline at end of file From aef9ce0cf06f4541ca624e8b6711bf4575a20f40 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 30 Oct 2023 20:30:02 +0800 Subject: [PATCH 093/813] update orc version --- contrib/orc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/orc b/contrib/orc index f31c271110a..2c31e314e4e 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit f31c271110a2f0dac908a152f11708193ae209ee +Subproject commit 2c31e314e4e36dcb1c58ca1cd7454fc4685af997 From 423df126254980d8a05e15659a8b45a479a8a5be Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 30 Oct 2023 20:38:56 +0800 Subject: [PATCH 094/813] update orc version --- contrib/orc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/orc b/contrib/orc index 2c31e314e4e..5f8db0fb0a4 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit 2c31e314e4e36dcb1c58ca1cd7454fc4685af997 +Subproject commit 5f8db0fb0a47fbc4902bf9d7f712e65309f13d2d From 1e5703a77b50a1fb4f54afcaf0c322cdf989c3d7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 30 Oct 2023 20:41:45 +0800 Subject: [PATCH 095/813] upgrade orc version --- contrib/orc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/orc b/contrib/orc index 5f8db0fb0a4..5046972fbab 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit 5f8db0fb0a47fbc4902bf9d7f712e65309f13d2d +Subproject commit 5046972fbabfe3cdf77a8768228793c7c0a61085 From f53fdbeeadf7a2be2fa962e0ee0be91628348b93 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Oct 2023 19:11:03 +0000 Subject: [PATCH 096/813] Fix vuild for s390 --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index 9d9c464ce68..8cdbf43f78a 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 9d9c464ce6883f52aaca9f913eec4cd50006c767 +Subproject commit 8cdbf43f78ad02615aef29dc7f9af0dea22a03e4 From e5db57204d42b578296a1d2f022f38641d702be9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 31 Oct 2023 11:57:47 +0800 Subject: [PATCH 097/813] fix bugs --- contrib/orc | 2 +- .../Impl/NativeORCBlockInputFormat.cpp | 26 +++++++++---------- .../02906_orc_tuple_field_prune.sql | 2 +- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/contrib/orc b/contrib/orc index 5046972fbab..e24f2c2a3ca 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit 5046972fbabfe3cdf77a8768228793c7c0a61085 +Subproject commit e24f2c2a3ca0769c96704ab20ad6f512a83ea2ad diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 0af4428b5f0..88b3fbeee2b 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -52,19 +52,19 @@ ORCInputStream::ORCInputStream(SeekableReadBuffer & in_, size_t file_size_) : in { } -uint64_t ORCInputStream::getLength() const +UInt64 ORCInputStream::getLength() const { return file_size; } -uint64_t ORCInputStream::getNaturalReadSize() const +UInt64 ORCInputStream::getNaturalReadSize() const { return 128 * 1024; } -void ORCInputStream::read(void * buf, uint64_t length, uint64_t offset) +void ORCInputStream::read(void * buf, UInt64 length, UInt64 offset) { - if (offset != static_cast(in.getPosition())) + if (offset != static_cast(in.getPosition())) in.seek(offset, SEEK_SET); in.readStrict(reinterpret_cast(buf), length); @@ -102,7 +102,7 @@ std::unique_ptr asORCInputStreamLoadIntoMemory(ReadBuffer & in static const orc::Type * getORCTypeByName(const orc::Type & schema, const String & name, bool case_insensitive_column_matching) { - for (uint64_t i = 0; i != schema.getSubtypeCount(); ++i) + for (UInt64 i = 0; i != schema.getSubtypeCount(); ++i) if (boost::equals(schema.getFieldName(i), name) || (case_insensitive_column_matching && boost::iequals(schema.getFieldName(i), name))) return schema.getSubtype(i); @@ -694,8 +694,8 @@ static void buildORCTypeNameIdMap( const orc::Type * orc_type, std::vector & columns, bool case_insensitive_column_matching, - std::map & id_type_map, - std::map & name_id_map) + std::map & id_type_map, + std::map & name_id_map) { id_type_map[orc_type->getColumnId()] = orc_type; if (orc::STRUCT == orc_type->getKind()) @@ -841,8 +841,8 @@ void NativeORCBlockInputFormat::prepareFileReader() const bool ignore_case = format_settings.orc.case_insensitive_column_matching; std::vector columns; - std::map id_type_map; - std::map name_id_map; + std::map id_type_map; + std::map name_id_map; buildORCTypeNameIdMap(&file_reader->getType(), columns, ignore_case, id_type_map, name_id_map); // std::cout << "subtypes:" << file_reader->getType().getSubtypeCount() << std::endl; @@ -854,7 +854,7 @@ void NativeORCBlockInputFormat::prepareFileReader() // std::cout << "name:" << k << ", id:" << v << std::endl; const auto & header = getPort().getHeader(); - std::unordered_set column_indices; + std::unordered_set include_typeids; for (const auto & column : header) { auto name = column.name; @@ -866,11 +866,11 @@ void NativeORCBlockInputFormat::prepareFileReader() auto id = name_id_map[name]; if (id_type_map.contains(id)) { - updateIncludeIndices(column.type, id_type_map[id], ignore_case, column_indices); + updateIncludeIndices(column.type, id_type_map[id], ignore_case, include_typeids); } } } - include_indices.assign(column_indices.begin(), column_indices.end()); + include_indices.assign(include_typeids.begin(), include_typeids.end()); if (format_settings.orc.filter_push_down && key_condition && !sarg) { @@ -895,7 +895,7 @@ bool NativeORCBlockInputFormat::prepareStripeReader() throw Exception(ErrorCodes::INCORRECT_DATA, "ORC stripe {} has no rows", current_stripe); orc::RowReaderOptions row_reader_options; - row_reader_options.include(include_indices); + row_reader_options.includeTypes(include_indices); row_reader_options.range(current_stripe_info->getOffset(), current_stripe_info->getLength()); if (format_settings.orc.filter_push_down && sarg) { diff --git a/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql b/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql index a7f2c31d3e1..834caa1da53 100644 --- a/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql +++ b/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql @@ -1,7 +1,7 @@ set engine_file_truncate_on_insert = 1; set flatten_nested = 0; -insert into function file('02906.orc') +insert into function file('02906.orc', 'ORC') select number::Int64 as int64_column, number::String as string_column, From c97b2c5be74a73305a2c0dbc905dc59ac77c0fd3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 31 Oct 2023 12:00:45 +0800 Subject: [PATCH 098/813] fix code style --- .../Impl/NativeORCBlockInputFormat.cpp | 24 +++++++++---------- tests/performance/orc_tuple_field_prune.xml | 2 +- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 88b3fbeee2b..fd0f4ee0ca0 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -747,13 +747,13 @@ static void getFileReaderAndSchema( } } -static void updateIncludeIndices( - DataTypePtr type, const orc::Type * orc_type, bool case_insensitive_column_matching, std::unordered_set & column_indices) +static void updateIncludeTypeIds( + DataTypePtr type, const orc::Type * orc_type, bool case_insensitive_column_matching, std::unordered_set & include_typeids) { /// Primitive types if (orc_type->getSubtypeCount() == 0) { - column_indices.insert(orc_type->getColumnId()); + include_typeids.insert(orc_type->getColumnId()); return; } @@ -764,8 +764,8 @@ static void updateIncludeIndices( const auto * array_type = typeid_cast(non_nullable_type.get()); if (array_type) { - updateIncludeIndices( - array_type->getNestedType(), orc_type->getSubtype(0), case_insensitive_column_matching, column_indices); + updateIncludeTypeIds( + array_type->getNestedType(), orc_type->getSubtype(0), case_insensitive_column_matching, include_typeids); } return; } @@ -773,8 +773,8 @@ static void updateIncludeIndices( const auto * map_type = typeid_cast(non_nullable_type.get()); if (map_type) { - updateIncludeIndices(map_type->getKeyType(), orc_type->getSubtype(0), case_insensitive_column_matching, column_indices); - updateIncludeIndices(map_type->getValueType(), orc_type->getSubtype(1), case_insensitive_column_matching, column_indices); + updateIncludeTypeIds(map_type->getKeyType(), orc_type->getSubtype(0), case_insensitive_column_matching, include_typeids); + updateIncludeTypeIds(map_type->getValueType(), orc_type->getSubtype(1), case_insensitive_column_matching, include_typeids); } return; } @@ -793,11 +793,11 @@ static void updateIncludeIndices( if (boost::equals(orc_type->getFieldName(struct_i), name) || (case_insensitive_column_matching && boost::iequals(orc_type->getFieldName(struct_i), name))) { - updateIncludeIndices( + updateIncludeTypeIds( tuple_type->getElement(tuple_i), orc_type->getSubtype(struct_i), case_insensitive_column_matching, - column_indices); + include_typeids); break; } } @@ -806,8 +806,8 @@ static void updateIncludeIndices( else { for (size_t i = 0; i < tuple_type->getElements().size() && i < orc_type->getSubtypeCount(); ++i) - updateIncludeIndices( - tuple_type->getElement(i), orc_type->getSubtype(i), case_insensitive_column_matching, column_indices); + updateIncludeTypeIds( + tuple_type->getElement(i), orc_type->getSubtype(i), case_insensitive_column_matching, include_typeids); } } return; @@ -866,7 +866,7 @@ void NativeORCBlockInputFormat::prepareFileReader() auto id = name_id_map[name]; if (id_type_map.contains(id)) { - updateIncludeIndices(column.type, id_type_map[id], ignore_case, include_typeids); + updateIncludeTypeIds(column.type, id_type_map[id], ignore_case, include_typeids); } } } diff --git a/tests/performance/orc_tuple_field_prune.xml b/tests/performance/orc_tuple_field_prune.xml index 2bcd15c8635..4e338733329 100644 --- a/tests/performance/orc_tuple_field_prune.xml +++ b/tests/performance/orc_tuple_field_prune.xml @@ -6,7 +6,7 @@ - insert into function file('test_orc_tfp.orc') select * from generateRandom('int64_column Nullable(Int64), tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 + insert into function file('test_orc_tfp.orc', 'ORC') select * from generateRandom('int64_column Nullable(Int64), tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 DROP TABLE IF EXISTS test_orc_tfp From 5e21d2459a00b38601bb2c5709795a9bda72fa65 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 31 Oct 2023 14:19:51 +0800 Subject: [PATCH 099/813] fix failed fast test --- tests/queries/0_stateless/02906_orc_tuple_field_prune.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql b/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql index 834caa1da53..5428abc40de 100644 --- a/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql +++ b/tests/queries/0_stateless/02906_orc_tuple_field_prune.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest, no-parallel + set engine_file_truncate_on_insert = 1; set flatten_nested = 0; From ce36a6475dcff34f9d4f0510c6608bdf261478e8 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 31 Oct 2023 10:12:31 +0100 Subject: [PATCH 100/813] Fixed cland tidy build --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 27fbce5311c..9655e8b2855 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -732,7 +732,7 @@ static std::tuple executeQueryImpl( is_create_parameterized_view = create_query->isParameterizedView(); else if (const auto * explain_query = ast->as()) { - assert(explain_query->children.size() >= 1); + assert(!explain_query->children.empty()); if (const auto * create_of_explain_query = explain_query->children[0]->as()) is_create_parameterized_view = create_of_explain_query->isParameterizedView(); } From 7c5a7fc03aa5a6f2675c9769976413d03f2b9f01 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 31 Oct 2023 17:17:21 +0800 Subject: [PATCH 101/813] update orc version --- contrib/orc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/orc b/contrib/orc index e24f2c2a3ca..f31c271110a 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit e24f2c2a3ca0769c96704ab20ad6f512a83ea2ad +Subproject commit f31c271110a2f0dac908a152f11708193ae209ee From 11b00e92fdf34621a26152f607ffa55f2eb09a7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 31 Oct 2023 12:27:21 +0100 Subject: [PATCH 102/813] Add more details to "Data after merge is not byte-identical to data on another replicas" It should be a very rare error, but in case of error you need as much details as there are: - print mismatched info (hash, size) - print all files with size and checksum (this may help for instance when new file is added, or when extesion changed -- compression marks) P.S. maybe detach_not_byte_identical_parts should be enabled by default? Thoughts? Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + .../MergeTree/MergeFromLogEntryTask.cpp | 13 ++++++- .../MergeTree/MergeTreeDataPartChecksum.cpp | 38 +++++++++++++++---- 3 files changed, 43 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 30c9b19fcbc..5201eefccf1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -364,6 +364,7 @@ public: void setBytesOnDisk(UInt64 bytes_on_disk_) { bytes_on_disk = bytes_on_disk_; } size_t getFileSizeOrZero(const String & file_name) const; + auto getFilesChecksums() const { return checksums.files; } /// Moves a part to detached/ directory and adds prefix to its name void renameToDetached(const String & prefix); diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 883cfee89c8..5d64950ab58 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -357,6 +357,13 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite ProfileEvents::increment(ProfileEvents::DataAfterMergeDiffersFromReplica); + Strings files_with_size; + for (const auto & file : part->getFilesChecksums()) + { + files_with_size.push_back(fmt::format("{}: {} ({})", + file.first, file.second.file_size, getHexUIntLowercase(file.second.file_hash))); + } + LOG_ERROR(log, "{}. Data after merge is not byte-identical to data on another replicas. There could be several reasons:" " 1. Using newer version of compression library after server update." @@ -368,8 +375,10 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " 7. Manual modification of source data after server startup." " 8. Manual modification of checksums stored in ZooKeeper." " 9. Part format related settings like 'enable_mixed_granularity_parts' are different on different replicas." - " We will download merged part from replica to force byte-identical result.", - getCurrentExceptionMessage(false)); + " We will download merged part from replica to force byte-identical result." + " List of files in local parts:\n{}", + getCurrentExceptionMessage(false), + fmt::join(files_with_size, "\n")); write_part_log(ExecutionStatus::fromCurrentException("", true)); diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index ed2202fcb19..4bda5ce469d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -34,15 +34,27 @@ void MergeTreeDataPartChecksum::checkEqual(const MergeTreeDataPartChecksum & rhs if (!rhs.is_compressed) throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "No uncompressed checksum for file {}", name); if (rhs.uncompressed_size != uncompressed_size) - throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected uncompressed size of file {} in data part", name); + { + throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected uncompressed size of file {} in data part ({} vs {})", + name, uncompressed_size, rhs.uncompressed_size); + } if (rhs.uncompressed_hash != uncompressed_hash) - throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Checksum mismatch for uncompressed file {} in data part", name); + { + throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Checksum mismatch for uncompressed file {} in data part ({} vs {})", + name, getHexUIntLowercase(uncompressed_hash), getHexUIntLowercase(rhs.uncompressed_hash)); + } return; } if (rhs.file_size != file_size) - throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected size of file {} in data part", name); + { + throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected size of file {} in data part ({} vs {})", + name, file_size, rhs.file_size); + } if (rhs.file_hash != file_hash) - throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Checksum mismatch for file {} in data part", name); + { + throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Checksum mismatch for file {} in data part ({} vs {})", + name, getHexUIntLowercase(file_hash), getHexUIntLowercase(rhs.file_hash)); + } } void MergeTreeDataPartChecksum::checkSize(const IDataPartStorage & storage, const String & name) const @@ -446,17 +458,29 @@ void MinimalisticDataPartChecksums::checkEqualImpl(const MinimalisticDataPartChe Strings errors; if (hash_of_uncompressed_files != rhs.hash_of_uncompressed_files) - errors.emplace_back("hash of uncompressed files doesn't match"); + { + errors.emplace_back(fmt::format("hash of uncompressed files doesn't match ({} vs {})", + getHexUIntLowercase(hash_of_uncompressed_files), + getHexUIntLowercase(rhs.hash_of_uncompressed_files))); + } if (check_uncompressed_hash_in_compressed_files) { if (uncompressed_hash_of_compressed_files != rhs.uncompressed_hash_of_compressed_files) - errors.emplace_back("uncompressed hash of compressed files doesn't match"); + { + errors.emplace_back(fmt::format("uncompressed hash of compressed files doesn't match ({} vs {})", + getHexUIntLowercase(uncompressed_hash_of_compressed_files), + getHexUIntLowercase(rhs.uncompressed_hash_of_compressed_files))); + } } else { if (hash_of_all_files != rhs.hash_of_all_files) - errors.emplace_back("total hash of all files doesn't match"); + { + errors.emplace_back(fmt::format("total hash of all files doesn't match ({} vs {})", + getHexUIntLowercase(hash_of_all_files), + getHexUIntLowercase(rhs.hash_of_all_files))); + } } if (!errors.empty()) From 8a1ab02b96e234a9cdc1018b692381e8e4abf9c2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 31 Oct 2023 12:59:16 +0100 Subject: [PATCH 103/813] Update s3_cache.xml --- tests/config/users.d/s3_cache.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/config/users.d/s3_cache.xml b/tests/config/users.d/s3_cache.xml index 4740f37a90c..69b24ecbbc4 100644 --- a/tests/config/users.d/s3_cache.xml +++ b/tests/config/users.d/s3_cache.xml @@ -3,7 +3,6 @@ 1 1 - 10 From 52a3d37ebe6008fc4301d369f6309587b32e648b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 27 Oct 2023 16:59:14 +0200 Subject: [PATCH 104/813] Try reducing number of different images --- .../integration/test_backward_compatibility/test.py | 2 +- .../test_aggregate_fixed_key.py | 2 +- .../test_convert_ordinary.py | 2 +- .../test_cte_distributed.py | 2 +- .../test_insert_profile_events.py | 2 +- .../test_memory_bound_aggregation.py | 4 ++-- .../test_normalized_count_comparison.py | 2 +- .../test_select_aggregate_alias_column.py | 2 +- .../test_vertical_merges_from_compact_parts.py | 2 +- .../test_default_compression_codec/test.py | 2 +- tests/integration/test_disk_over_web_server/test.py | 2 +- .../test_distributed_backward_compatability/test.py | 2 +- .../test.py | 2 +- .../test_distributed_inter_server_secret/test.py | 2 +- .../test_groupBitmapAnd_on_distributed/test.py | 2 +- tests/integration/test_old_versions/test.py | 2 +- tests/integration/test_polymorphic_parts/test.py | 2 +- .../test_replicated_merge_tree_compatibility/test.py | 4 ++-- tests/integration/test_replicating_constants/test.py | 2 +- tests/integration/test_ttl_replicated/test.py | 6 +++--- tests/integration/test_version_update/test.py | 12 ++++++------ .../test_version_update_after_mutation/test.py | 6 +++--- 22 files changed, 33 insertions(+), 33 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index 6f21b184a95..847483f2b9b 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -7,7 +7,7 @@ node1 = cluster.add_instance( "node1", with_zookeeper=True, image="yandex/clickhouse-server", - tag="19.17.8.54", + tag="19.16.9.37", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index cf258987cbf..94bc1d3bfc9 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -7,7 +7,7 @@ node1 = cluster.add_instance( "node1", with_zookeeper=True, image="yandex/clickhouse-server", - tag="21.3", + tag="20.8.11.17", with_installed_binary=True, allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index 36facdd59b1..034a68e0f30 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -5,7 +5,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", image="yandex/clickhouse-server", - tag="19.17.8.54", + tag="19.16.9.37", stay_alive=True, with_zookeeper=True, with_installed_binary=True, diff --git a/tests/integration/test_backward_compatibility/test_cte_distributed.py b/tests/integration/test_backward_compatibility/test_cte_distributed.py index c68468aad75..d47ae3aa255 100644 --- a/tests/integration/test_backward_compatibility/test_cte_distributed.py +++ b/tests/integration/test_backward_compatibility/test_cte_distributed.py @@ -8,7 +8,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=False, image="yandex/clickhouse-server", - tag="21.7.3.14", + tag="21.6", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_backward_compatibility/test_insert_profile_events.py b/tests/integration/test_backward_compatibility/test_insert_profile_events.py index 8564c6b5952..d38bece7855 100644 --- a/tests/integration/test_backward_compatibility/test_insert_profile_events.py +++ b/tests/integration/test_backward_compatibility/test_insert_profile_events.py @@ -11,7 +11,7 @@ upstream_node = cluster.add_instance("upstream_node", allow_analyzer=False) old_node = cluster.add_instance( "old_node", image="clickhouse/clickhouse-server", - tag="22.5.1.2079", + tag="22.6", with_installed_binary=True, allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py index 96b41c81384..5261a279a4f 100644 --- a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py @@ -7,7 +7,7 @@ node1 = cluster.add_instance( "node1", with_zookeeper=False, image="yandex/clickhouse-server", - tag="21.1", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, allow_analyzer=False, @@ -16,7 +16,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=False, image="yandex/clickhouse-server", - tag="21.1", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py index 3cd708d5029..cf7a25e8dc1 100644 --- a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py +++ b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py @@ -8,7 +8,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=False, image="yandex/clickhouse-server", - tag="21.7.2.7", + tag="21.6", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py index 7e10b6ab430..ec1d7fedac5 100644 --- a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py +++ b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py @@ -8,7 +8,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=False, image="yandex/clickhouse-server", - tag="21.7.2.7", + tag="21.6", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 9c9d1a4d312..e0a9b5ebad6 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -7,7 +7,7 @@ cluster = ClickHouseCluster(__file__) node_old = cluster.add_instance( "node1", image="clickhouse/clickhouse-server", - tag="22.8", + tag="22.6", stay_alive=True, with_installed_binary=True, with_zookeeper=True, diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py index 82d5eb04d2a..db116ff42f3 100644 --- a/tests/integration/test_default_compression_codec/test.py +++ b/tests/integration/test_default_compression_codec/test.py @@ -29,7 +29,7 @@ node3 = cluster.add_instance( "node3", main_configs=["configs/default_compression.xml", "configs/wide_parts_only.xml"], image="yandex/clickhouse-server", - tag="20.3.16", + tag="19.16.9.37", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 7695d235425..a71fdeff302 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -38,7 +38,7 @@ def cluster(): stay_alive=True, with_installed_binary=True, image="clickhouse/clickhouse-server", - tag="22.8.14.53", + tag="22.6", allow_analyzer=False, ) diff --git a/tests/integration/test_distributed_backward_compatability/test.py b/tests/integration/test_distributed_backward_compatability/test.py index c48a7ad1fa1..319a4c08e60 100644 --- a/tests/integration/test_distributed_backward_compatability/test.py +++ b/tests/integration/test_distributed_backward_compatability/test.py @@ -8,7 +8,7 @@ node_old = cluster.add_instance( "node1", main_configs=["configs/remote_servers.xml"], image="yandex/clickhouse-server", - tag="20.8.9.6", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_distributed_insert_backward_compatibility/test.py b/tests/integration/test_distributed_insert_backward_compatibility/test.py index 1e566d5e2da..7cfea61ffff 100644 --- a/tests/integration/test_distributed_insert_backward_compatibility/test.py +++ b/tests/integration/test_distributed_insert_backward_compatibility/test.py @@ -11,7 +11,7 @@ node_dist = cluster.add_instance( "node2", main_configs=["configs/remote_servers.xml"], image="yandex/clickhouse-server", - tag="21.11.9.1", + tag="21.6", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 1aeaddcf3c5..62beeee80e1 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -31,7 +31,7 @@ backward = make_instance( "configs/remote_servers_backward.xml", image="clickhouse/clickhouse-server", # version without DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 - tag="23.2.3", + tag="22.6", with_installed_binary=True, allow_analyzer=False, ) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index 8cf7e0fb2c1..5d3dda8ecf2 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -26,7 +26,7 @@ node4 = cluster.add_instance( "node4", main_configs=["configs/clusters.xml"], image="yandex/clickhouse-server", - tag="21.5", + tag="21.6", with_zookeeper=True, allow_analyzer=False, ) diff --git a/tests/integration/test_old_versions/test.py b/tests/integration/test_old_versions/test.py index aff07c53114..b59bfcc4f6b 100644 --- a/tests/integration/test_old_versions/test.py +++ b/tests/integration/test_old_versions/test.py @@ -55,7 +55,7 @@ node19_13 = cluster.add_instance( node19_16 = cluster.add_instance( "node19_16", image="yandex/clickhouse-server", - tag="19.16.2.2", + tag="19.16.9.37", with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], allow_analyzer=False, diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index debb509de90..ba9b5ec6cac 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -360,7 +360,7 @@ node7 = cluster.add_instance( user_configs=["configs_old/users.d/not_optimize_count.xml"], with_zookeeper=True, image="yandex/clickhouse-server", - tag="19.17.8.54", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_replicated_merge_tree_compatibility/test.py b/tests/integration/test_replicated_merge_tree_compatibility/test.py index c30a0d86c98..32a44aa65b9 100644 --- a/tests/integration/test_replicated_merge_tree_compatibility/test.py +++ b/tests/integration/test_replicated_merge_tree_compatibility/test.py @@ -6,7 +6,7 @@ node1 = cluster.add_instance( "node1", with_zookeeper=True, image="yandex/clickhouse-server", - tag="20.12.4.5", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, allow_analyzer=False, @@ -15,7 +15,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=True, image="yandex/clickhouse-server", - tag="20.12.4.5", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, allow_analyzer=False, diff --git a/tests/integration/test_replicating_constants/test.py b/tests/integration/test_replicating_constants/test.py index 00781e473c7..9669e890cd3 100644 --- a/tests/integration/test_replicating_constants/test.py +++ b/tests/integration/test_replicating_constants/test.py @@ -9,7 +9,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=True, image="yandex/clickhouse-server", - tag="19.1.14", + tag="19.16.9.37", with_installed_binary=True, allow_analyzer=False, ) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 117ebe37dd2..29ce2b3dc8d 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -17,7 +17,7 @@ node4 = cluster.add_instance( "node4", with_zookeeper=True, image="yandex/clickhouse-server", - tag="20.12.4.5", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, main_configs=[ @@ -30,7 +30,7 @@ node5 = cluster.add_instance( "node5", with_zookeeper=True, image="yandex/clickhouse-server", - tag="20.12.4.5", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, main_configs=[ @@ -42,7 +42,7 @@ node6 = cluster.add_instance( "node6", with_zookeeper=True, image="yandex/clickhouse-server", - tag="20.12.4.5", + tag="20.8.11.17", stay_alive=True, with_installed_binary=True, main_configs=[ diff --git a/tests/integration/test_version_update/test.py b/tests/integration/test_version_update/test.py index b8fa3e7ebb4..a752960bc76 100644 --- a/tests/integration/test_version_update/test.py +++ b/tests/integration/test_version_update/test.py @@ -12,18 +12,18 @@ node2 = cluster.add_instance( "node2", with_zookeeper=True, image="yandex/clickhouse-server", - tag="21.2", + tag="20.8.11.17", with_installed_binary=True, stay_alive=True, allow_analyzer=False, ) -# Use differents nodes because if there is node.restart_from_latest_version(), then in later tests +# Use different nodes because if there is node.restart_from_latest_version(), then in later tests # it will be with latest version, but shouldn't, order of tests in CI is shuffled. node3 = cluster.add_instance( "node3", image="yandex/clickhouse-server", - tag="21.5", + tag="21.6", with_installed_binary=True, stay_alive=True, allow_analyzer=False, @@ -31,7 +31,7 @@ node3 = cluster.add_instance( node4 = cluster.add_instance( "node4", image="yandex/clickhouse-server", - tag="21.5", + tag="21.6", with_installed_binary=True, stay_alive=True, allow_analyzer=False, @@ -39,7 +39,7 @@ node4 = cluster.add_instance( node5 = cluster.add_instance( "node5", image="yandex/clickhouse-server", - tag="21.5", + tag="21.6", with_installed_binary=True, stay_alive=True, allow_analyzer=False, @@ -47,7 +47,7 @@ node5 = cluster.add_instance( node6 = cluster.add_instance( "node6", image="yandex/clickhouse-server", - tag="21.5", + tag="21.6", with_installed_binary=True, stay_alive=True, allow_analyzer=False, diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index f3ae190ee46..9fb396b1c14 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -10,7 +10,7 @@ node1 = cluster.add_instance( "node1", with_zookeeper=True, image="yandex/clickhouse-server", - tag="20.4.9.110", + tag="20.8.11.17", with_installed_binary=True, stay_alive=True, main_configs=[ @@ -22,7 +22,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=True, image="yandex/clickhouse-server", - tag="20.4.9.110", + tag="20.8.11.17", with_installed_binary=True, stay_alive=True, main_configs=[ @@ -34,7 +34,7 @@ node3 = cluster.add_instance( "node3", with_zookeeper=True, image="yandex/clickhouse-server", - tag="20.4.9.110", + tag="20.8.11.17", with_installed_binary=True, stay_alive=True, main_configs=[ From 957671bf744cd173676f5be0f8ca14d0f03118df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 31 Oct 2023 18:06:28 +0100 Subject: [PATCH 105/813] Adapt to work with releases without DROP SYNC --- tests/integration/test_ttl_replicated/test.py | 222 ++++++++---------- 1 file changed, 102 insertions(+), 120 deletions(-) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 29ce2b3dc8d..119a211ae45 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -66,47 +66,41 @@ def started_cluster(): cluster.shutdown() -def drop_table(nodes, table_name): - for node in nodes: - node.query("DROP TABLE IF EXISTS {} SYNC".format(table_name)) - - # Column TTL works only with wide parts, because it's very expensive to apply it for compact parts def test_ttl_columns(started_cluster): - drop_table([node1, node2], "test_ttl") + table_name = f"test_ttl_{node1.name}_{node2.name}" for node in [node1, node2]: node.query( """ - CREATE TABLE test_ttl(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) + CREATE TABLE {table_name}(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_columns', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0, min_bytes_for_wide_part=0, max_merge_selecting_sleep_ms=6000; """.format( - replica=node.name + table_name=table_name, replica=node.name ) ) node1.query( - "INSERT INTO test_ttl VALUES (toDateTime('2000-10-10 00:00:00'), 1, 1, 3)" + f"INSERT INTO {table_name} VALUES (toDateTime('2000-10-10 00:00:00'), 1, 1, 3)" ) node1.query( - "INSERT INTO test_ttl VALUES (toDateTime('2000-10-11 10:00:00'), 2, 2, 4)" + f"INSERT INTO {table_name} VALUES (toDateTime('2000-10-11 10:00:00'), 2, 2, 4)" ) time.sleep(1) # sleep to allow use ttl merge selector for second time - node1.query("OPTIMIZE TABLE test_ttl FINAL") + node1.query(f"OPTIMIZE TABLE {table_name} FINAL") expected = "1\t0\t0\n2\t0\t0\n" - assert TSV(node1.query("SELECT id, a, b FROM test_ttl ORDER BY id")) == TSV( + assert TSV(node1.query(f"SELECT id, a, b FROM {table_name} ORDER BY id")) == TSV( expected ) - assert TSV(node2.query("SELECT id, a, b FROM test_ttl ORDER BY id")) == TSV( + assert TSV(node2.query(f"SELECT id, a, b FROM {table_name} ORDER BY id")) == TSV( expected ) def test_merge_with_ttl_timeout(started_cluster): - table = "test_merge_with_ttl_timeout" - drop_table([node1, node2], table) + table = f"test_merge_with_ttl_timeout_{node1.name}_{node2.name}" for node in [node1, node2]: node.query( """ @@ -157,11 +151,11 @@ def test_merge_with_ttl_timeout(started_cluster): def test_ttl_many_columns(started_cluster): - drop_table([node1, node2], "test_ttl_2") + table = f"test_ttl_2{node1.name}_{node2.name}" for node in [node1, node2]: node.query( """ - CREATE TABLE test_ttl_2(date DateTime, id UInt32, + CREATE TABLE {table}(date DateTime, id UInt32, a Int32 TTL date, _idx Int32 TTL date, _offset Int32 TTL date, @@ -169,44 +163,40 @@ def test_ttl_many_columns(started_cluster): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_2', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0, max_merge_selecting_sleep_ms=6000; """.format( - replica=node.name + table=table, replica=node.name ) ) - node1.query("SYSTEM STOP TTL MERGES test_ttl_2") - node2.query("SYSTEM STOP TTL MERGES test_ttl_2") + node1.query(f"SYSTEM STOP TTL MERGES {table}") + node2.query(f"SYSTEM STOP TTL MERGES {table}") node1.query( - "INSERT INTO test_ttl_2 VALUES (toDateTime('2000-10-10 00:00:00'), 1, 2, 3, 4, 5)" + f"INSERT INTO {table} VALUES (toDateTime('2000-10-10 00:00:00'), 1, 2, 3, 4, 5)" ) node1.query( - "INSERT INTO test_ttl_2 VALUES (toDateTime('2100-10-10 10:00:00'), 6, 7, 8, 9, 10)" + f"INSERT INTO {table} VALUES (toDateTime('2100-10-10 10:00:00'), 6, 7, 8, 9, 10)" ) - node2.query("SYSTEM SYNC REPLICA test_ttl_2", timeout=5) + node2.query(f"SYSTEM SYNC REPLICA {table}", timeout=5) # Check that part will appear in result of merge - node1.query("SYSTEM STOP FETCHES test_ttl_2") - node2.query("SYSTEM STOP FETCHES test_ttl_2") + node1.query(f"SYSTEM STOP FETCHES {table}") + node2.query(f"SYSTEM STOP FETCHES {table}") - node1.query("SYSTEM START TTL MERGES test_ttl_2") - node2.query("SYSTEM START TTL MERGES test_ttl_2") + node1.query(f"SYSTEM START TTL MERGES {table}") + node2.query(f"SYSTEM START TTL MERGES {table}") time.sleep(1) # sleep to allow use ttl merge selector for second time - node1.query("OPTIMIZE TABLE test_ttl_2 FINAL", timeout=5) + node1.query(f"OPTIMIZE TABLE {table} FINAL", timeout=5) - node2.query("SYSTEM SYNC REPLICA test_ttl_2", timeout=5) + node2.query(f"SYSTEM SYNC REPLICA {table}", timeout=5) expected = "1\t0\t0\t0\t0\n6\t7\t8\t9\t10\n" assert TSV( - node1.query( - "SELECT id, a, _idx, _offset, _partition FROM test_ttl_2 ORDER BY id" - ) + node1.query(f"SELECT id, a, _idx, _offset, _partition FROM {table} ORDER BY id") ) == TSV(expected) assert TSV( - node2.query( - "SELECT id, a, _idx, _offset, _partition FROM test_ttl_2 ORDER BY id" - ) + node2.query(f"SELECT id, a, _idx, _offset, _partition FROM {table} ORDER BY id") ) == TSV(expected) @@ -218,107 +208,107 @@ def test_ttl_many_columns(started_cluster): ], ) def test_ttl_table(started_cluster, delete_suffix): - drop_table([node1, node2], "test_ttl") + table = f"test_ttl_table_{delete_suffix}_{node1.name}_{node2.name}" for node in [node1, node2]: node.query( """ - CREATE TABLE test_ttl(date DateTime, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}') + CREATE TABLE {table}(date DateTime, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 1 DAY {delete_suffix} SETTINGS merge_with_ttl_timeout=0, max_merge_selecting_sleep_ms=6000; """.format( - replica=node.name, delete_suffix=delete_suffix + table=table, replica=node.name, delete_suffix=delete_suffix ) ) - node1.query("INSERT INTO test_ttl VALUES (toDateTime('2000-10-10 00:00:00'), 1)") - node1.query("INSERT INTO test_ttl VALUES (toDateTime('2000-10-11 10:00:00'), 2)") + node1.query(f"INSERT INTO {table} VALUES (toDateTime('2000-10-10 00:00:00'), 1)") + node1.query(f"INSERT INTO {table} VALUES (toDateTime('2000-10-11 10:00:00'), 2)") time.sleep(1) # sleep to allow use ttl merge selector for second time - node1.query("OPTIMIZE TABLE test_ttl FINAL") + node1.query(f"OPTIMIZE TABLE {table} FINAL") - assert TSV(node1.query("SELECT * FROM test_ttl")) == TSV("") - assert TSV(node2.query("SELECT * FROM test_ttl")) == TSV("") + assert TSV(node1.query(f"SELECT * FROM {table}")) == TSV("") + assert TSV(node2.query(f"SELECT * FROM {table}")) == TSV("") def test_modify_ttl(started_cluster): - drop_table([node1, node2], "test_ttl") + table = f"test_modify_ttl_{node1.name}_{node2.name}" for node in [node1, node2]: node.query( """ - CREATE TABLE test_ttl(d DateTime, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_modify', '{replica}') + CREATE TABLE {table}(d DateTime, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') ORDER BY id """.format( - replica=node.name + table=table, replica=node.name ) ) node1.query( - "INSERT INTO test_ttl VALUES (now() - INTERVAL 5 HOUR, 1), (now() - INTERVAL 3 HOUR, 2), (now() - INTERVAL 1 HOUR, 3)" + f"INSERT INTO {table} VALUES (now() - INTERVAL 5 HOUR, 1), (now() - INTERVAL 3 HOUR, 2), (now() - INTERVAL 1 HOUR, 3)" ) - node2.query("SYSTEM SYNC REPLICA test_ttl", timeout=20) + node2.query(f"SYSTEM SYNC REPLICA {table}", timeout=20) node1.query( - "ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 4 HOUR SETTINGS replication_alter_partitions_sync = 2" + f"ALTER TABLE {table} MODIFY TTL d + INTERVAL 4 HOUR SETTINGS replication_alter_partitions_sync = 2" ) - assert node2.query("SELECT id FROM test_ttl") == "2\n3\n" + assert node2.query(f"SELECT id FROM {table}") == "2\n3\n" node2.query( - "ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 2 HOUR SETTINGS replication_alter_partitions_sync = 2" + f"ALTER TABLE {table} MODIFY TTL d + INTERVAL 2 HOUR SETTINGS replication_alter_partitions_sync = 2" ) - assert node1.query("SELECT id FROM test_ttl") == "3\n" + assert node1.query(f"SELECT id FROM {table}") == "3\n" node1.query( - "ALTER TABLE test_ttl MODIFY TTL d + INTERVAL 30 MINUTE SETTINGS replication_alter_partitions_sync = 2" + f"ALTER TABLE {table} MODIFY TTL d + INTERVAL 30 MINUTE SETTINGS replication_alter_partitions_sync = 2" ) - assert node2.query("SELECT id FROM test_ttl") == "" + assert node2.query(f"SELECT id FROM {table}") == "" def test_modify_column_ttl(started_cluster): - drop_table([node1, node2], "test_ttl") + table = f"test_modify_column_ttl_{node1.name}_{node2.name}" for node in [node1, node2]: node.query( """ - CREATE TABLE test_ttl(d DateTime, id UInt32 DEFAULT 42) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_column', '{replica}') + CREATE TABLE {table}(d DateTime, id UInt32 DEFAULT 42) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') ORDER BY d """.format( - replica=node.name + table=table, replica=node.name ) ) node1.query( - "INSERT INTO test_ttl VALUES (now() - INTERVAL 5 HOUR, 1), (now() - INTERVAL 3 HOUR, 2), (now() - INTERVAL 1 HOUR, 3)" + f"INSERT INTO {table} VALUES (now() - INTERVAL 5 HOUR, 1), (now() - INTERVAL 3 HOUR, 2), (now() - INTERVAL 1 HOUR, 3)" ) - node2.query("SYSTEM SYNC REPLICA test_ttl", timeout=20) + node2.query(f"SYSTEM SYNC REPLICA {table}", timeout=20) node1.query( - "ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 4 HOUR SETTINGS replication_alter_partitions_sync = 2" + f"ALTER TABLE {table} MODIFY COLUMN id UInt32 TTL d + INTERVAL 4 HOUR SETTINGS replication_alter_partitions_sync = 2" ) - assert node2.query("SELECT id FROM test_ttl") == "42\n2\n3\n" + assert node2.query(f"SELECT id FROM {table}") == "42\n2\n3\n" node1.query( - "ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 2 HOUR SETTINGS replication_alter_partitions_sync = 2" + f"ALTER TABLE {table} MODIFY COLUMN id UInt32 TTL d + INTERVAL 2 HOUR SETTINGS replication_alter_partitions_sync = 2" ) - assert node1.query("SELECT id FROM test_ttl") == "42\n42\n3\n" + assert node1.query(f"SELECT id FROM {table}") == "42\n42\n3\n" node1.query( - "ALTER TABLE test_ttl MODIFY COLUMN id UInt32 TTL d + INTERVAL 30 MINUTE SETTINGS replication_alter_partitions_sync = 2" + f"ALTER TABLE {table} MODIFY COLUMN id UInt32 TTL d + INTERVAL 30 MINUTE SETTINGS replication_alter_partitions_sync = 2" ) - assert node2.query("SELECT id FROM test_ttl") == "42\n42\n42\n" + assert node2.query(f"SELECT id FROM {table}") == "42\n42\n42\n" def test_ttl_double_delete_rule_returns_error(started_cluster): - drop_table([node1, node2], "test_ttl") + table = "test_ttl_double_delete_rule_returns_error" try: node1.query( """ - CREATE TABLE test_ttl(date DateTime, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_double_delete', '{replica}') + CREATE TABLE {table}(date DateTime, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 1 DAY, date + INTERVAL 2 DAY SETTINGS merge_with_ttl_timeout=0, max_merge_selecting_sleep_ms=6000 """.format( - replica=node1.name + table=table, replica=node1.name ) ) assert False @@ -364,7 +354,6 @@ def test_ttl_alter_delete(started_cluster, name, engine): for a table that has TTL delete expression defined but no explicit storage policy assigned. """ - drop_table([node1], name) node1.query( """ @@ -426,7 +415,6 @@ def test_ttl_alter_delete(started_cluster, name, engine): def test_ttl_empty_parts(started_cluster): - drop_table([node1, node2], "test_ttl_empty_parts") for node in [node1, node2]: node.query( """ @@ -519,65 +507,59 @@ def test_ttl_empty_parts(started_cluster): [(node1, node2, 0), (node3, node4, 1), (node5, node6, 2)], ) def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): - drop_table([node_left, node_right], "test_ttl_delete") - drop_table([node_left, node_right], "test_ttl_group_by") - drop_table([node_left, node_right], "test_ttl_where") - + table = f"test_ttl_compatibility_{node_left.name}_{node_right.name}_{num_run}" for node in [node_left, node_right]: node.query( """ - CREATE TABLE test_ttl_delete(date DateTime, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_delete_{suff}', '{replica}') + CREATE TABLE {table}_delete(date DateTime, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}_delete', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 """.format( - suff=num_run, replica=node.name + table=table, replica=node.name ) ) node.query( """ - CREATE TABLE test_ttl_group_by(date DateTime, id UInt32, val UInt64) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_group_by_{suff}', '{replica}') + CREATE TABLE {table}_group_by(date DateTime, id UInt32, val UInt64) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}_group_by', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND GROUP BY id SET val = sum(val) - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 """.format( - suff=num_run, replica=node.name + table=table, replica=node.name ) ) node.query( """ - CREATE TABLE test_ttl_where(date DateTime, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_where_{suff}', '{replica}') + CREATE TABLE {table}_where(date DateTime, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}_where', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND DELETE WHERE id % 2 = 1 - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 """.format( - suff=num_run, replica=node.name + table=table, replica=node.name ) ) - node_left.query("INSERT INTO test_ttl_delete VALUES (now(), 1)") + node_left.query(f"INSERT INTO {table}_delete VALUES (now(), 1)") node_left.query( - "INSERT INTO test_ttl_delete VALUES (toDateTime('2100-10-11 10:00:00'), 2)" + f"INSERT INTO {table}_delete VALUES (toDateTime('2100-10-11 10:00:00'), 2)" ) - node_right.query("INSERT INTO test_ttl_delete VALUES (now(), 3)") + node_right.query(f"INSERT INTO {table}_delete VALUES (now(), 3)") node_right.query( - "INSERT INTO test_ttl_delete VALUES (toDateTime('2100-10-11 10:00:00'), 4)" + f"INSERT INTO {table}_delete VALUES (toDateTime('2100-10-11 10:00:00'), 4)" ) - node_left.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 1)") - node_left.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 2)") - node_right.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 3)") - node_right.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 4)") + node_left.query(f"INSERT INTO {table}_group_by VALUES (now(), 0, 1)") + node_left.query(f"INSERT INTO {table}_group_by VALUES (now(), 0, 2)") + node_right.query(f"INSERT INTO {table}_group_by VALUES (now(), 0, 3)") + node_right.query(f"INSERT INTO {table}_group_by VALUES (now(), 0, 4)") - node_left.query("INSERT INTO test_ttl_where VALUES (now(), 1)") - node_left.query("INSERT INTO test_ttl_where VALUES (now(), 2)") - node_right.query("INSERT INTO test_ttl_where VALUES (now(), 3)") - node_right.query("INSERT INTO test_ttl_where VALUES (now(), 4)") + node_left.query(f"INSERT INTO {table}_where VALUES (now(), 1)") + node_left.query(f"INSERT INTO {table}_where VALUES (now(), 2)") + node_right.query(f"INSERT INTO {table}_where VALUES (now(), 3)") + node_right.query(f"INSERT INTO {table}_where VALUES (now(), 4)") if node_left.with_installed_binary: node_left.restart_with_latest_version() @@ -588,13 +570,13 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): time.sleep(5) # Wait for TTL # after restart table can be in readonly mode - exec_query_with_retry(node_right, "OPTIMIZE TABLE test_ttl_delete FINAL") - node_right.query("OPTIMIZE TABLE test_ttl_group_by FINAL") - node_right.query("OPTIMIZE TABLE test_ttl_where FINAL") + exec_query_with_retry(node_right, f"OPTIMIZE TABLE {table}_delete FINAL") + node_right.query(f"OPTIMIZE TABLE {table}_group_by FINAL") + node_right.query(f"OPTIMIZE TABLE {table}_where FINAL") - exec_query_with_retry(node_left, "OPTIMIZE TABLE test_ttl_delete FINAL") - node_left.query("OPTIMIZE TABLE test_ttl_group_by FINAL", timeout=20) - node_left.query("OPTIMIZE TABLE test_ttl_where FINAL", timeout=20) + exec_query_with_retry(node_left, f"OPTIMIZE TABLE {table}_delete FINAL") + node_left.query(f"OPTIMIZE TABLE {table}_group_by FINAL", timeout=20) + node_left.query(f"OPTIMIZE TABLE {table}_where FINAL", timeout=20) # After OPTIMIZE TABLE, it is not guaranteed that everything is merged. # Possible scenario (for test_ttl_group_by): @@ -605,19 +587,19 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): # 4. OPTIMIZE FINAL does nothing, cause there is an entry for 0_3 # # So, let's also sync replicas for node_right (for now). - exec_query_with_retry(node_right, "SYSTEM SYNC REPLICA test_ttl_delete") - node_right.query("SYSTEM SYNC REPLICA test_ttl_group_by", timeout=20) - node_right.query("SYSTEM SYNC REPLICA test_ttl_where", timeout=20) + exec_query_with_retry(node_right, f"SYSTEM SYNC REPLICA {table}_delete") + node_right.query(f"SYSTEM SYNC REPLICA {table}_group_by", timeout=20) + node_right.query(f"SYSTEM SYNC REPLICA {table}_where", timeout=20) - exec_query_with_retry(node_left, "SYSTEM SYNC REPLICA test_ttl_delete") - node_left.query("SYSTEM SYNC REPLICA test_ttl_group_by", timeout=20) - node_left.query("SYSTEM SYNC REPLICA test_ttl_where", timeout=20) + exec_query_with_retry(node_left, f"SYSTEM SYNC REPLICA {table}_delete") + node_left.query(f"SYSTEM SYNC REPLICA {table}_group_by", timeout=20) + node_left.query(f"SYSTEM SYNC REPLICA {table}_where", timeout=20) - assert node_left.query("SELECT id FROM test_ttl_delete ORDER BY id") == "2\n4\n" - assert node_right.query("SELECT id FROM test_ttl_delete ORDER BY id") == "2\n4\n" + assert node_left.query(f"SELECT id FROM {table}_delete ORDER BY id") == "2\n4\n" + assert node_right.query(f"SELECT id FROM {table}_delete ORDER BY id") == "2\n4\n" - assert node_left.query("SELECT val FROM test_ttl_group_by ORDER BY id") == "10\n" - assert node_right.query("SELECT val FROM test_ttl_group_by ORDER BY id") == "10\n" + assert node_left.query(f"SELECT val FROM {table}_group_by ORDER BY id") == "10\n" + assert node_right.query(f"SELECT val FROM {table}_group_by ORDER BY id") == "10\n" - assert node_left.query("SELECT id FROM test_ttl_where ORDER BY id") == "2\n4\n" - assert node_right.query("SELECT id FROM test_ttl_where ORDER BY id") == "2\n4\n" + assert node_left.query(f"SELECT id FROM {table}_where ORDER BY id") == "2\n4\n" + assert node_right.query(f"SELECT id FROM {table}_where ORDER BY id") == "2\n4\n" From f2f84fe6b7f49ae3bad1ed6f8c19c608f73d53d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 31 Oct 2023 18:24:33 +0100 Subject: [PATCH 106/813] Adapt version changes --- .../test_vertical_merges_from_compact_parts.py | 2 +- tests/integration/test_default_compression_codec/test.py | 2 +- .../integration/test_version_update_after_mutation/test.py | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index e0a9b5ebad6..9c9d1a4d312 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -7,7 +7,7 @@ cluster = ClickHouseCluster(__file__) node_old = cluster.add_instance( "node1", image="clickhouse/clickhouse-server", - tag="22.6", + tag="22.8", stay_alive=True, with_installed_binary=True, with_zookeeper=True, diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py index db116ff42f3..ffe22c62325 100644 --- a/tests/integration/test_default_compression_codec/test.py +++ b/tests/integration/test_default_compression_codec/test.py @@ -27,7 +27,7 @@ node2 = cluster.add_instance( ) node3 = cluster.add_instance( "node3", - main_configs=["configs/default_compression.xml", "configs/wide_parts_only.xml"], + main_configs=["configs/default_compression.xml"], image="yandex/clickhouse-server", tag="19.16.9.37", stay_alive=True, diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 9fb396b1c14..4e84b4c10ca 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -72,8 +72,8 @@ def test_mutate_and_upgrade(start_cluster): node1.query("DETACH TABLE mt") # stop being leader node1.query("SYSTEM FLUSH LOGS") node2.query("SYSTEM FLUSH LOGS") - node1.restart_with_latest_version(signal=9, fix_metadata=True) - node2.restart_with_latest_version(signal=9, fix_metadata=True) + node1.restart_with_latest_version(signal=9, fix_metadata=False) + node2.restart_with_latest_version(signal=9, fix_metadata=False) # After hard restart table can be in readonly mode exec_query_with_retry( @@ -129,7 +129,7 @@ def test_upgrade_while_mutation(start_cluster): # (We could be in process of creating some system table, which will leave empty directory on restart, # so when we start moving system tables from ordinary to atomic db, it will complain about some undeleted files) node3.query("SYSTEM FLUSH LOGS") - node3.restart_with_latest_version(signal=9, fix_metadata=True) + node3.restart_with_latest_version(signal=9, fix_metadata=False) # checks for readonly exec_query_with_retry(node3, "OPTIMIZE TABLE mt1", sleep_time=5, retry_count=60) From b27658742223e750902ec3f181d2a662fc7bba1f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Nov 2023 15:43:20 +0800 Subject: [PATCH 107/813] fix failed uts --- .../Impl/NativeORCBlockInputFormat.cpp | 130 ++++++++++++------ 1 file changed, 90 insertions(+), 40 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index fd0f4ee0ca0..2c3db6432c8 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -100,11 +100,11 @@ std::unique_ptr asORCInputStreamLoadIntoMemory(ReadBuffer & in return std::make_unique(std::move(file_data), file_size); } -static const orc::Type * getORCTypeByName(const orc::Type & schema, const String & name, bool case_insensitive_column_matching) +static const orc::Type * getORCTypeByName(const orc::Type & schema, const String & name, bool ignore_case) { for (UInt64 i = 0; i != schema.getSubtypeCount(); ++i) if (boost::equals(schema.getFieldName(i), name) - || (case_insensitive_column_matching && boost::iequals(schema.getFieldName(i), name))) + || (ignore_case && boost::iequals(schema.getFieldName(i), name))) return schema.getSubtype(i); return nullptr; } @@ -690,10 +690,10 @@ static std::string toDotColumnPath(const std::vector & columns) return column_path.substr(0, column_path.length() - 1); } -static void buildORCTypeNameIdMap( +[[maybe_unused]] static void buildORCTypeNameIdMap( const orc::Type * orc_type, std::vector & columns, - bool case_insensitive_column_matching, + bool ignore_case, std::map & id_type_map, std::map & name_id_map) { @@ -705,10 +705,10 @@ static void buildORCTypeNameIdMap( const std::string & field_name = orc_type->getFieldName(i); columns.push_back(field_name); auto column_path = toDotColumnPath(columns); - if (case_insensitive_column_matching) + if (ignore_case) boost::to_lower(column_path); name_id_map[column_path] = orc_type->getSubtype(i)->getColumnId(); - buildORCTypeNameIdMap(orc_type->getSubtype(i), columns, case_insensitive_column_matching, id_type_map, name_id_map); + buildORCTypeNameIdMap(orc_type->getSubtype(i), columns, ignore_case, id_type_map, name_id_map); columns.pop_back(); } } @@ -716,7 +716,7 @@ static void buildORCTypeNameIdMap( { // other non-primitive type for (size_t j = 0; j < orc_type->getSubtypeCount(); ++j) - buildORCTypeNameIdMap(orc_type->getSubtype(j), columns, case_insensitive_column_matching, id_type_map, name_id_map); + buildORCTypeNameIdMap(orc_type->getSubtype(j), columns, ignore_case, id_type_map, name_id_map); } } @@ -747,10 +747,54 @@ static void getFileReaderAndSchema( } } -static void updateIncludeTypeIds( - DataTypePtr type, const orc::Type * orc_type, bool case_insensitive_column_matching, std::unordered_set & include_typeids) +static const orc::Type * traverseDownORCTypeByName( + const std::string & target, + const orc::Type * orc_type, + DataTypePtr & type, + bool ignore_case) { - /// Primitive types + // std::cout << "target:" << target << ", orc_type:" << orc_type->toString() << ", type:" << type->getName() << std::endl; + if (target.empty()) + return orc_type; + + auto split = Nested::splitName(target); + if (orc::STRUCT == orc_type->getKind()) + { + const auto * orc_field_type = getORCTypeByName(*orc_type, split.first, ignore_case); + return orc_field_type ? traverseDownORCTypeByName(split.second, orc_field_type, type, ignore_case) : nullptr; + } + else if (orc::LIST == orc_type->getKind()) + { + /// For cases in which header contains subcolumns flattened from nested columns. + /// For example, "a Nested(x String, y Int64)" is flattened to "a.x Array(String), a.y Array(Int64)", and orc file schema is still "a array>". + /// In this case, we should skip possible array type and traverse down to its nested struct type. + const auto * array_type = typeid_cast(removeNullable(type).get()); + const auto * orc_nested_type = orc_type->getSubtype(0); + if (array_type && orc::STRUCT == orc_nested_type->getKind()) + { + const auto * orc_field_type = getORCTypeByName(*orc_nested_type, split.first, ignore_case); + if (orc_field_type) + { + /// Avoid inconsistency between CH and ORC type brought by flattened Nested type. + type = array_type->getNestedType(); + return traverseDownORCTypeByName(split.second, orc_field_type, type, ignore_case); + } + else + return nullptr; + } + else + return nullptr; + } + else + return nullptr; +} + +static void updateIncludeTypeIds( + DataTypePtr type, const orc::Type * orc_type, bool ignore_case, std::unordered_set & include_typeids) +{ + // std::cout << "ch type:" << type->getName() << ", orc_type:" << orc_type->toString() << std::endl; + + /// For primitive types, directly append column id into result if (orc_type->getSubtypeCount() == 0) { include_typeids.insert(orc_type->getColumnId()); @@ -765,7 +809,7 @@ static void updateIncludeTypeIds( if (array_type) { updateIncludeTypeIds( - array_type->getNestedType(), orc_type->getSubtype(0), case_insensitive_column_matching, include_typeids); + array_type->getNestedType(), orc_type->getSubtype(0), ignore_case, include_typeids); } return; } @@ -773,8 +817,8 @@ static void updateIncludeTypeIds( const auto * map_type = typeid_cast(non_nullable_type.get()); if (map_type) { - updateIncludeTypeIds(map_type->getKeyType(), orc_type->getSubtype(0), case_insensitive_column_matching, include_typeids); - updateIncludeTypeIds(map_type->getValueType(), orc_type->getSubtype(1), case_insensitive_column_matching, include_typeids); + updateIncludeTypeIds(map_type->getKeyType(), orc_type->getSubtype(0), ignore_case, include_typeids); + updateIncludeTypeIds(map_type->getValueType(), orc_type->getSubtype(1), ignore_case, include_typeids); } return; } @@ -791,12 +835,12 @@ static void updateIncludeTypeIds( for (size_t struct_i = 0; struct_i < orc_type->getSubtypeCount(); ++struct_i) { if (boost::equals(orc_type->getFieldName(struct_i), name) - || (case_insensitive_column_matching && boost::iequals(orc_type->getFieldName(struct_i), name))) + || (ignore_case && boost::iequals(orc_type->getFieldName(struct_i), name))) { updateIncludeTypeIds( tuple_type->getElement(tuple_i), orc_type->getSubtype(struct_i), - case_insensitive_column_matching, + ignore_case, include_typeids); break; } @@ -807,7 +851,7 @@ static void updateIncludeTypeIds( { for (size_t i = 0; i < tuple_type->getElements().size() && i < orc_type->getSubtypeCount(); ++i) updateIncludeTypeIds( - tuple_type->getElement(i), orc_type->getSubtype(i), case_insensitive_column_matching, include_typeids); + tuple_type->getElement(i), orc_type->getSubtype(i), ignore_case, include_typeids); } } return; @@ -838,40 +882,47 @@ void NativeORCBlockInputFormat::prepareFileReader() format_settings.null_as_default, format_settings.orc.case_insensitive_column_matching); - const bool ignore_case = format_settings.orc.case_insensitive_column_matching; - std::vector columns; - std::map id_type_map; - std::map name_id_map; - buildORCTypeNameIdMap(&file_reader->getType(), columns, ignore_case, id_type_map, name_id_map); - - // std::cout << "subtypes:" << file_reader->getType().getSubtypeCount() << std::endl; - // std::cout << "id type map" << std::endl; - // for (const auto & [k, v]: id_type_map) - // std::cout << "id:" << k << ", type:" << v->toString() << std::endl; - // std::cout << "name id map" << std::endl; - // for (const auto & [k, v]: name_id_map) - // std::cout << "name:" << k << ", id:" << v << std::endl; const auto & header = getPort().getHeader(); + const auto & file_schema = file_reader->getType(); std::unordered_set include_typeids; for (const auto & column : header) { - auto name = column.name; - if (ignore_case) - boost::to_lower(name); - - if (name_id_map.contains(name)) + auto split = Nested::splitName(column.name); + if (split.second.empty()) { - auto id = name_id_map[name]; - if (id_type_map.contains(id)) - { - updateIncludeTypeIds(column.type, id_type_map[id], ignore_case, include_typeids); - } + const auto * orc_type = getORCTypeByName(file_schema, column.name, ignore_case); + updateIncludeTypeIds(column.type, orc_type, ignore_case, include_typeids); + } + else + { + auto type = column.type; + const auto * orc_type = traverseDownORCTypeByName(column.name, &file_schema, type, ignore_case); + if (orc_type) + updateIncludeTypeIds(type, orc_type, ignore_case, include_typeids); } } include_indices.assign(include_typeids.begin(), include_typeids.end()); + /// Just for Debug + // std::vector tmp; + // std::map id_type_map; + // std::map name_id_map; + // buildORCTypeNameIdMap(&file_schema, tmp, ignore_case, id_type_map, name_id_map); + // std::cout << "just for debug:" << std::endl; + // std::cout << "subtypes:" << file_reader->getType().getSubtypeCount() << std::endl; + // std::cout << "ch output type:" << getPort().getHeader().dumpStructure() << std::endl; + // std::cout << "orc ouput type:" << file_reader->getType().toString() << std::endl; + // std::cout << "id type map" << std::endl; + // for (const auto & [k, v] : id_type_map) + // std::cout << "id:" << k << ", type:" << v->toString() << std::endl; + // std::cout << "name id map" << std::endl; + // for (const auto & [k, v] : name_id_map) + // std::cout << "name:" << k << ", id:" << v << std::endl; + // for (const auto & x : include_indices) + // std::cout << "choose " << x << std::endl; + if (format_settings.orc.filter_push_down && key_condition && !sarg) { sarg = buildORCSearchArgument(*key_condition, getPort().getHeader(), file_reader->getType(), format_settings); @@ -951,7 +1002,6 @@ Chunk NativeORCBlockInputFormat::generate() Chunk res; size_t num_rows = batch->numElements; const auto & schema = stripe_reader->getSelectedType(); - // std::cout << "output schema:" << schema.toString() << std::endl; orc_column_to_ch_column->orcTableToCHChunk(res, &schema, batch.get(), num_rows, &block_missing_values); approx_bytes_read_for_chunk = num_rows * current_stripe_info->getLength() / current_stripe_info->getNumberOfRows(); From 001cbe79126ea96b8f70ae3c5e17655c73df30cd Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Nov 2023 16:58:25 +0800 Subject: [PATCH 108/813] fix typos --- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 2c3db6432c8..3f98224f8aa 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -913,7 +913,7 @@ void NativeORCBlockInputFormat::prepareFileReader() // std::cout << "just for debug:" << std::endl; // std::cout << "subtypes:" << file_reader->getType().getSubtypeCount() << std::endl; // std::cout << "ch output type:" << getPort().getHeader().dumpStructure() << std::endl; - // std::cout << "orc ouput type:" << file_reader->getType().toString() << std::endl; + // std::cout << "orc output type:" << file_reader->getType().toString() << std::endl; // std::cout << "id type map" << std::endl; // for (const auto & [k, v] : id_type_map) // std::cout << "id:" << k << ", type:" << v->toString() << std::endl; From 1ae951efc2aec7b3fc236cc16fa8272af1d5e515 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Wed, 25 Oct 2023 21:41:32 +0800 Subject: [PATCH 109/813] Add unittest MergeTree.CombineFilters This commit adds a unittest for validating DB::combineFilters. --- .../MergeTree/tests/gtest_combine_filters.cpp | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) create mode 100644 src/Storages/MergeTree/tests/gtest_combine_filters.cpp diff --git a/src/Storages/MergeTree/tests/gtest_combine_filters.cpp b/src/Storages/MergeTree/tests/gtest_combine_filters.cpp new file mode 100644 index 00000000000..64cccd4cbad --- /dev/null +++ b/src/Storages/MergeTree/tests/gtest_combine_filters.cpp @@ -0,0 +1,66 @@ +#include +#include + +// I know that inclusion of .cpp is not good at all +#include // NOLINT + +using namespace DB; + +/* The combineFilters function from MergeTreeRangeReader.cpp could be optimized with Intel's AVX512VBMI2 intrinsic, + * _mm512_mask_expandloadu_epi8. And this test is added to ensure that the vectorized code outputs the exact results + * as the original scalar code when the required hardware feature is supported on the device. + * + * To avoid the contingency of the all-one/all-zero sequences, this test fills in the filters with alternating 1s and + * 0s so that only the 4i-th (i is a non-negative integer) elements in the combined filter equals 1s and others are 0s. + * For example, given the size of the first filter to be 11, the generated and the output filters are: + * + * first_filter: [1 0 1 0 1 0 1 0 1 0 1] + * second_filter: [1 0 1 0 1 0] + * output_filter: [1 0 0 0 1 0 0 0 1 0 0] + */ +bool testCombineFilters(size_t size) +{ + auto generateFilterWithAlternatingOneAndZero = [](size_t len)->ColumnPtr + { + auto filter = ColumnUInt8::create(len, 0); + auto & filter_data = filter->getData(); + + for (size_t i = 0; i < len; i += 2) + filter_data[i] = 1; + + return filter; + }; + + auto first_filter = generateFilterWithAlternatingOneAndZero(size); + /// The count of 1s in the first_filter is floor((size + 1) / 2), which should be the size of the second_filter. + auto second_filter = generateFilterWithAlternatingOneAndZero((size + 1) / 2); + + auto result = combineFilters(first_filter, second_filter); + + if (result->size() != size) return false; + + for (size_t i = 0; i < size; i++) + { + if (i % 4 == 0) + { + if (result->get64(i) != 1) return false; + } + else + { + if (result->get64(i) != 0) return false; + } + } + + return true; +} + +TEST(MergeTree, CombineFilters) +{ + EXPECT_TRUE(testCombineFilters(1)); + EXPECT_TRUE(testCombineFilters(2)); + EXPECT_TRUE(testCombineFilters(63)); + EXPECT_TRUE(testCombineFilters(64)); + EXPECT_TRUE(testCombineFilters(65)); + EXPECT_TRUE(testCombineFilters(200)); + EXPECT_TRUE(testCombineFilters(201)); +} From b66a9e8fd34eae519dee1f3120cf2a4082b602c7 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Fri, 27 Oct 2023 16:18:30 +0800 Subject: [PATCH 110/813] Optimize DB::combineFilters with AVX512_VBMI2 intrinsic The DB::combineFilters combines two UInt8 columns (filters) by scanning the first column for non-zero elements and replacing them with elements in the second column contiguously. This function is optimized with Intel's AVX512_VBMI2 intrinsic, _mm512_mask_expandloadu_epi8, which is designed for implementing the exact operation when the bits in the writemask are set when the corresponding packed integers in the first filter are non-zero, and are not set otherwise. The dynamic dispatch ensures that this optimization only applies to devices with AVX512_VBMI2 support. --- .../MergeTree/MergeTreeRangeReader.cpp | 47 +++++++++++++++++-- 1 file changed, 43 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 21e526778e1..84e3c10eace 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -18,6 +18,10 @@ #include #endif +#if USE_MULTITARGET_CODE +#include +#endif + #if defined(__aarch64__) && defined(__ARM_NEON) # include # pragma clang diagnostic ignored "-Wreserved-identifier" @@ -1253,6 +1257,32 @@ static void checkCombinedFiltersSize(size_t bytes_in_first_filter, size_t second "does not match second filter size ({})", bytes_in_first_filter, second_filter_size); } +DECLARE_AVX512VBMI2_SPECIFIC_CODE( +inline void combineFiltersImpl(UInt8 * first_begin, const UInt8 * first_end, const UInt8 * second_begin) +{ + constexpr size_t AVX512_VEC_SIZE_IN_BYTES = 64; + + while (first_begin + AVX512_VEC_SIZE_IN_BYTES <= first_end) + { + UInt64 mask = bytes64MaskToBits64Mask(first_begin); + __m512i src = _mm512_loadu_si512(reinterpret_cast(first_begin)); + __m512i dst = _mm512_mask_expandloadu_epi8(src, static_cast<__mmask64>(mask), reinterpret_cast(second_begin)); + _mm512_storeu_si512(reinterpret_cast(first_begin), dst); + + first_begin += AVX512_VEC_SIZE_IN_BYTES; + second_begin += std::popcount(mask); + } + + for (/* empty */; first_begin < first_end; ++first_begin) + { + if (*first_begin) + { + *first_begin = *second_begin++; + } + } +} +) + /// Second filter size must be equal to number of 1s in the first filter. /// The result has size equal to first filter size and contains 1s only where both filters contain 1s. static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) @@ -1295,12 +1325,21 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) auto & first_data = typeid_cast(mut_first.get())->getData(); const auto * second_data = second_descr.data->data(); - for (auto & val : first_data) +#if USE_MULTITARGET_CODE + if (isArchSupported(TargetArch::AVX512VBMI2)) { - if (val) + TargetSpecific::AVX512VBMI2::combineFiltersImpl(first_data.begin(), first_data.end(), second_data); + } + else +#endif + { + for (auto & val : first_data) { - val = *second_data; - ++second_data; + if (val) + { + val = *second_data; + ++second_data; + } } } From 24c45a4ee060c1fb2447d8acb4f0281f97ab65f1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Nov 2023 18:47:11 +0800 Subject: [PATCH 111/813] fix failed uts --- .../Impl/NativeORCBlockInputFormat.cpp | 20 +++++-------------- 1 file changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 3f98224f8aa..7a835274bb2 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -775,7 +775,7 @@ static const orc::Type * traverseDownORCTypeByName( const auto * orc_field_type = getORCTypeByName(*orc_nested_type, split.first, ignore_case); if (orc_field_type) { - /// Avoid inconsistency between CH and ORC type brought by flattened Nested type. + /// Adjust CH type to avoid inconsistency between CH and ORC type brought by flattened Nested type. type = array_type->getNestedType(); return traverseDownORCTypeByName(split.second, orc_field_type, type, ignore_case); } @@ -883,25 +883,15 @@ void NativeORCBlockInputFormat::prepareFileReader() format_settings.orc.case_insensitive_column_matching); const bool ignore_case = format_settings.orc.case_insensitive_column_matching; - const auto & header = getPort().getHeader(); const auto & file_schema = file_reader->getType(); std::unordered_set include_typeids; for (const auto & column : header) { - auto split = Nested::splitName(column.name); - if (split.second.empty()) - { - const auto * orc_type = getORCTypeByName(file_schema, column.name, ignore_case); - updateIncludeTypeIds(column.type, orc_type, ignore_case, include_typeids); - } - else - { - auto type = column.type; - const auto * orc_type = traverseDownORCTypeByName(column.name, &file_schema, type, ignore_case); - if (orc_type) - updateIncludeTypeIds(type, orc_type, ignore_case, include_typeids); - } + auto adjusted_type = column.type; + const auto * orc_type = traverseDownORCTypeByName(column.name, &file_schema, adjusted_type, ignore_case); + if (orc_type) + updateIncludeTypeIds(adjusted_type, orc_type, ignore_case, include_typeids); } include_indices.assign(include_typeids.begin(), include_typeids.end()); From dc897215dacea4f447b127254914e32f76ca001e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Nov 2023 20:42:07 +0800 Subject: [PATCH 112/813] fix failed uts tests/queries/0_stateless/02312_parquet_orc_arrow_names_tuples.sql --- .../Impl/NativeORCBlockInputFormat.cpp | 35 +++++++++++++++---- 1 file changed, 29 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 7a835274bb2..9501efbabb7 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -757,11 +757,32 @@ static const orc::Type * traverseDownORCTypeByName( if (target.empty()) return orc_type; - auto split = Nested::splitName(target); + auto search_struct_field = [&](const std::string & target_, const orc::Type * type_) -> std::pair + { + auto target_copy = target_; + if (ignore_case) + boost::to_lower(target_copy); + + for (size_t i = 0; i < type_->getSubtypeCount(); ++i) + { + auto field_name = type_->getFieldName(i); + if (ignore_case) + boost::to_lower(field_name); + + if (startsWith(target_copy, field_name) && (target_copy.size() == field_name.size() || target_copy[field_name.size()] == '.')) + { + return {target_copy.size() == field_name.size() ? "" : target_.substr(field_name.size() + 1), type_->getSubtype(i)}; + } + } + return {"", nullptr}; + }; + if (orc::STRUCT == orc_type->getKind()) { - const auto * orc_field_type = getORCTypeByName(*orc_type, split.first, ignore_case); - return orc_field_type ? traverseDownORCTypeByName(split.second, orc_field_type, type, ignore_case) : nullptr; + auto next_type_and_target = search_struct_field(target, orc_type); + const auto & next_target = next_type_and_target.first; + const auto * next_orc_type = next_type_and_target.second; + return next_orc_type ? traverseDownORCTypeByName(next_target, next_orc_type, type, ignore_case) : nullptr; } else if (orc::LIST == orc_type->getKind()) { @@ -772,12 +793,14 @@ static const orc::Type * traverseDownORCTypeByName( const auto * orc_nested_type = orc_type->getSubtype(0); if (array_type && orc::STRUCT == orc_nested_type->getKind()) { - const auto * orc_field_type = getORCTypeByName(*orc_nested_type, split.first, ignore_case); - if (orc_field_type) + auto next_type_and_target = search_struct_field(target, orc_nested_type); + const auto & next_target = next_type_and_target.first; + const auto * next_orc_type = next_type_and_target.second; + if (next_orc_type) { /// Adjust CH type to avoid inconsistency between CH and ORC type brought by flattened Nested type. type = array_type->getNestedType(); - return traverseDownORCTypeByName(split.second, orc_field_type, type, ignore_case); + return traverseDownORCTypeByName(next_target, next_orc_type, type, ignore_case); } else return nullptr; From 77507b843b66c4994ea46401d12b67f447154a39 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 1 Nov 2023 19:09:43 +0100 Subject: [PATCH 113/813] Fix build --- src/Common/ProfileEvents.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index d0589f7d281..31a62d0ff5c 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -415,7 +415,6 @@ The server successfully detected this situation and will download merged part fr M(FilesystemCacheEvictMicroseconds, "Filesystem cache eviction time") \ M(FilesystemCacheGetOrSetMicroseconds, "Filesystem cache getOrSet() time") \ M(FilesystemCacheGetMicroseconds, "Filesystem cache get() time") \ - M(FilesystemCacheUnusedHoldFileSegments, "Filesystem cache file segments count, which were hold, but not used (because of seek or LIMIT n, etc)") \ M(FileSegmentWaitMicroseconds, "Wait on DOWNLOADING state") \ M(FileSegmentCompleteMicroseconds, "Duration of FileSegment::complete() in filesystem cache") \ M(FileSegmentLockMicroseconds, "Lock file segment time") \ From 09c1e76982ce514e34da81aaff821b5ffcd63753 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 1 Nov 2023 21:42:34 +0100 Subject: [PATCH 114/813] Allow manual compaction of rocksdb via OPTIMIZE query Signed-off-by: Azat Khuzhin --- .../RocksDB/StorageEmbeddedRocksDB.cpp | 31 +++++++++++++++++++ src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 10 ++++++ .../02910_rocksdb_optimize.reference | 0 .../0_stateless/02910_rocksdb_optimize.sql | 5 +++ 4 files changed, 46 insertions(+) create mode 100644 tests/queries/0_stateless/02910_rocksdb_optimize.reference create mode 100644 tests/queries/0_stateless/02910_rocksdb_optimize.sql diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 5e8d54bcdf1..e81603d67b4 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -47,6 +47,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ROCKSDB_ERROR; + extern const int NOT_IMPLEMENTED; } using FieldVectorPtr = std::shared_ptr; @@ -310,6 +311,36 @@ void StorageEmbeddedRocksDB::drop() rocksdb_ptr = nullptr; } +bool StorageEmbeddedRocksDB::optimize( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & /* deduplicate_by_columns */, + bool cleanup, + ContextPtr /*context*/) +{ + if (partition) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Partition cannot be specified when optimizing table of type EmbeddedRocksDB"); + + if (final) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "FINAL cannot be specified when optimizing table of type EmbeddedRocksDB"); + + if (deduplicate) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DEDUPLICATE cannot be specified when optimizing table of type EmbeddedRocksDB"); + + if (cleanup) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CLEANUP cannot be specified when optimizing table of type EmbeddedRocksDB"); + + std::shared_lock lock(rocksdb_ptr_mx); + rocksdb::CompactRangeOptions compact_options; + auto status = rocksdb_ptr->CompactRange(compact_options, nullptr, nullptr); + if (!status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Compaction failed: {}", status.ToString()); + return true; +} + void StorageEmbeddedRocksDB::initDB() { rocksdb::Status status; diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 336f6a8abe3..9f83f3f2cf0 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -55,6 +55,16 @@ public: void mutate(const MutationCommands &, ContextPtr) override; void drop() override; + bool optimize( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & deduplicate_by_columns, + bool cleanup, + ContextPtr context) override; + bool supportsParallelInsert() const override { return true; } bool supportsIndexForIn() const override { return true; } bool mayBenefitFromIndexForIn( diff --git a/tests/queries/0_stateless/02910_rocksdb_optimize.reference b/tests/queries/0_stateless/02910_rocksdb_optimize.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02910_rocksdb_optimize.sql b/tests/queries/0_stateless/02910_rocksdb_optimize.sql new file mode 100644 index 00000000000..575ba6db212 --- /dev/null +++ b/tests/queries/0_stateless/02910_rocksdb_optimize.sql @@ -0,0 +1,5 @@ +-- Tags: use-rocksdb + +CREATE TABLE dict (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY key; +INSERT INTO dict SELECT number, toString(number) FROM numbers(1e3); +OPTIMIZE TABLE dict; From 1af52d3b0092c0e376075e389d8141e203e5c36b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 2 Nov 2023 00:42:57 +0000 Subject: [PATCH 115/813] Better --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 11 ++++++----- src/Storages/StorageReplicatedMergeTree.cpp | 12 ++++++------ ...8_replicated_merge_tree_creation_failure.sh | 18 +++++++++--------- 4 files changed, 22 insertions(+), 21 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 955ad815e00..213f5575716 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -607,7 +607,7 @@ class IColumn; 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, create_replicated_merge_tree_fault_injection, false, "If true, the creation of table will be aborted after creating metadata in ZooKeeper", 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) \ \ M(Bool, use_query_cache, false, "Enable the query cache", 0) \ M(Bool, enable_writes_to_query_cache, true, "Enable storing results of SELECT queries in the query cache", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3da6c9fa264..c88b1b5f968 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -104,7 +104,6 @@ namespace ErrorCodes extern const int UNKNOWN_STORAGE; extern const int SYNTAX_ERROR; extern const int SUPPORT_IS_DISABLED; - extern const int ABORTED; } namespace fs = std::filesystem; @@ -1443,11 +1442,13 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, "ATTACH ... FROM ... query is not supported for {} table engine, " "because such tables do not store any data on disk. Use CREATE instead.", res->getName()); - if (getContext()->getSettingsRef().create_replicated_merge_tree_fault_injection) + bool is_replicated_storage = typeid_cast(res.get()) != nullptr; + if (is_replicated_storage) { - bool is_replicated_storage = typeid_cast(res.get()) != nullptr; - if (is_replicated_storage) - throw Exception(ErrorCodes::ABORTED, "Shutdown is called for table"); + const auto probability = getContext()->getSettingsRef().create_replicated_merge_tree_fault_injection_probability; + std::bernoulli_distribution fault(probability); + if (fault(thread_local_rng)) + throw Coordination::Exception(Coordination::Error::ZCONNECTIONLOSS, "Fault injected (during table creation)"); } database->createTable(getContext(), create.getTable(), res, query_ptr); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0781684b7b7..d40e99aae1f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -836,7 +836,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_uuid", toString(ServerUUID::get()), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_info", toString(getStorageID().getFullTableName()) + "|" + toString(ServerUUID::get()), zkutil::CreateMode::Persistent)); Coordination::Responses responses; @@ -869,7 +869,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada const String local_metadata = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); const String local_columns = metadata_snapshot->getColumns().toString(); const String local_metadata_version = toString(metadata_snapshot->getMetadataVersion()); - const String creator_uuid = toString(ServerUUID::get()); + const String creator_info = toString(getStorageID().getFullTableName()) + "|" + toString(ServerUUID::get()); /// It is possible for the replica to fail after creating ZK nodes without saving local metadata. /// Because of that we need to check whether the replica exists and is newly created. @@ -888,7 +888,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada replica_path + "/min_unprocessed_insert_time", replica_path + "/max_processed_insert_time", replica_path + "/mutation_pointer", - replica_path + "/creator_uuid" + replica_path + "/creator_info" }; auto response_exists = zookeeper->tryGet(paths_exists); @@ -919,7 +919,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada const auto & zk_min_unprocessed_insert_time = response_exists[response_num++].data; const auto & zk_max_processed_insert_time = response_exists[response_num++].data; const auto & zk_mutation_pointer = response_exists[response_num++].data; - const auto & zk_creator_uuid = response_exists[response_num++].data; + const auto & zk_creator_info = response_exists[response_num++].data; if (zk_host.empty() && zk_log_pointer.empty() && @@ -933,7 +933,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada zk_min_unprocessed_insert_time.empty() && zk_max_processed_insert_time.empty() && zk_mutation_pointer.empty() && - zk_creator_uuid == creator_uuid) + zk_creator_info == creator_info) { LOG_DEBUG(log, "Empty replica {} exists, will use it", replica_path); return; @@ -986,7 +986,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_uuid", creator_uuid, + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_info", creator_info, zkutil::CreateMode::Persistent)); /// Check version of /replicas to see if there are any replicas created at the same moment of time. diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh index 0c5705c7a35..34bddc04be1 100755 --- a/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh @@ -11,12 +11,12 @@ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC" #### 1 - There is only one replica -${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected" -# We will see that the replica is empty and throw the same ABORT exception as before -${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" +# We will see that the replica is empty and throw the same 'Fault injected' exception as before +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected" # We will succeed ${CLICKHOUSE_CLIENT} \ @@ -26,10 +26,10 @@ ${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC" #### 2 - There are two replicas -${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" -${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r2') ORDER BY date" 2>&1 | grep -c "ABORT" +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected" +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \ + -q "CREATE TABLE test_exception_replicated_2 (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r2') ORDER BY date" 2>&1 | grep -c "Fault injected" # We will succeed ${CLICKHOUSE_CLIENT} \ From c899ff9da2a8d0b22eba33471cd9bbb183021e73 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 2 Nov 2023 10:30:14 +0800 Subject: [PATCH 116/813] Apply suggestions from code review Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 9501efbabb7..542d13d9363 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -779,9 +779,7 @@ static const orc::Type * traverseDownORCTypeByName( if (orc::STRUCT == orc_type->getKind()) { - auto next_type_and_target = search_struct_field(target, orc_type); - const auto & next_target = next_type_and_target.first; - const auto * next_orc_type = next_type_and_target.second; + const auto [next_target, next_orc_type]= search_struct_field(target, orc_type); return next_orc_type ? traverseDownORCTypeByName(next_target, next_orc_type, type, ignore_case) : nullptr; } else if (orc::LIST == orc_type->getKind()) From 424f1bec933c580d2dfa2101f58fd26004599139 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 2 Nov 2023 10:30:28 +0800 Subject: [PATCH 117/813] Apply suggestions from code review Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 542d13d9363..1cbb3b07c00 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -800,14 +800,9 @@ static const orc::Type * traverseDownORCTypeByName( type = array_type->getNestedType(); return traverseDownORCTypeByName(next_target, next_orc_type, type, ignore_case); } - else - return nullptr; } - else - return nullptr; } - else - return nullptr; + return nullptr; } static void updateIncludeTypeIds( From b142489c3c011a58e547bc4102fa79d452f4e712 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Nov 2023 10:49:18 +0800 Subject: [PATCH 118/813] fix code style --- tests/performance/orc_filter_push_down.xml | 2 +- tests/performance/orc_tuple_field_prune.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/orc_filter_push_down.xml b/tests/performance/orc_filter_push_down.xml index 9f49c20a075..318c6eca991 100644 --- a/tests/performance/orc_filter_push_down.xml +++ b/tests/performance/orc_filter_push_down.xml @@ -23,4 +23,4 @@ select a % 10, length(b) % 10, count(1) from test_orc_fpd where a in (9000000, 1000) group by a % 10, length(b) % 10 - \ No newline at end of file + diff --git a/tests/performance/orc_tuple_field_prune.xml b/tests/performance/orc_tuple_field_prune.xml index 4e338733329..b3064f35f39 100644 --- a/tests/performance/orc_tuple_field_prune.xml +++ b/tests/performance/orc_tuple_field_prune.xml @@ -14,4 +14,4 @@ select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(c Nullable(Int64))') format Null select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(c Nullable(Int64)))') format Null select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(c Nullable(Int64)))') format Null - \ No newline at end of file + From 6ccde98943b2746d2acb3f0cafb77163f56f089d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Nov 2023 11:01:50 +0800 Subject: [PATCH 119/813] change as request --- .../Impl/NativeORCBlockInputFormat.cpp | 40 ++++++++----------- 1 file changed, 17 insertions(+), 23 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 1cbb3b07c00..c53d8c92b2a 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -720,12 +720,8 @@ static std::string toDotColumnPath(const std::vector & columns) } } -static void getFileReaderAndSchema( - ReadBuffer & in, - std::unique_ptr & file_reader, - Block & header, - const FormatSettings & format_settings, - std::atomic & is_stopped) +static void getFileReader( + ReadBuffer & in, std::unique_ptr & file_reader, const FormatSettings & format_settings, std::atomic & is_stopped) { if (is_stopped) return; @@ -733,18 +729,6 @@ static void getFileReaderAndSchema( orc::ReaderOptions options; auto input_stream = asORCInputStream(in, format_settings, is_stopped); file_reader = orc::createReader(std::move(input_stream), options); - const auto & schema = file_reader->getType(); - - for (size_t i = 0; i < schema.getSubtypeCount(); ++i) - { - const std::string & name = schema.getFieldName(i); - const orc::Type * orc_type = schema.getSubtype(i); - - bool skipped = false; - DataTypePtr type = parseORCType(orc_type, format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference, skipped); - if (!skipped) - header.insert(ColumnWithTypeAndName{type, name}); - } } static const orc::Type * traverseDownORCTypeByName( @@ -884,8 +868,7 @@ NativeORCBlockInputFormat::NativeORCBlockInputFormat(ReadBuffer & in_, Block hea void NativeORCBlockInputFormat::prepareFileReader() { - Block schema; - getFileReaderAndSchema(*in, file_reader, schema, format_settings, is_stopped); + getFileReader(*in, file_reader, format_settings, is_stopped); if (is_stopped) return; @@ -1037,17 +1020,28 @@ NativeORCSchemaReader::NativeORCSchemaReader(ReadBuffer & in_, const FormatSetti NamesAndTypesList NativeORCSchemaReader::readSchema() { - Block header; std::unique_ptr file_reader; std::atomic is_stopped = 0; - getFileReaderAndSchema(in, file_reader, header, format_settings, is_stopped); + getFileReader(in, file_reader, format_settings, is_stopped); + + const auto & schema = file_reader->getType(); + Block header; + for (size_t i = 0; i < schema.getSubtypeCount(); ++i) + { + const std::string & name = schema.getFieldName(i); + const orc::Type * orc_type = schema.getSubtype(i); + + bool skipped = false; + DataTypePtr type = parseORCType(orc_type, format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference, skipped); + if (!skipped) + header.insert(ColumnWithTypeAndName{type, name}); + } if (format_settings.schema_inference_make_columns_nullable) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } - ORCColumnToCHColumn::ORCColumnToCHColumn( const Block & header_, bool allow_missing_columns_, bool null_as_default_, bool case_insensitive_matching_) : header(header_) From 59d11559e102dde49840eb95d9ed7d3470647b56 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 2 Nov 2023 04:42:33 +0000 Subject: [PATCH 120/813] Fix tests --- .../0_stateless/02221_system_zookeeper_unrestricted.reference | 2 ++ .../02221_system_zookeeper_unrestricted_like.reference | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index 53b44764d5c..137fb0587cc 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -16,6 +16,8 @@ columns columns columns columns +creator_info +creator_info failed_parts failed_parts flags diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index ccc3064ccbd..2893c2a845f 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -7,6 +7,7 @@ block_numbers blocks columns columns +creator_info failed_parts flags host @@ -49,6 +50,7 @@ block_numbers blocks columns columns +creator_info failed_parts flags host From b8665a610cb7d8920a24a1e3753c785bf42f46a3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Nov 2023 15:27:48 +0800 Subject: [PATCH 121/813] fix failed perf test --- tests/performance/orc_tuple_field_prune.xml | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/performance/orc_tuple_field_prune.xml b/tests/performance/orc_tuple_field_prune.xml index b3064f35f39..d95787af93b 100644 --- a/tests/performance/orc_tuple_field_prune.xml +++ b/tests/performance/orc_tuple_field_prune.xml @@ -2,16 +2,15 @@ 1 10000 - 0 - insert into function file('test_orc_tfp.orc', 'ORC') select * from generateRandom('int64_column Nullable(Int64), tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 + insert into function file('test_orc_tfp.orc', 'ORC') select * from generateRandom('tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Nested(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 DROP TABLE IF EXISTS test_orc_tfp - select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(c Nullable(Int64))') format Null - select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(c Nullable(Int64)))') format Null - select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(c Nullable(Int64)))') format Null + select * from file('test_orc_tfp.orc', 'ORC', 'tuple_column Tuple(c Nullable(Int64))') format Null + select * from file('test_orc_tfp.orc', 'ORC', 'array_tuple_column Nested(c Nullable(Int64))') format Null + select * from file('test_orc_tfp.orc', 'ORC', 'map_tuple_column Map(String, Tuple(c Nullable(Int64)))') format Null From b88048e6d9a5ce72475433ac709a01a4ae515cec Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 2 Nov 2023 16:51:50 +0000 Subject: [PATCH 122/813] Fix style --- ...e.reference => 02888_replicated_merge_tree_creation.reference} | 0 ...reation_failure.sh => 02888_replicated_merge_tree_creation.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02888_replicated_merge_tree_creation_failure.reference => 02888_replicated_merge_tree_creation.reference} (100%) rename tests/queries/0_stateless/{02888_replicated_merge_tree_creation_failure.sh => 02888_replicated_merge_tree_creation.sh} (100%) diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.reference similarity index 100% rename from tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference rename to tests/queries/0_stateless/02888_replicated_merge_tree_creation.reference diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh similarity index 100% rename from tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh rename to tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh From e1ec380339a34529c575e74b991f49a411d0bbb2 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 2 Nov 2023 17:39:41 +0000 Subject: [PATCH 123/813] Add test --- .../02910_bad_logs_level_in_local.reference | 0 .../0_stateless/02910_bad_logs_level_in_local.sh | 15 +++++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02910_bad_logs_level_in_local.reference create mode 100755 tests/queries/0_stateless/02910_bad_logs_level_in_local.sh diff --git a/tests/queries/0_stateless/02910_bad_logs_level_in_local.reference b/tests/queries/0_stateless/02910_bad_logs_level_in_local.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh b/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh new file mode 100755 index 00000000000..badf7232a95 --- /dev/null +++ b/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh @@ -0,0 +1,15 @@ +#!/usr/bin/expect -f + +log_user 0 +set timeout 60 +match_max 100000 + +spawn bash -c "clickhouse-local" + +expect ":) " +send -- "SET send_logs_level = 't'\r" +expect "Exception on client:" +expect ":) " +send -- "exit\r" +expect eof + From 1078047fb59aff24306c26952026278863833721 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 24 Oct 2023 13:44:00 +0000 Subject: [PATCH 124/813] Fix 'Cannot read from file:' while running client in a background --- src/Client/ClientBase.cpp | 22 +++++++++++++++++-- .../02903_client_insert_in_background.sh | 13 +++++++++++ 2 files changed, 33 insertions(+), 2 deletions(-) create mode 100755 tests/queries/0_stateless/02903_client_insert_in_background.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9c7bfe5974f..76f2060da93 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -108,6 +108,7 @@ namespace ErrorCodes extern const int FILE_ALREADY_EXISTS; extern const int USER_SESSION_LIMIT_EXCEEDED; extern const int NOT_IMPLEMENTED; + extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; } } @@ -1443,6 +1444,23 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars } } +namespace +{ + bool isStdinNotEmptyAndValid(ReadBufferFromFileDescriptor & std_in) + { + try + { + return !std_in.eof(); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR) + return false; + throw; + } + } +} + void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query) { @@ -1460,7 +1478,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des if (!parsed_insert_query) return; - bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && !std_in.eof(); + bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && isStdinNotEmptyAndValid(std_in); if (need_render_progress) { @@ -1851,7 +1869,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (is_async_insert_with_inlined_data) { - bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && !std_in.eof(); + bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && isStdinNotEmptyAndValid(std_in); bool have_external_data = have_data_in_stdin || insert->infile; if (have_external_data) diff --git a/tests/queries/0_stateless/02903_client_insert_in_background.sh b/tests/queries/0_stateless/02903_client_insert_in_background.sh new file mode 100755 index 00000000000..d5fc56752f6 --- /dev/null +++ b/tests/queries/0_stateless/02903_client_insert_in_background.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test (x UInt64) engine=Memory" +nohup $CLICKHOUSE_CLIENT -q "insert into test values (42)" 2> $CLICKHOUSE_TEST_UNIQUE_NAME.out +tail -n +2 $CLICKHOUSE_TEST_UNIQUE_NAME.out +$CLICKHOUSE_CLIENT -q "drop table test" +rm $CLICKHOUSE_TEST_UNIQUE_NAME.out + From b306fdb11d01d3fd1d4c38d5a8facafb77052ecf Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Oct 2023 10:25:58 +0000 Subject: [PATCH 125/813] Add refernce file --- .../0_stateless/02903_client_insert_in_background.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02903_client_insert_in_background.reference diff --git a/tests/queries/0_stateless/02903_client_insert_in_background.reference b/tests/queries/0_stateless/02903_client_insert_in_background.reference new file mode 100644 index 00000000000..e69de29bb2d From 7c69dee5cd535ae64062207283b41124270ffee1 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 2 Nov 2023 18:05:02 +0000 Subject: [PATCH 126/813] Fix review comment --- src/Client/ClientBase.cpp | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 76f2060da93..d3cb828e8f7 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1385,6 +1385,23 @@ void ClientBase::addMultiquery(std::string_view query, Arguments & common_argume common_arguments.emplace_back(query); } +namespace +{ +bool isStdinNotEmptyAndValid(ReadBufferFromFileDescriptor & std_in) +{ + try + { + return !std_in.eof(); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR) + return false; + throw; + } +} +} + void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr parsed_query) { @@ -1404,7 +1421,7 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars /// Process the query that requires transferring data blocks to the server. const auto & parsed_insert_query = parsed_query->as(); - if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && std_in.eof()))) + if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && !isStdinNotEmptyAndValid(std_in)))) { const auto & settings = global_context->getSettingsRef(); if (settings.throw_if_no_data_to_insert) @@ -1444,23 +1461,6 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars } } -namespace -{ - bool isStdinNotEmptyAndValid(ReadBufferFromFileDescriptor & std_in) - { - try - { - return !std_in.eof(); - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR) - return false; - throw; - } - } -} - void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query) { From 4b7146d47ed2894068ba6972bdf077df1625e525 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 3 Nov 2023 16:03:26 +0800 Subject: [PATCH 127/813] remove useless codes --- .../Impl/NativeORCBlockInputFormat.cpp | 63 ------------------- 1 file changed, 63 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index c53d8c92b2a..49379405c26 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -678,48 +678,6 @@ buildORCSearchArgument(const KeyCondition & key_condition, const Block & header, return builder->build(); } - -static std::string toDotColumnPath(const std::vector & columns) -{ - if (columns.empty()) - return {}; - - std::ostringstream column_stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - std::copy(columns.begin(), columns.end(), std::ostream_iterator(column_stream, ".")); - std::string column_path = column_stream.str(); - return column_path.substr(0, column_path.length() - 1); -} - -[[maybe_unused]] static void buildORCTypeNameIdMap( - const orc::Type * orc_type, - std::vector & columns, - bool ignore_case, - std::map & id_type_map, - std::map & name_id_map) -{ - id_type_map[orc_type->getColumnId()] = orc_type; - if (orc::STRUCT == orc_type->getKind()) - { - for (size_t i = 0; i < orc_type->getSubtypeCount(); ++i) - { - const std::string & field_name = orc_type->getFieldName(i); - columns.push_back(field_name); - auto column_path = toDotColumnPath(columns); - if (ignore_case) - boost::to_lower(column_path); - name_id_map[column_path] = orc_type->getSubtype(i)->getColumnId(); - buildORCTypeNameIdMap(orc_type->getSubtype(i), columns, ignore_case, id_type_map, name_id_map); - columns.pop_back(); - } - } - else - { - // other non-primitive type - for (size_t j = 0; j < orc_type->getSubtypeCount(); ++j) - buildORCTypeNameIdMap(orc_type->getSubtype(j), columns, ignore_case, id_type_map, name_id_map); - } -} - static void getFileReader( ReadBuffer & in, std::unique_ptr & file_reader, const FormatSettings & format_settings, std::atomic & is_stopped) { @@ -737,7 +695,6 @@ static const orc::Type * traverseDownORCTypeByName( DataTypePtr & type, bool ignore_case) { - // std::cout << "target:" << target << ", orc_type:" << orc_type->toString() << ", type:" << type->getName() << std::endl; if (target.empty()) return orc_type; @@ -792,8 +749,6 @@ static const orc::Type * traverseDownORCTypeByName( static void updateIncludeTypeIds( DataTypePtr type, const orc::Type * orc_type, bool ignore_case, std::unordered_set & include_typeids) { - // std::cout << "ch type:" << type->getName() << ", orc_type:" << orc_type->toString() << std::endl; - /// For primitive types, directly append column id into result if (orc_type->getSubtypeCount() == 0) { @@ -894,24 +849,6 @@ void NativeORCBlockInputFormat::prepareFileReader() } include_indices.assign(include_typeids.begin(), include_typeids.end()); - /// Just for Debug - // std::vector tmp; - // std::map id_type_map; - // std::map name_id_map; - // buildORCTypeNameIdMap(&file_schema, tmp, ignore_case, id_type_map, name_id_map); - // std::cout << "just for debug:" << std::endl; - // std::cout << "subtypes:" << file_reader->getType().getSubtypeCount() << std::endl; - // std::cout << "ch output type:" << getPort().getHeader().dumpStructure() << std::endl; - // std::cout << "orc output type:" << file_reader->getType().toString() << std::endl; - // std::cout << "id type map" << std::endl; - // for (const auto & [k, v] : id_type_map) - // std::cout << "id:" << k << ", type:" << v->toString() << std::endl; - // std::cout << "name id map" << std::endl; - // for (const auto & [k, v] : name_id_map) - // std::cout << "name:" << k << ", id:" << v << std::endl; - // for (const auto & x : include_indices) - // std::cout << "choose " << x << std::endl; - if (format_settings.orc.filter_push_down && key_condition && !sarg) { sarg = buildORCSearchArgument(*key_condition, getPort().getHeader(), file_reader->getType(), format_settings); From 15cb8d1b8975464f7f440e116fb99a48ca0c4865 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 3 Nov 2023 13:16:08 +0000 Subject: [PATCH 128/813] Fix use-of-unitialized-value --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index 8cdbf43f78a..ba5c67934e8 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 8cdbf43f78ad02615aef29dc7f9af0dea22a03e4 +Subproject commit ba5c67934e8274d649befcffab56731632dc5253 From 4235c690d40a082139cf13c30670f083e5c1dea1 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 3 Nov 2023 15:14:25 +0000 Subject: [PATCH 129/813] init commit --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 12 +++++++++++- src/Processors/Formats/Impl/NpyRowInputFormat.h | 5 +++++ .../0_stateless/02908_Npy_files_caching.reference | 4 ++++ tests/queries/0_stateless/02908_Npy_files_caching.sh | 12 ++++++++++++ 4 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02908_Npy_files_caching.reference create mode 100755 tests/queries/0_stateless/02908_Npy_files_caching.sh diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index 9acb2909626..bd8b442f180 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -265,6 +265,11 @@ NpyRowInputFormat::NpyRowInputFormat(ReadBuffer & in_, Block header_, Params par nested_type = getNestedType(types[0]); } +size_t NpyRowInputFormat::countRows(size_t max_block_size) +{ + return int(max_block_size) > header.shape[0] ? header.shape[0] : max_block_size; +} + template void NpyRowInputFormat::readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness) { @@ -395,13 +400,18 @@ NpySchemaReader::NpySchemaReader(ReadBuffer & in_) NamesAndTypesList NpySchemaReader::readSchema() { - NumpyHeader header = parseHeader(in); + header = parseHeader(in); DataTypePtr nested_type = getDataTypeFromNumpyType(header.numpy_type); DataTypePtr result_type = createNestedArrayType(nested_type, header.shape.size()); return {{"array", result_type}}; } +std::optional NpySchemaReader::readNumberOrRows() +{ + return header.shape[0]; +} + void registerInputFormatNpy(FormatFactory & factory) { factory.registerInputFormat("Npy", []( diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.h b/src/Processors/Formats/Impl/NpyRowInputFormat.h index ad32bdba3bf..8d228a3593a 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.h +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.h @@ -29,6 +29,9 @@ public: String getName() const override { return "NpyRowInputFormat"; } private: + bool supportsCountRows() const override { return true; } + size_t countRows(size_t max_block_size) override; + void readPrefix() override; bool readRow(MutableColumns & columns, RowReadExtension &) override; void readData(MutableColumns & columns); @@ -59,7 +62,9 @@ public: explicit NpySchemaReader(ReadBuffer & in_); private: + std::optional readNumberOrRows() override; NamesAndTypesList readSchema() override; + NumpyHeader header; }; } diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.reference b/tests/queries/0_stateless/02908_Npy_files_caching.reference new file mode 100644 index 00000000000..75baea619a9 --- /dev/null +++ b/tests/queries/0_stateless/02908_Npy_files_caching.reference @@ -0,0 +1,4 @@ +3 +3 +array Int64 +3 diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.sh b/tests/queries/0_stateless/02908_Npy_files_caching.sh new file mode 100755 index 00000000000..1c27d35b1c8 --- /dev/null +++ b/tests/queries/0_stateless/02908_Npy_files_caching.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 + +$CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=0" +$CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=1" +$CLICKHOUSE_LOCAL -nm -q " +desc file('$CURDIR/data_npy/one_dim.npy'); +select number_of_rows from system.schema_inference_cache where format='Npy'; +" From 6d9b517b194a83c17623b1f018b7c90b863d80a2 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 3 Nov 2023 17:21:37 +0000 Subject: [PATCH 130/813] fix suggestions --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 8 +++++++- src/Processors/Formats/Impl/NpyRowInputFormat.h | 1 + .../queries/0_stateless/02908_Npy_files_caching.reference | 1 + tests/queries/0_stateless/02908_Npy_files_caching.sh | 1 + 4 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index bd8b442f180..7b0c8e275a9 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -267,7 +267,13 @@ NpyRowInputFormat::NpyRowInputFormat(ReadBuffer & in_, Block header_, Params par size_t NpyRowInputFormat::countRows(size_t max_block_size) { - return int(max_block_size) > header.shape[0] ? header.shape[0] : max_block_size; + size_t count; + if (counted_rows + max_block_size <= size_t(header.shape[0])) + count = max_block_size; + else + count = header.shape[0] - counted_rows; + counted_rows += count; + return count; } template diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.h b/src/Processors/Formats/Impl/NpyRowInputFormat.h index 8d228a3593a..faa9dbb2ec3 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.h +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.h @@ -54,6 +54,7 @@ private: DataTypePtr nested_type; NumpyHeader header; + size_t counted_rows = 0; }; class NpySchemaReader : public ISchemaReader diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.reference b/tests/queries/0_stateless/02908_Npy_files_caching.reference index 75baea619a9..6159dd69b57 100644 --- a/tests/queries/0_stateless/02908_Npy_files_caching.reference +++ b/tests/queries/0_stateless/02908_Npy_files_caching.reference @@ -1,4 +1,5 @@ 3 3 +3 array Int64 3 diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.sh b/tests/queries/0_stateless/02908_Npy_files_caching.sh index 1c27d35b1c8..f77351f1180 100755 --- a/tests/queries/0_stateless/02908_Npy_files_caching.sh +++ b/tests/queries/0_stateless/02908_Npy_files_caching.sh @@ -6,6 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=0" $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=1" +$CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy', auto, 'array Int64') settings optimize_count_from_files=1" $CLICKHOUSE_LOCAL -nm -q " desc file('$CURDIR/data_npy/one_dim.npy'); select number_of_rows from system.schema_inference_cache where format='Npy'; From 18484373e36e3b994fefbb31b09e77da67db57ed Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 3 Nov 2023 18:20:04 +0000 Subject: [PATCH 131/813] Make test non-dependable on max_threads --- tests/queries/0_stateless/02884_parallel_window_functions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.sql b/tests/queries/0_stateless/02884_parallel_window_functions.sql index 5e71fadb3ff..383c48e8b39 100644 --- a/tests/queries/0_stateless/02884_parallel_window_functions.sql +++ b/tests/queries/0_stateless/02884_parallel_window_functions.sql @@ -23,7 +23,7 @@ FROM ) GROUP BY nw ORDER BY R DESC -LIMIT 10) where explain ilike '%ScatterByPartitionTransform%'; +LIMIT 10) where explain ilike '%ScatterByPartitionTransform%' SETTINGS max_threads = 4; -- { echoOn } From 4d7b957e8a7cdccc528ffa2d1937aafcb44bd42e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 3 Nov 2023 18:31:56 +0000 Subject: [PATCH 132/813] Make test non-flaky --- .../0_stateless/02884_parallel_window_functions.reference | 6 +++--- .../0_stateless/02884_parallel_window_functions.sql | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.reference b/tests/queries/0_stateless/02884_parallel_window_functions.reference index cab6195b625..bac15838dc2 100644 --- a/tests/queries/0_stateless/02884_parallel_window_functions.reference +++ b/tests/queries/0_stateless/02884_parallel_window_functions.reference @@ -16,7 +16,7 @@ FROM GROUP BY ac, nw ) GROUP BY nw -ORDER BY R DESC +ORDER BY nw ASC, R DESC LIMIT 10; 0 2 0 1 2 0 @@ -36,7 +36,7 @@ FROM GROUP BY ac, nw ) GROUP BY nw -ORDER BY R DESC +ORDER BY nw ASC, R DESC LIMIT 10 SETTINGS max_threads = 1; 0 2 0 @@ -93,7 +93,7 @@ FROM nw ) GROUP BY nw -ORDER BY R DESC +ORDER BY nw ASC, R DESC LIMIT 10; 0 2 0 1 2 0 diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.sql b/tests/queries/0_stateless/02884_parallel_window_functions.sql index 383c48e8b39..3151b42f896 100644 --- a/tests/queries/0_stateless/02884_parallel_window_functions.sql +++ b/tests/queries/0_stateless/02884_parallel_window_functions.sql @@ -22,7 +22,7 @@ FROM GROUP BY ac, nw ) GROUP BY nw -ORDER BY R DESC +ORDER BY nw ASC, R DESC LIMIT 10) where explain ilike '%ScatterByPartitionTransform%' SETTINGS max_threads = 4; -- { echoOn } @@ -42,7 +42,7 @@ FROM GROUP BY ac, nw ) GROUP BY nw -ORDER BY R DESC +ORDER BY nw ASC, R DESC LIMIT 10; SELECT @@ -60,7 +60,7 @@ FROM GROUP BY ac, nw ) GROUP BY nw -ORDER BY R DESC +ORDER BY nw ASC, R DESC LIMIT 10 SETTINGS max_threads = 1; @@ -115,5 +115,5 @@ FROM nw ) GROUP BY nw -ORDER BY R DESC +ORDER BY nw ASC, R DESC LIMIT 10; From 8a6ae6e150271d2a421f7fd3aa95f232d447b582 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 4 Nov 2023 09:41:01 +0800 Subject: [PATCH 133/813] change as request --- .../Impl/NativeORCBlockInputFormat.cpp | 37 ++++++++++++------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 49379405c26..60d43cc049a 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -783,22 +783,31 @@ static void updateIncludeTypeIds( { if (tuple_type->haveExplicitNames()) { - const auto & names = tuple_type->getElementNames(); - for (size_t tuple_i = 0; tuple_i < names.size(); ++tuple_i) + std::unordered_map orc_field_name_to_index; + orc_field_name_to_index.reserve(orc_type->getSubtypeCount()); + for (size_t struct_i = 0; struct_i < orc_type->getSubtypeCount(); ++struct_i) { - const auto & name = names[tuple_i]; - for (size_t struct_i = 0; struct_i < orc_type->getSubtypeCount(); ++struct_i) + String field_name = orc_type->getFieldName(struct_i); + if (ignore_case) + boost::to_lower(field_name); + + orc_field_name_to_index[field_name] = struct_i; + } + + const auto & element_names = tuple_type->getElementNames(); + for (size_t tuple_i = 0; tuple_i < element_names.size(); ++tuple_i) + { + String element_name = element_names[tuple_i]; + if (ignore_case) + boost::to_lower(element_name); + + if (orc_field_name_to_index.contains(element_name)) { - if (boost::equals(orc_type->getFieldName(struct_i), name) - || (ignore_case && boost::iequals(orc_type->getFieldName(struct_i), name))) - { - updateIncludeTypeIds( - tuple_type->getElement(tuple_i), - orc_type->getSubtype(struct_i), - ignore_case, - include_typeids); - break; - } + updateIncludeTypeIds( + tuple_type->getElement(tuple_i), + orc_type->getSubtype(orc_field_name_to_index[element_name]), + ignore_case, + include_typeids); } } } From 8e23dd909024ba8109b41b8d54d6e355f11ea0ff Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 4 Nov 2023 10:08:26 +0800 Subject: [PATCH 134/813] add some comments --- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 60d43cc049a..3be4b20524f 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -778,6 +778,9 @@ static void updateIncludeTypeIds( return; } case orc::STRUCT: { + /// To make sure tuple field pruning work fine, we should include only the fields of orc struct type which are also contained in CH tuple types, instead of all fields of orc struct type. + /// For example, CH tupe type in header is "x Tuple(a String)", ORC struct type is "x struct", then only type id of field "x.a" should be included. + /// For tuple field pruning purpose, we should never include "x.b" for it is not required in format header. const auto * tuple_type = typeid_cast(non_nullable_type.get()); if (tuple_type) { From 83689c2a04b60288cbeda25d2c57762180273c29 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 4 Nov 2023 14:35:39 +0000 Subject: [PATCH 135/813] Support create and materialized index in the same alter query --- src/Interpreters/InterpreterAlterQuery.cpp | 36 +++++++++++-------- ..._add_index_and_materialize_index.reference | 0 .../02911_add_index_and_materialize_index.sql | 16 +++++++++ 3 files changed, 38 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02911_add_index_and_materialize_index.reference create mode 100644 tests/queries/0_stateless/02911_add_index_and_materialize_index.sql diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index f851607000c..c9a1bd17a46 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -128,10 +128,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) } else if (auto mut_command = MutationCommand::parse(command_ast)) { - if (mut_command->type == MutationCommand::MATERIALIZE_TTL && !metadata_snapshot->hasAnyTTL()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot MATERIALIZE TTL as there is no TTL set for table {}", - table->getStorageID().getNameForLogs()); - if (mut_command->type == MutationCommand::UPDATE || mut_command->type == MutationCommand::DELETE) { /// TODO: add a check for result query size. @@ -162,8 +158,30 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) "to execute ALTERs of different types (replicated and non replicated) in single query"); } + if (!alter_commands.empty()) + { + auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout); + StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); + alter_commands.validate(table, getContext()); + alter_commands.prepare(metadata); + table->checkAlterIsPossible(alter_commands, getContext()); + table->alter(alter_commands, getContext(), alter_lock); + } + + /// Get newest metadata_snapshot after execute ALTER command, in order to + /// support like materialize index in the same ALTER query that creates it. + metadata_snapshot = table->getInMemoryMetadataPtr(); + if (mutation_commands.hasNonEmptyMutationCommands()) { + for (const auto & command : mutation_commands) + { + /// Check it after alter finished, so we can add TTL and materialize TTL in the same ALTER query. + if (command.type == MutationCommand::MATERIALIZE_TTL && !metadata_snapshot->hasAnyTTL()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot MATERIALIZE TTL as there is no TTL set for table {}", + table->getStorageID().getNameForLogs()); + + } table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); MutationsInterpreter::Settings settings(false); MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate(); @@ -178,16 +196,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) res.pipeline = QueryPipeline(std::move(partition_commands_pipe)); } - if (!alter_commands.empty()) - { - auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout); - StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); - alter_commands.validate(table, getContext()); - alter_commands.prepare(metadata); - table->checkAlterIsPossible(alter_commands, getContext()); - table->alter(alter_commands, getContext(), alter_lock); - } - return res; } diff --git a/tests/queries/0_stateless/02911_add_index_and_materialize_index.reference b/tests/queries/0_stateless/02911_add_index_and_materialize_index.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02911_add_index_and_materialize_index.sql b/tests/queries/0_stateless/02911_add_index_and_materialize_index.sql new file mode 100644 index 00000000000..57b144a3a8d --- /dev/null +++ b/tests/queries/0_stateless/02911_add_index_and_materialize_index.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS index_test; + +CREATE TABLE index_test +( + x UInt32, + y UInt32, + z UInt32 +) ENGINE = MergeTree order by x; + +ALTER TABLE index_test + ADD INDEX i_x mortonDecode(2, z).1 TYPE minmax GRANULARITY 1, + ADD INDEX i_y mortonDecode(2, z).2 TYPE minmax GRANULARITY 1, + MATERIALIZE INDEX i_x, + MATERIALIZE INDEX i_y; + +drop table index_test; From d8b44dadd5c28a16e627c4c815996af75c37036e Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 4 Nov 2023 16:16:55 +0000 Subject: [PATCH 136/813] update test --- .../0_stateless/02911_add_index_and_materialize_index.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02911_add_index_and_materialize_index.sql b/tests/queries/0_stateless/02911_add_index_and_materialize_index.sql index 57b144a3a8d..f8785ec9a38 100644 --- a/tests/queries/0_stateless/02911_add_index_and_materialize_index.sql +++ b/tests/queries/0_stateless/02911_add_index_and_materialize_index.sql @@ -1,3 +1,5 @@ +-- Tags: no-replicated-database + DROP TABLE IF EXISTS index_test; CREATE TABLE index_test From f5e439d9dfe32a679f84a6720b461c5a3c5e3a4a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 4 Nov 2023 21:53:05 +0100 Subject: [PATCH 137/813] Add an option to enable or disable coverage collection in clickhouse-test --- tests/clickhouse-test | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 36846a4aeb1..debbb5116da 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1174,7 +1174,7 @@ class TestCase: description_full += result.description - if BuildFlags.SANITIZE_COVERAGE in args.build_flags: + if args.collect_per_test_coverage and BuildFlags.SANITIZE_COVERAGE in args.build_flags: clickhouse_execute( args, f"INSERT INTO system.coverage SELECT now(), '{self.case}', coverage()", @@ -1248,7 +1248,7 @@ class TestCase: ) # We want to calculate per-test code coverage. That's why we reset it before each test. - if BuildFlags.SANITIZE_COVERAGE in args.build_flags: + if args.collect_per_test_coverage and BuildFlags.SANITIZE_COVERAGE in args.build_flags: clickhouse_execute( args, "SYSTEM RESET COVERAGE", @@ -2363,7 +2363,7 @@ def main(args): print(f"Failed to create databases for tests: {e}") server_died.set() - if BuildFlags.SANITIZE_COVERAGE in args.build_flags: + if args.collect_per_test_coverage and BuildFlags.SANITIZE_COVERAGE in args.build_flags: clickhouse_execute( args, """ @@ -2726,6 +2726,12 @@ def parse_args(): default=False, help="Check what high-level server components were covered by tests", ) + parser.add_argument( + "--collect-per-test-coverage", + action="store_true", + default=False, + help="Create `system.coverage` table on the server and collect information about low-level code coverage on a per test basis there", + ) parser.add_argument( "--report-logs-stats", action="store_true", From 501d2106473f7963d10eb37554c42cbab7d5fe4d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 4 Nov 2023 21:04:39 +0000 Subject: [PATCH 138/813] Automatic style fix --- tests/clickhouse-test | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index debbb5116da..a1a270ec033 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1174,7 +1174,10 @@ class TestCase: description_full += result.description - if args.collect_per_test_coverage and BuildFlags.SANITIZE_COVERAGE in args.build_flags: + if ( + args.collect_per_test_coverage + and BuildFlags.SANITIZE_COVERAGE in args.build_flags + ): clickhouse_execute( args, f"INSERT INTO system.coverage SELECT now(), '{self.case}', coverage()", @@ -1248,7 +1251,10 @@ class TestCase: ) # We want to calculate per-test code coverage. That's why we reset it before each test. - if args.collect_per_test_coverage and BuildFlags.SANITIZE_COVERAGE in args.build_flags: + if ( + args.collect_per_test_coverage + and BuildFlags.SANITIZE_COVERAGE in args.build_flags + ): clickhouse_execute( args, "SYSTEM RESET COVERAGE", @@ -2363,7 +2369,10 @@ def main(args): print(f"Failed to create databases for tests: {e}") server_died.set() - if args.collect_per_test_coverage and BuildFlags.SANITIZE_COVERAGE in args.build_flags: + if ( + args.collect_per_test_coverage + and BuildFlags.SANITIZE_COVERAGE in args.build_flags + ): clickhouse_execute( args, """ From cc5179078c3c01bc797732edfabf852befcfaf2f Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 5 Nov 2023 02:29:15 +0000 Subject: [PATCH 139/813] remove unused code Fix --- src/Interpreters/InterpreterAlterQuery.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c9a1bd17a46..54b4334eda9 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -105,7 +105,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (table->isStaticStorage()) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); - auto metadata_snapshot = table->getInMemoryMetadataPtr(); /// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc. AddDefaultDatabaseVisitor visitor(getContext(), table_id.getDatabaseName()); @@ -170,7 +169,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) /// Get newest metadata_snapshot after execute ALTER command, in order to /// support like materialize index in the same ALTER query that creates it. - metadata_snapshot = table->getInMemoryMetadataPtr(); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); if (mutation_commands.hasNonEmptyMutationCommands()) { From 4c21ba7b6fa72555bcfcd6ab77645f275007a4ce Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 12 Sep 2023 06:00:40 +0000 Subject: [PATCH 140/813] tables auto initialize new disks without restart Signed-off-by: Duc Canh Le --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 2 +- src/Disks/StoragePolicy.cpp | 18 +++- src/Disks/StoragePolicy.h | 2 +- src/Interpreters/Context.cpp | 27 ++++-- src/Interpreters/DatabaseCatalog.cpp | 40 ++++++++- src/Interpreters/DatabaseCatalog.h | 14 ++- src/Storages/IStorage.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++ src/Storages/MergeTree/MergeTreeData.h | 2 + src/Storages/StorageDistributed.cpp | 15 ++++ src/Storages/StorageDistributed.h | 2 + .../__init__.py | 0 .../configs/storage_configuration.xml | 21 +++++ .../test_hot_reload_storage_policy/test.py | 86 +++++++++++++++++++ 15 files changed, 237 insertions(+), 17 deletions(-) create mode 100644 tests/integration/test_hot_reload_storage_policy/__init__.py create mode 100644 tests/integration/test_hot_reload_storage_policy/configs/storage_configuration.xml create mode 100644 tests/integration/test_hot_reload_storage_policy/test.py diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 452af7a4ec0..b66b556dbca 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -763,7 +763,7 @@ void LocalServer::processConfig() { DatabaseCatalog::instance().createBackgroundTasks(); loadMetadata(global_context); - DatabaseCatalog::instance().startupBackgroundCleanup(); + DatabaseCatalog::instance().startupBackgroundTasks(); } /// For ClickHouse local if path is not set the loader will be disabled. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 85ae6d7796c..2b98f9e8e78 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1691,7 +1691,7 @@ try /// Then, load remaining databases loadMetadata(global_context, default_database); convertDatabasesEnginesIfNeed(global_context); - database_catalog.startupBackgroundCleanup(); + database_catalog.startupBackgroundTasks(); /// After loading validate that default database exists database_catalog.assertDatabaseExists(default_database); /// Load user-defined SQL functions. diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 33a8719a975..6cf22cbaa1b 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -9,6 +9,7 @@ #include #include +#include #include @@ -429,10 +430,11 @@ StoragePolicySelector::StoragePolicySelector( } -StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks) const +StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks, Strings & new_disks) const { std::shared_ptr result = std::make_shared(config, config_prefix, disks); - + std::set disks_before_reload; + std::set disks_after_reload; /// First pass, check. for (const auto & [name, policy] : policies) { @@ -443,6 +445,8 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage policy {} is missing in new configuration", backQuote(name)); policy->checkCompatibleWith(result->policies[name]); + for (const auto & disk : policy->getDisks()) + disks_before_reload.insert(disk->getName()); } /// Second pass, load. @@ -453,8 +457,18 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti result->policies[name] = policy; else result->policies[name] = std::make_shared(policy, config, config_prefix + "." + name, disks); + + for (const auto & disk : result->policies[name]->getDisks()) + disks_after_reload.insert(disk->getName()); } + std::set_difference( + disks_after_reload.begin(), + disks_after_reload.end(), + disks_before_reload.begin(), + disks_before_reload.end(), + std::back_inserter(new_disks)); + return result; } diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index 69cfb830818..d210d8c1e2f 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -122,7 +122,7 @@ public: StoragePolicySelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks); - StoragePolicySelectorPtr updateFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks) const; + StoragePolicySelectorPtr updateFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks, Strings & new_disks) const; /// Policy by name StoragePolicyPtr get(const String & name) const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 138741a2f2b..cbb99a693af 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3768,23 +3768,33 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration { { std::lock_guard lock(shared->storage_policies_mutex); + Strings disks_to_reinit; if (shared->merge_tree_disk_selector) shared->merge_tree_disk_selector = shared->merge_tree_disk_selector->updateFromConfig(config, "storage_configuration.disks", shared_from_this()); if (shared->merge_tree_storage_policy_selector) { - try + if (shared->merge_tree_storage_policy_selector) { - shared->merge_tree_storage_policy_selector = shared->merge_tree_storage_policy_selector->updateFromConfig( - config, "storage_configuration.policies", shared->merge_tree_disk_selector); - } - catch (Exception & e) - { - LOG_ERROR( - shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); + try + { + shared->merge_tree_storage_policy_selector = shared->merge_tree_storage_policy_selector->updateFromConfig( + config, "storage_configuration.policies", shared->merge_tree_disk_selector, disks_to_reinit); + } + catch (Exception & e) + { + LOG_ERROR( + shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); + } } } + + if (!disks_to_reinit.empty()) + { + LOG_DEBUG(shared->log, "Reloading disks: {}", fmt::join(disks_to_reinit, ", ")); + DatabaseCatalog::instance().triggerReloadDisksTask(disks_to_reinit); + } } { @@ -3792,6 +3802,7 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration if (shared->storage_s3_settings) shared->storage_s3_settings->loadFromConfig("s3", config, getSettingsRef()); } + } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 2e82ff464d2..50b0e5d4d1e 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -200,11 +201,14 @@ void DatabaseCatalog::createBackgroundTasks() cleanup_task = std::make_unique(std::move(cleanup_task_holder)); } - auto task_holder = getContext()->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->dropTableDataTask(); }); - drop_task = std::make_unique(std::move(task_holder)); + auto drop_task_holder = getContext()->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->dropTableDataTask(); }); + drop_task = std::make_unique(std::move(drop_task_holder)); + + auto reload_disks_task_holder = getContext()->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->reloadDisksTask(); }); + reload_disks_task = std::make_unique(std::move(reload_disks_task_holder)); } -void DatabaseCatalog::startupBackgroundCleanup() +void DatabaseCatalog::startupBackgroundTasks() { /// And it has to be done after all databases are loaded, otherwise cleanup_task may remove something that should not be removed if (cleanup_task) @@ -1576,6 +1580,30 @@ bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskP } } +void DatabaseCatalog::reloadDisksTask() +{ + std::set disks; + { + std::lock_guard lock{reload_disks_mutex}; + disks.swap(disks_to_reload); + } + + for (auto & database : getDatabases()) + { + auto it = database.second->getTablesIterator(getContext()); + while (it->isValid()) + { + auto table = it->table(); + table->initializeDiskOnConfigChange(disks); + it->next(); + } + } + + std::lock_guard lock{reload_disks_mutex}; + if (!disks_to_reload.empty()) /// during reload, another disks configuration change + (*reload_disks_task)->scheduleAfter(default_reload_period_sec * 1000); +} + static void maybeUnlockUUID(UUID uuid) { if (uuid == UUIDHelpers::Nil) @@ -1659,4 +1687,10 @@ DDLGuard::~DDLGuard() releaseTableLock(); } +void DatabaseCatalog::triggerReloadDisksTask(const Strings & new_added_disks) +{ + std::lock_guard lock{reload_disks_mutex}; + disks_to_reload.insert(new_added_disks.begin(), new_added_disks.end()); + (*reload_disks_task)->schedule(); +} } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index edf1036b438..8e81358e589 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -166,7 +166,7 @@ public: void createBackgroundTasks(); void initializeAndLoadTemporaryDatabase(); - void startupBackgroundCleanup(); + void startupBackgroundTasks(); void loadMarkedAsDroppedTables(); /// Get an object that protects the table from concurrently executing multiple DDL operations. @@ -286,6 +286,9 @@ public: std::lock_guard lock(tables_marked_dropped_mutex); return tables_marked_dropped; } + + void triggerReloadDisksTask(const Strings & new_added_disks); + private: // The global instance of database catalog. unique_ptr is to allow // deferred initialization. Thought I'd use std::optional, but I can't @@ -319,6 +322,8 @@ private: void cleanupStoreDirectoryTask(); bool maybeRemoveDirectory(const String & disk_name, const DiskPtr & disk, const String & unused_dir); + void reloadDisksTask(); + static constexpr size_t reschedule_time_ms = 100; mutable std::mutex databases_mutex; @@ -380,8 +385,15 @@ private: static constexpr time_t default_drop_error_cooldown_sec = 5; time_t drop_error_cooldown_sec = default_drop_error_cooldown_sec; + + std::unique_ptr reload_disks_task; + std::mutex reload_disks_mutex; + std::set disks_to_reload; + static constexpr time_t default_reload_period_sec = 5; }; + + /// This class is useful when creating a table or database. /// Usually we create IStorage/IDatabase object first and then add it to IDatabase/DatabaseCatalog. /// But such object may start using a directory in store/ since its creation. diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 803ab5e92ba..1d6672fc7c6 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -714,6 +714,9 @@ public: return getStorageSnapshot(metadata_snapshot, query_context); } + /// Re initialize disks in case the underlying storage policy changed + virtual bool initializeDiskOnConfigChange(const std::set & /*new_added_disks*/) { return true; } + /// A helper to implement read() static void readFromPipe( QueryPlan & query_plan, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index aa0b6b2ff37..118a971a71e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8191,4 +8191,24 @@ CurrentlySubmergingEmergingTagger::~CurrentlySubmergingEmergingTagger() storage.currently_emerging_big_parts.erase(emerging_part_name); } +bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_added_disks) +{ + auto storage_policy = getStoragePolicy(); + const auto format_version_path = fs::path(relative_data_path) / MergeTreeData::FORMAT_VERSION_FILE_NAME; + for (const auto & name : new_added_disks) + { + auto disk = storage_policy->tryGetDiskByName(name); + if (disk) + { + disk->createDirectories(relative_data_path); + disk->createDirectories(fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME); + auto buf = disk->writeFile(format_version_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, getContext()->getWriteSettings()); + writeIntText(format_version.toUnderType(), *buf); + buf->finalize(); + if (getContext()->getSettingsRef().fsync_metadata) + buf->sync(); + } + } + return true; +} } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index aab04260b0e..95cc9907236 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1093,6 +1093,8 @@ public: const SelectQueryInfo & query_info, const ActionDAGNodes & added_filter_nodes) const; + bool initializeDiskOnConfigChange(const std::set & /*new_added_disks*/) override; + protected: friend class IMergeTreeDataPart; friend class MergeTreeDataMergerMutator; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 7705d0f193f..b4a9f252554 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1875,4 +1875,19 @@ void registerStorageDistributed(StorageFactory & factory) }); } +bool StorageDistributed::initializeDiskOnConfigChange(const std::set & new_added_disks) +{ + if (!data_volume) + return true; + + for (auto & disk : data_volume->getDisks()) + { + if (new_added_disks.contains(disk->getName())) + { + initializeDirectoryQueuesForDisk(disk); + } + } + + return true; +} } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 520e1445d09..fdf57410af4 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -159,6 +159,8 @@ public: /// Used by ClusterCopier size_t getShardCount() const; + bool initializeDiskOnConfigChange(const std::set & new_added_disks) override; + private: void renameOnDisk(const String & new_path_to_table_data); diff --git a/tests/integration/test_hot_reload_storage_policy/__init__.py b/tests/integration/test_hot_reload_storage_policy/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_hot_reload_storage_policy/configs/storage_configuration.xml b/tests/integration/test_hot_reload_storage_policy/configs/storage_configuration.xml new file mode 100644 index 00000000000..466ecde137d --- /dev/null +++ b/tests/integration/test_hot_reload_storage_policy/configs/storage_configuration.xml @@ -0,0 +1,21 @@ + + + + + /var/lib/clickhouse/disk0/ + + + /var/lib/clickhouse/disk1/ + + + + + + + disk0 + + + + + + \ No newline at end of file diff --git a/tests/integration/test_hot_reload_storage_policy/test.py b/tests/integration/test_hot_reload_storage_policy/test.py new file mode 100644 index 00000000000..8654b0462e4 --- /dev/null +++ b/tests/integration/test_hot_reload_storage_policy/test.py @@ -0,0 +1,86 @@ +import os +import sys +import time + +import pytest + +sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__)))) + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +node0 = cluster.add_instance( + "node0", with_zookeeper=True, main_configs=["configs/storage_configuration.xml"] +) +node1 = cluster.add_instance( + "node1", with_zookeeper=True, main_configs=["configs/storage_configuration.xml"] +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +new_disk_config = """ + + + + + /var/lib/clickhouse/disk0/ + + + /var/lib/clickhouse/disk1/ + + + /var/lib/clickhouse/disk2/ + + + + + + + disk2 + disk1 + disk0 + + + + + + +""" + + +def set_config(node, config): + node.replace_config( + "/etc/clickhouse-server/config.d/storage_configuration.xml", config + ) + node.query("SYSTEM RELOAD CONFIG") + + +def test_hot_reload_policy(started_cluster): + node0.query( + "CREATE TABLE t (d Int32, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/t', '0') PARTITION BY d ORDER BY tuple() SETTINGS storage_policy = 'default_policy'" + ) + node0.query("INSERT INTO TABLE t VALUES (1, 'foo') (1, 'bar')") + + node1.query( + "CREATE TABLE t (d Int32, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/t_mirror', '1') PARTITION BY d ORDER BY tuple() SETTINGS storage_policy = 'default_policy'" + ) + set_config(node1, new_disk_config) + time.sleep(1) + node1.query("ALTER TABLE t FETCH PARTITION 1 FROM '/clickhouse/tables/t'") + result = int(node1.query("SELECT count() FROM t")) + assert ( + result == 4, + "Node should have 2 x full data (4 rows) after reloading storage configuration and fetch new partition, but get {} rows".format( + result + ), + ) From d900f8a352ff263b75e9c9a2df68a7f59b721943 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 6 Nov 2023 15:18:25 +0300 Subject: [PATCH 141/813] merge_row_policy: aliases slightly work --- src/Storages/StorageMerge.cpp | 108 +++++++++++++----- src/Storages/StorageMerge.h | 2 + src/TableFunctions/TableFunctionMerge.cpp | 2 +- .../02763_row_policy_storage_merge.sql.j2 | 32 ++++++ 4 files changed, 117 insertions(+), 27 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 7df8b8cc6c1..8002a5f4b8f 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -403,6 +403,8 @@ private: ExpressionActionsPtr filter_actions; }; +// using RowPolicyDataPtr = std::unique_ptr; + ReadFromMerge::ReadFromMerge( Block common_header_, StorageListWithLocks selected_tables_, @@ -492,6 +494,22 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu if (sampling_requested && !storage->supportsSampling()) throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: table doesn't support sampling"); + const auto & [database_name, _, _b, table_name] = table; + + std::unique_ptr row_policy_data_ptr; + + auto row_policy_filter_ptr = context->getRowPolicyFilter( + database_name, + table_name, + RowPolicyFilterType::SELECT_FILTER); + if (row_policy_filter_ptr) + { + row_policy_data_ptr = std::make_unique(row_policy_filter_ptr, storage, context); + row_policy_data_ptr->extendNames(column_names); + } + + + Aliases aliases; auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, context); @@ -529,6 +547,11 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu auto type = sample_block.getByName(column).type; aliases.push_back({ .name = column, .type = type, .expression = column_expr->clone() }); + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), + "adding new alias name {}, expression {}", + column, column_expr->formatForLogging()); + } else column_expr = std::make_shared(column); @@ -542,6 +565,12 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), + "alias_actions->getRequiredColumns: {}", alias_actions->getRequiredColumns().toString()); + + // if (row_policy_data_ptr) + // row_policy_data_ptr->extendNames(column_names_as_aliases); + if (column_names_as_aliases.empty()) column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); } @@ -556,6 +585,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu aliases, table, column_names_as_aliases.empty() ? column_names : column_names_as_aliases, + std::move(row_policy_data_ptr), context, current_streams); @@ -657,6 +687,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( const Aliases & aliases, const StorageWithLockAndName & storage_with_lock, Names real_column_names, + std::unique_ptr row_policy_data_ptr, ContextMutablePtr modified_context, size_t streams_num, bool concat_streams) @@ -679,7 +710,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage_snapshot, modified_query_info); - std::optional row_policy_data; + // std::optional row_policy_data; if (processed_stage <= storage_stage || (allow_experimental_analyzer && processed_stage == QueryProcessingStage::FetchColumns)) { @@ -692,15 +723,15 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( StorageView * view = dynamic_cast(storage.get()); if (!view || allow_experimental_analyzer) { - auto row_policy_filter_ptr = modified_context->getRowPolicyFilter( - database_name, - table_name, - RowPolicyFilterType::SELECT_FILTER); - if (row_policy_filter_ptr) - { - row_policy_data.emplace(row_policy_filter_ptr, storage, modified_context); - row_policy_data->extendNames(real_column_names); - } + // auto row_policy_filter_ptr = modified_context->getRowPolicyFilter( + // database_name, + // table_name, + // RowPolicyFilterType::SELECT_FILTER); + // if (row_policy_data_ptr) + // { + // row_policy_data.emplace(row_policy_filter_ptr, storage, modified_context); + // row_policy_data_ptr->extendNames(real_column_names); + // } storage->read(plan, real_column_names, @@ -732,11 +763,11 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (!plan.isInitialized()) return {}; - if (row_policy_data) + if (row_policy_data_ptr) { if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) { - row_policy_data->addStorageFilter(source_step_with_filter); + row_policy_data_ptr->addStorageFilter(source_step_with_filter); } } @@ -833,14 +864,15 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( }); } - if (row_policy_data) - { - row_policy_data->addFilterTransform(*builder); - } + // if (row_policy_data_ptr) + // { + // row_policy_data_ptr->addFilterTransform(*builder); + // } /// 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. - convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, *builder, processed_stage); + convertingSourceStream(header, storage_snapshot->metadata, aliases, std::move(row_policy_data_ptr), modified_context, *builder, processed_stage); + } return builder; @@ -851,15 +883,16 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter ContextPtr local_context) : row_policy_filter_ptr(row_policy_filter_ptr_) { - assert(row_policy_filter_ptr_); + auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto storage_columns = storage_metadata_snapshot->getColumns(); + auto needed_columns = storage_columns.getAll/*Physical*/(); ASTPtr expr = row_policy_filter_ptr->expression; - auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto storage_columns = storage_metadata_snapshot->getColumns(); - auto needed_columns = storage_columns.getAllPhysical(); - - auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns); + auto syntax_result = TreeRewriter(local_context).analyze(expr, + needed_columns /*, + storage, + storage->getStorageSnapshot(storage_metadata_snapshot, local_context)*/); auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); @@ -894,12 +927,18 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) if (!added_names.empty()) { std::copy(added_names.begin(), added_names.end(), std::back_inserter(names)); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), + "{} names added", added_names.size()); + // for (const auto & added_name : added_names) + // { + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), + // " added name {}", added_name); + // } } } void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) { - assert(row_policy_filter_ptr); LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::addStorageFilter"), "filter_actions_dag: {},<> {}, <> {}", filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), @@ -910,8 +949,6 @@ void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & builder) { - assert(row_policy_filter_ptr); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::addFilterTransform"), "filter_actions_dag: {},<> {}, <> {}", filter_actions->getActionsDAG().dumpNames(), filter_actions->getActionsDAG().dumpDAG(), @@ -919,6 +956,10 @@ void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & bui builder.addSimpleTransform([&](const Block & stream_header) { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::addFilterTransform"), + "stream_header.dumpNames {}", stream_header.dumpNames()); + + return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove filter column */); }); } @@ -1097,6 +1138,7 @@ void ReadFromMerge::convertingSourceStream( const Block & header, const StorageMetadataPtr & metadata_snapshot, const Aliases & aliases, + std::unique_ptr row_policy_data_ptr, ContextPtr local_context, QueryPipelineBuilder & builder, QueryProcessingStage::Enum processed_stage) @@ -1123,11 +1165,24 @@ void ReadFromMerge::convertingSourceStream( }); } + + ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns) convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), + "builder.getHeader(): {}, header.getColumnsWithTypeAndName: {}", + builder.getHeader().dumpStructure(), + header.dumpStructure()); + + + if (row_policy_data_ptr) + { + row_policy_data_ptr->addFilterTransform(builder); + } + auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); @@ -1139,6 +1194,7 @@ void ReadFromMerge::convertingSourceStream( { return std::make_shared(stream_header, actions); }); + } bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 04c5a70651f..80c5790d319 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -193,6 +193,7 @@ private: const Aliases & aliases, const StorageWithLockAndName & storage_with_lock, Names real_column_names, + std::unique_ptr row_policy_data_ptr, ContextMutablePtr modified_context, size_t streams_num, bool concat_streams = false); @@ -201,6 +202,7 @@ private: const Block & header, const StorageMetadataPtr & metadata_snapshot, const Aliases & aliases, + std::unique_ptr row_policy_data_ptr, ContextPtr context, QueryPipelineBuilder & builder, QueryProcessingStage::Enum processed_stage); diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 599953a1add..5231c651f17 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -126,7 +126,7 @@ ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr contex { auto storage = DatabaseCatalog::instance().tryGetTable(StorageID{db_with_tables.first, table}, context); if (storage) - return ColumnsDescription{storage->getInMemoryMetadataPtr()->getColumns().getAllPhysical()}; + return ColumnsDescription{storage->getInMemoryMetadataPtr()->getColumns().getAllPhysical()}; // !!! } } diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index 9b3197cc34f..0bc1292d4a5 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -3,6 +3,7 @@ DROP TABLE IF EXISTS 02763_merge_log_2; DROP TABLE IF EXISTS 02763_merge_merge_1; DROP TABLE IF EXISTS 02763_merge_merge_2; DROP TABLE IF EXISTS 02763_merge_fancycols; +DROP TABLE IF EXISTS 02763_merge_aliases; DROP ROW POLICY IF EXISTS 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY IF EXISTS 02763_filter_2 ON 02763_merge_merge_1; DROP ROW POLICY IF EXISTS 02763_filter_3 ON 02763_merge_log_1; @@ -17,6 +18,8 @@ CREATE TABLE 02763_merge_log_2 (x UInt8, y UInt64) ENGINE = Log; CREATE TABLE 02763_merge_merge_1 (x UInt8, y UInt64) ENGINE = MergeTree ORDER BY x; CREATE TABLE 02763_merge_merge_2 (x UInt8, y UInt64) ENGINE = MergeTree ORDER BY x; +CREATE TABLE 02763_engine_merge_12 (x UInt8, y UInt64) ENGINE = Merge(currentDatabase(), '02763_merge_merge'); + INSERT INTO 02763_merge_log_1 VALUES (1, 11), (2, 12), (3, 13), (4, 14); INSERT INTO 02763_merge_log_2 VALUES (1, 11), (2, 12), (3, 13), (4, 14); INSERT INTO 02763_merge_merge_1 VALUES (1, 11), (2, 12), (3, 13), (4, 14); @@ -56,6 +59,10 @@ SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2'; SELECT * FROM merge(currentDatabase(), '02763_merge_merge') WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT * FROM engine_merge_12 WHERE x>2'; +SELECT * FROM 02763_engine_merge_12 WHERE x>2 ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + + SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge)'; SELECT * FROM merge(currentDatabase(), '02763_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge) WHERE x>2'; @@ -125,6 +132,29 @@ SELECT x, y from merge(currentDatabase(), '02763_merge') ORDER BY x; DROP TABLE 02763_merge_fancycols; +SELECT '02763_merge_aliases'; +CREATE TABLE 02763_alias (x UInt8, y UInt64, z UInt64 ALIAS plus(x,y)) ENGINE = MergeTree ORDER BY x; +INSERT INTO 02763_alias VALUES (1, 11), (2, 12), (3, 13), (4, 14); + +CREATE ROW POLICY 02763_filter_7 ON 02763_alias USING z>15 AS permissive TO ALL; + +-- SELECT 'SELECT * FROM 02763_alias ORDER BY x'; +-- SELECT x, y, z FROM 02763_alias ORDER BY x; + +CREATE TABLE 02763_a_merge (x UInt8, y UInt64, z UInt64) ENGINE = Merge(currentDatabase(), '02763_alias'); + + +-- SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_alias) ORDER BY x'; +-- SELECT * FROM merge(currentDatabase(), '02763_merge_alias') ORDER BY x; + +SELECT 'x, y, z FROM 02763_a_merge ORDER BY x'; +SELECT x, y, z FROM 02763_a_merge ORDER BY x; +-- SELECT 'SELECT x, y, z FROM merge(currentDatabase(), 02763_merge_alias) ORDER BY x'; +-- SELECT x, y, z FROM merge(currentDatabase(), '02763_merge_alias') ORDER BY x; + +DROP TABLE 02763_alias; +DROP TABLE 02763_a_merge; + DROP ROW POLICY 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY 02763_filter_2 ON 02763_merge_merge_1; @@ -134,4 +164,6 @@ DROP ROW POLICY 02763_filter_4 ON 02763_merge_merge_1; DROP ROW POLICY 02763_filter_5 ON 02763_merge_fancycols; DROP ROW POLICY 02763_filter_6 ON 02763_merge_fancycols; +DROP ROW POLICY 02763_filter_7 ON 02763_alias; + {% endfor %} From a52e22a8580f3d780a7271b5acc672b01b8ca896 Mon Sep 17 00:00:00 2001 From: edef Date: Mon, 30 Oct 2023 08:08:10 +0000 Subject: [PATCH 142/813] Support reading arrow::LargeListArray --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 33 +++++++++++++++---- 1 file changed, 26 insertions(+), 7 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index cb24554f9f4..50e9ece0399 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -436,6 +436,22 @@ static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr return nullmap_column; } +template +struct ArrowOffsetArray; + +template <> +struct ArrowOffsetArray +{ + using type = arrow::Int32Array; +}; + +template <> +struct ArrowOffsetArray +{ + using type = arrow::Int64Array; +}; + +template static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) { auto offsets_column = ColumnUInt64::create(); @@ -444,9 +460,9 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptrnum_chunks(); chunk_i < num_chunks; ++chunk_i) { - arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + ArrowListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); auto arrow_offsets_array = list_chunk.offsets(); - auto & arrow_offsets = dynamic_cast(*arrow_offsets_array); + auto & arrow_offsets = dynamic_cast::type &>(*arrow_offsets_array); /* * CH uses element size as "offsets", while arrow uses actual offsets as offsets. @@ -602,13 +618,14 @@ static ColumnPtr readColumnWithIndexesData(std::shared_ptr } } +template static std::shared_ptr getNestedArrowColumn(std::shared_ptr & arrow_column) { arrow::ArrayVector array_vector; array_vector.reserve(arrow_column->num_chunks()); for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { - arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + ArrowListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); /* * It seems like arrow::ListArray::values() (nested column data) might or might not be shared across chunks. @@ -819,12 +836,12 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( key_type_hint = map_type_hint->getKeyType(); } } - auto arrow_nested_column = getNestedArrowColumn(arrow_column); + auto arrow_nested_column = getNestedArrowColumn(arrow_column); auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint, true); if (skipped) return {}; - auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); + auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); const auto * tuple_column = assert_cast(nested_column.column.get()); const auto * tuple_type = assert_cast(nested_column.type.get()); @@ -846,7 +863,9 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( return {std::move(map_column), std::move(map_type), column_name}; } case arrow::Type::LIST: + case arrow::Type::LARGE_LIST: { + bool is_large = arrow_column->type()->id() == arrow::Type::LARGE_LIST; DataTypePtr nested_type_hint; if (type_hint) { @@ -854,11 +873,11 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( if (array_type_hint) nested_type_hint = array_type_hint->getNestedType(); } - auto arrow_nested_column = getNestedArrowColumn(arrow_column); + auto arrow_nested_column = is_large ? getNestedArrowColumn(arrow_column) : getNestedArrowColumn(arrow_column); auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint); if (skipped) return {}; - auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); + auto offsets_column = is_large ? readOffsetsFromArrowListColumn(arrow_column) : readOffsetsFromArrowListColumn(arrow_column); auto array_column = ColumnArray::create(nested_column.column, offsets_column); auto array_type = std::make_shared(nested_column.type); return {std::move(array_column), std::move(array_type), column_name}; From 1fa61300724e5781aecc304a8db2cc4dd30e65e0 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 7 Nov 2023 01:27:53 +0000 Subject: [PATCH 143/813] small refactor Signed-off-by: Duc Canh Le --- src/Interpreters/Context.cpp | 21 +++++++++------------ src/Interpreters/DatabaseCatalog.cpp | 15 ++++++++------- src/Interpreters/DatabaseCatalog.h | 3 +-- 3 files changed, 18 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cbb99a693af..81d5751f17b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3775,24 +3775,21 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration if (shared->merge_tree_storage_policy_selector) { - if (shared->merge_tree_storage_policy_selector) + try { - try - { - shared->merge_tree_storage_policy_selector = shared->merge_tree_storage_policy_selector->updateFromConfig( - config, "storage_configuration.policies", shared->merge_tree_disk_selector, disks_to_reinit); - } - catch (Exception & e) - { - LOG_ERROR( - shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); - } + shared->merge_tree_storage_policy_selector = shared->merge_tree_storage_policy_selector->updateFromConfig( + config, "storage_configuration.policies", shared->merge_tree_disk_selector, disks_to_reinit); + } + catch (Exception & e) + { + LOG_ERROR( + shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); } } if (!disks_to_reinit.empty()) { - LOG_DEBUG(shared->log, "Reloading disks: {}", fmt::join(disks_to_reinit, ", ")); + LOG_INFO(shared->log, "Initializing disks: ({}) for all tables", fmt::join(disks_to_reinit, ", ")); DatabaseCatalog::instance().triggerReloadDisksTask(disks_to_reinit); } } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 50b0e5d4d1e..8dc90e1a2f1 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1601,7 +1601,14 @@ void DatabaseCatalog::reloadDisksTask() std::lock_guard lock{reload_disks_mutex}; if (!disks_to_reload.empty()) /// during reload, another disks configuration change - (*reload_disks_task)->scheduleAfter(default_reload_period_sec * 1000); + (*reload_disks_task)->scheduleAfter(DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC * 1000); +} + +void DatabaseCatalog::triggerReloadDisksTask(const Strings & new_added_disks) +{ + std::lock_guard lock{reload_disks_mutex}; + disks_to_reload.insert(new_added_disks.begin(), new_added_disks.end()); + (*reload_disks_task)->schedule(); } static void maybeUnlockUUID(UUID uuid) @@ -1687,10 +1694,4 @@ DDLGuard::~DDLGuard() releaseTableLock(); } -void DatabaseCatalog::triggerReloadDisksTask(const Strings & new_added_disks) -{ - std::lock_guard lock{reload_disks_mutex}; - disks_to_reload.insert(new_added_disks.begin(), new_added_disks.end()); - (*reload_disks_task)->schedule(); -} } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 8e81358e589..ee5d0b84b23 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -389,11 +389,10 @@ private: std::unique_ptr reload_disks_task; std::mutex reload_disks_mutex; std::set disks_to_reload; - static constexpr time_t default_reload_period_sec = 5; + static constexpr time_t DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC = 5; }; - /// This class is useful when creating a table or database. /// Usually we create IStorage/IDatabase object first and then add it to IDatabase/DatabaseCatalog. /// But such object may start using a directory in store/ since its creation. From e4400ec24c1a7949638f80642f8510978dc6bbea Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 7 Nov 2023 13:33:02 -0300 Subject: [PATCH 144/813] add transition from reading key to reading quoted key when double quotes are found --- src/Functions/keyvaluepair/impl/StateHandlerImpl.h | 5 +++++ ..._extract_key_value_pairs_multiple_input.reference | 12 ++++++++++++ .../02499_extract_key_value_pairs_multiple_input.sql | 12 ++++++++++++ 3 files changed, 29 insertions(+) diff --git a/src/Functions/keyvaluepair/impl/StateHandlerImpl.h b/src/Functions/keyvaluepair/impl/StateHandlerImpl.h index 7fc3ba54833..b4fd91ec3c9 100644 --- a/src/Functions/keyvaluepair/impl/StateHandlerImpl.h +++ b/src/Functions/keyvaluepair/impl/StateHandlerImpl.h @@ -106,6 +106,11 @@ public: { return {next_pos, State::WAITING_KEY}; } + else if (isQuotingCharacter(*p)) + { + // +1 to skip quoting character + return {next_pos, State::READING_QUOTED_KEY}; + } pos = next_pos; } diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference index d0cf9ff680b..f646583bbd3 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference @@ -345,6 +345,18 @@ WITH SELECT x; {'argument1':'1','argument2':'2','char':'=','char2':'=','formula':'1+2=3','result':'3','string':'foo=bar'} +-- https://github.com/ClickHouse/ClickHouse/issues/56357 +WITH + extractKeyValuePairs('{"a":"1", "b":"2"}') as s_map, + CAST( + arrayMap( + (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) + ), + 'Map(String,String)' + ) AS x +SELECT + x; +{'a':'1','b':'2'} -- check str_to_map alias (it is case-insensitive) WITH sTr_tO_mAp('name:neymar, age:31 team:psg,nationality:brazil') AS s_map, diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql index 804ff4ce880..9277ba6d7ec 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql @@ -481,6 +481,18 @@ WITH SELECT x; +-- https://github.com/ClickHouse/ClickHouse/issues/56357 +WITH + extractKeyValuePairs('{"a":"1", "b":"2"}') as s_map, + CAST( + arrayMap( + (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) + ), + 'Map(String,String)' + ) AS x +SELECT + x; + -- check str_to_map alias (it is case-insensitive) WITH sTr_tO_mAp('name:neymar, age:31 team:psg,nationality:brazil') AS s_map, From 28ca29fda24350a954dc0747a822161e74992a44 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 7 Nov 2023 14:31:34 -0300 Subject: [PATCH 145/813] remove stale comment --- src/Functions/keyvaluepair/impl/StateHandlerImpl.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/keyvaluepair/impl/StateHandlerImpl.h b/src/Functions/keyvaluepair/impl/StateHandlerImpl.h index b4fd91ec3c9..687d8d95d42 100644 --- a/src/Functions/keyvaluepair/impl/StateHandlerImpl.h +++ b/src/Functions/keyvaluepair/impl/StateHandlerImpl.h @@ -108,7 +108,6 @@ public: } else if (isQuotingCharacter(*p)) { - // +1 to skip quoting character return {next_pos, State::READING_QUOTED_KEY}; } From 181231d50004c73e87852874850235c5eb983610 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 7 Nov 2023 17:56:02 +0000 Subject: [PATCH 146/813] init --- src/Formats/NumpyDataTypes.h | 2 + .../Formats/Impl/NpyRowInputFormat.cpp | 65 +++++++++++++++--- .../Formats/Impl/NpyRowInputFormat.h | 2 +- .../0_stateless/02895_npy_format.reference | 2 + tests/queries/0_stateless/02895_npy_format.sh | 2 + .../queries/0_stateless/data_npy/float_16.npy | Bin 0 -> 140 bytes 6 files changed, 61 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/data_npy/float_16.npy diff --git a/src/Formats/NumpyDataTypes.h b/src/Formats/NumpyDataTypes.h index 43fd9064dae..712797515c9 100644 --- a/src/Formats/NumpyDataTypes.h +++ b/src/Formats/NumpyDataTypes.h @@ -17,6 +17,7 @@ enum class NumpyDataTypeIndex UInt16, UInt32, UInt64, + Float16, Float32, Float64, String, @@ -79,6 +80,7 @@ public: { switch (size) { + case 2: type_index = NumpyDataTypeIndex::Float16; break; case 4: type_index = NumpyDataTypeIndex::Float32; break; case 8: type_index = NumpyDataTypeIndex::Float64; break; default: diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index 9acb2909626..387269b8974 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -1,20 +1,16 @@ +#include #include -#include #include #include -#include -#include #include #include #include -#include #include #include #include #include #include #include -#include #include #include @@ -34,6 +30,38 @@ namespace ErrorCodes namespace { +float convertFloat16ToFloat32(uint16_t float16_value) { + uint16_t sign = (float16_value >> 15) & 0x1; + uint16_t exponent = (float16_value >> 10) & 0x1F; + uint16_t fraction = float16_value & 0x3FF; + + // Handling special cases for exponent + if (exponent == 0x1F) { + // NaN or Infinity in float16 + return (fraction == 0) ? std::numeric_limits::infinity() : std::numeric_limits::quiet_NaN(); + } + + // Convert exponent from float16 to float32 format + int32_t new_exponent = static_cast(exponent) - 15 + 127; + + // Constructing the float32 representation + uint32_t float32_value = (static_cast(sign) << 31) | + (static_cast(new_exponent) << 23) | + (static_cast(fraction) << 13); + + // Interpret the binary representation as a float + float result; + std::memcpy(&result, &float32_value, sizeof(float)); + + // Determine decimal places dynamically based on the magnitude of the number + int decimal_places = std::max(0, 6 - static_cast(std::log10(std::abs(result)))); + // Truncate the decimal part to the determined number of decimal places + float multiplier = static_cast(std::pow(10.0f, decimal_places)); + result = std::round(result * multiplier) / multiplier; + + return result; +} + DataTypePtr getDataTypeFromNumpyType(const std::shared_ptr & numpy_type) { switch (numpy_type->getTypeIndex()) @@ -54,6 +82,8 @@ DataTypePtr getDataTypeFromNumpyType(const std::shared_ptr & nump return std::make_shared(); case NumpyDataTypeIndex::UInt64: return std::make_shared(); + case NumpyDataTypeIndex::Float16: + return std::make_shared(); case NumpyDataTypeIndex::Float32: return std::make_shared(); case NumpyDataTypeIndex::Float64: @@ -266,14 +296,26 @@ NpyRowInputFormat::NpyRowInputFormat(ReadBuffer & in_, Block header_, Params par } template -void NpyRowInputFormat::readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness) +void NpyRowInputFormat::readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness, bool isFloat16) { - DataValue value; - if (endianness == NumpyDataType::Endianness::BIG) - readBinaryBigEndian(value, *in); + if (!isFloat16) + { + DataValue value; + if (endianness == NumpyDataType::Endianness::BIG) + readBinaryBigEndian(value, *in); + else + readBinaryLittleEndian(value, *in); + assert_cast &>(*column).insertValue((static_cast(value))); + } else - readBinaryLittleEndian(value, *in); - assert_cast &>(*column).insertValue(static_cast(value)); + { + uint16_t value; + if (endianness == NumpyDataType::Endianness::BIG) + readBinaryBigEndian(value, *in); + else + readBinaryLittleEndian(value, *in); + assert_cast &>(*column).insertValue(static_cast(convertFloat16ToFloat32(value))); + } } template @@ -300,6 +342,7 @@ void NpyRowInputFormat::readAndInsertFloat(IColumn * column, const DataTypePtr & { switch (npy_type.getTypeIndex()) { + case NumpyDataTypeIndex::Float16: readBinaryValueAndInsert(column->getPtr(), npy_type.getEndianness(), true); break; case NumpyDataTypeIndex::Float32: readBinaryValueAndInsert(column->getPtr(), npy_type.getEndianness()); break; case NumpyDataTypeIndex::Float64: readBinaryValueAndInsert(column->getPtr(), npy_type.getEndianness()); break; default: diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.h b/src/Processors/Formats/Impl/NpyRowInputFormat.h index ad32bdba3bf..2bc28121d11 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.h +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.h @@ -43,7 +43,7 @@ private: void readAndInsertString(MutableColumnPtr column, const DataTypePtr & data_type, const NumpyDataType & npy_type, bool is_fixed); template - void readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness); + void readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness, bool isFloat16 = false); void readRows(MutableColumns & columns); diff --git a/tests/queries/0_stateless/02895_npy_format.reference b/tests/queries/0_stateless/02895_npy_format.reference index 0c90fbfd418..e1c168d1c67 100644 --- a/tests/queries/0_stateless/02895_npy_format.reference +++ b/tests/queries/0_stateless/02895_npy_format.reference @@ -84,3 +84,5 @@ c 0 0 1 +[2.199219,1.099609,3.300781] +[4.25,3.34961,6.628906] diff --git a/tests/queries/0_stateless/02895_npy_format.sh b/tests/queries/0_stateless/02895_npy_format.sh index 1dbf62ceaa2..b60f324467d 100755 --- a/tests/queries/0_stateless/02895_npy_format.sh +++ b/tests/queries/0_stateless/02895_npy_format.sh @@ -56,3 +56,5 @@ $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_str.npy', Npy $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_unicode.npy', Npy, 'value Float32')" 2>&1 | grep -c "BAD_ARGUMENTS" $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/complex.npy')" 2>&1 | grep -c "BAD_ARGUMENTS" + +$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/float_16.npy')" diff --git a/tests/queries/0_stateless/data_npy/float_16.npy b/tests/queries/0_stateless/data_npy/float_16.npy new file mode 100644 index 0000000000000000000000000000000000000000..3224d1e57e3ea08e74382200dccf3af3ab33706e GIT binary patch literal 140 zcmbR27wQ`j$;eQ~P_3SlTAW;@Zl$1ZlV+r@qoAIaUsO_*m=~X4l#&V(cT3DEP6dh= kXCxM+0{I$7Its>`ItsN4WCN}=hcug6P7W@coff(Q0J_p2?f?J) literal 0 HcmV?d00001 From 4da512d6a3a2bcd0ee9d3687445f799a1d865320 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 7 Nov 2023 17:57:39 +0000 Subject: [PATCH 147/813] ehnanced docs --- docs/en/interfaces/formats.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index eb963de0c35..e11f74cfef5 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2469,6 +2469,7 @@ This function is designed to load a NumPy array from a .npy file into ClickHouse | u2 | UInt16 | | u4 | UInt32 | | u8 | UInt64 | +| f2 | Float32 | | f4 | Float32 | | f8 | Float64 | | S | String | From 6f6454e696b62d1f36b4fe5b33fece01272db230 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 7 Nov 2023 18:12:56 +0000 Subject: [PATCH 148/813] style check --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index 387269b8974..9e985dffe0c 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -30,13 +30,15 @@ namespace ErrorCodes namespace { -float convertFloat16ToFloat32(uint16_t float16_value) { +float convertFloat16ToFloat32(uint16_t float16_value) +{ uint16_t sign = (float16_value >> 15) & 0x1; uint16_t exponent = (float16_value >> 10) & 0x1F; uint16_t fraction = float16_value & 0x3FF; // Handling special cases for exponent - if (exponent == 0x1F) { + if (exponent == 0x1F) + { // NaN or Infinity in float16 return (fraction == 0) ? std::numeric_limits::infinity() : std::numeric_limits::quiet_NaN(); } From f619f73f284039c4e3bfebeda2228dd5a799d6e3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 29 Aug 2023 11:53:32 +0000 Subject: [PATCH 149/813] Fix incorrect header in grace hash join and filter pushdown --- src/QueryPipeline/QueryPipelineBuilder.cpp | 12 +++++------- .../02861_filter_pushdown_const_bug.reference | 2 ++ .../0_stateless/02861_filter_pushdown_const_bug.sql | 4 ++++ 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index f9726339872..f13d1c56d7f 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -483,8 +483,6 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe Block left_header = left->getHeader(); - Block joined_header = JoiningTransform::transformHeader(left_header, join); - for (size_t i = 0; i < num_streams; ++i) { auto joining = std::make_shared( @@ -496,9 +494,9 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe { // Process delayed joined blocks when all JoiningTransform are finished. auto delayed = std::make_shared( - joined_header, - [left_header, joined_header, max_block_size, join]() - { return join->getNonJoinedBlocks(left_header, joined_header, max_block_size); }); + output_header, + [left_header, output_header, max_block_size, join]() + { return join->getNonJoinedBlocks(left_header, output_header, max_block_size); }); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); @@ -533,7 +531,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe for (size_t i = 1; i < joined_output_ports.size(); i += 2) delayed_ports_numbers.push_back(i); - auto delayed_processor = std::make_shared(joined_header, 2 * num_streams, delayed_ports_numbers); + auto delayed_processor = std::make_shared(output_header, 2 * num_streams, delayed_ports_numbers); if (collected_processors) collected_processors->emplace_back(delayed_processor); left->pipe.processors->emplace_back(delayed_processor); @@ -545,7 +543,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe left->pipe.output_ports.clear(); for (OutputPort & port : delayed_processor->getOutputs()) left->pipe.output_ports.push_back(&port); - left->pipe.header = joined_header; + left->pipe.header = output_header; left->resize(num_streams); } diff --git a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.reference b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.reference index 428ba88bff0..df8198bc856 100644 --- a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.reference +++ b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.reference @@ -6,3 +6,5 @@ 1 1 1 1 +1 1 +1 1 diff --git a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql index a5ddf830d48..a299e50984f 100644 --- a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql +++ b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql @@ -15,4 +15,8 @@ SELECT key FROM ( SELECT key FROM t1 ) AS t1 JOIN ( SELECT key FROM t1 ) AS t2 O SELECT key FROM ( SELECT 1 AS key ) AS t1 JOIN ( SELECT 1 AS key ) AS t2 ON t1.key = t2.key WHERE key; SELECT * FROM ( SELECT 1 AS key GROUP BY NULL ) AS t1 INNER JOIN (SELECT 1 AS key) AS t2 ON t1.key = t2.key WHERE t1.key ORDER BY key; +SET join_algorithm = 'grace_hash'; + +SELECT * FROM (SELECT key AS a FROM t1 ) t1 INNER JOIN (SELECT key AS c FROM t1 ) t2 ON c = a WHERE a; + DROP TABLE IF EXISTS t1; From 86ba6ad1e85c5c6651ef4d6d2f83567e50ab6d69 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 8 Nov 2023 10:22:44 +0000 Subject: [PATCH 150/813] Local backup and restore --- .../BackupCoordinationKeeperMapTables.cpp | 23 + .../BackupCoordinationKeeperMapTables.h | 23 + src/Backups/BackupCoordinationLocal.cpp | 12 + src/Backups/BackupCoordinationLocal.h | 14 + src/Backups/BackupCoordinationRemote.cpp | 13 + src/Backups/BackupCoordinationRemote.h | 13 + src/Backups/BackupsWorker.cpp | 11 +- src/Backups/IBackupCoordination.h | 6 + src/Backups/IRestoreCoordination.h | 4 + src/Backups/RestoreCoordinationLocal.cpp | 6 + src/Backups/RestoreCoordinationLocal.h | 5 + src/Backups/RestoreCoordinationRemote.cpp | 5 + src/Backups/RestoreCoordinationRemote.h | 4 + src/Backups/WithRetries.cpp | 20 + src/Backups/WithRetries.h | 5 + src/Core/Settings.h | 1 + src/Storages/StorageKeeperMap.cpp | 420 +++++++++++++++--- src/Storages/StorageKeeperMap.h | 22 +- 18 files changed, 527 insertions(+), 80 deletions(-) create mode 100644 src/Backups/BackupCoordinationKeeperMapTables.cpp create mode 100644 src/Backups/BackupCoordinationKeeperMapTables.h diff --git a/src/Backups/BackupCoordinationKeeperMapTables.cpp b/src/Backups/BackupCoordinationKeeperMapTables.cpp new file mode 100644 index 00000000000..50561560dd5 --- /dev/null +++ b/src/Backups/BackupCoordinationKeeperMapTables.cpp @@ -0,0 +1,23 @@ +#include + +namespace DB +{ + +void BackupCoordinationKeeperMapTables::addTable(const std::string & table_zookeeper_root_path, const std::string & table_id, const std::string & data_path_in_backup) +{ + if (auto it = tables_with_info.find(table_zookeeper_root_path); it != tables_with_info.end()) + { + if (table_id > it->second.table_id) + it->second = KeeperMapTableInfo{table_id, data_path_in_backup}; + return; + } + + tables_with_info.emplace(table_zookeeper_root_path, KeeperMapTableInfo{table_id, data_path_in_backup}); +} + +std::string BackupCoordinationKeeperMapTables::getDataPath(const std::string & table_zookeeper_root_path) const +{ + return tables_with_info.at(table_zookeeper_root_path).data_path_in_backup; +} + +} diff --git a/src/Backups/BackupCoordinationKeeperMapTables.h b/src/Backups/BackupCoordinationKeeperMapTables.h new file mode 100644 index 00000000000..28894bb9c6e --- /dev/null +++ b/src/Backups/BackupCoordinationKeeperMapTables.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct BackupCoordinationKeeperMapTables +{ + void addTable(const std::string & table_zookeeper_root_path, const std::string & table_id, const std::string & data_path_in_backup); + std::string getDataPath(const std::string & table_zookeeper_root_path) const; +private: + struct KeeperMapTableInfo + { + std::string table_id; + std::string data_path_in_backup; + }; + + std::unordered_map tables_with_info; +}; + +} diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 27e0f173cf3..fb91bae2303 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -97,6 +97,18 @@ Strings BackupCoordinationLocal::getReplicatedSQLObjectsDirs(const String & load return replicated_sql_objects.getDirectories(loader_zk_path, object_type, ""); } +void BackupCoordinationLocal::addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) +{ + std::lock_guard lock(keeper_map_tables_mutex); + keeper_map_tables.addTable(table_zookeeper_root_path, table_id, data_path_in_backup); +} + +String BackupCoordinationLocal::getKeeperMapDataPath(const String & table_zookeeper_root_path) const +{ + std::lock_guard lock(keeper_map_tables_mutex); + return keeper_map_tables.getDataPath(table_zookeeper_root_path); +} + void BackupCoordinationLocal::addFileInfos(BackupFileInfos && file_infos_) { diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 60fcc014720..1fecf30c51c 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -6,6 +6,8 @@ #include #include #include +#include "Backups/BackupCoordinationKeeperMapTables.h" +#include #include #include @@ -44,6 +46,9 @@ public: void addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) override; Strings getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const override; + void addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) override; + String getKeeperMapDataPath(const String & table_zookeeper_root_path) const override; + void addFileInfos(BackupFileInfos && file_infos) override; BackupFileInfos getFileInfos() const override; BackupFileInfos getFileInfosForAllHosts() const override; @@ -58,13 +63,22 @@ private: BackupCoordinationReplicatedAccess TSA_GUARDED_BY(replicated_access_mutex) replicated_access; BackupCoordinationReplicatedSQLObjects TSA_GUARDED_BY(replicated_sql_objects_mutex) replicated_sql_objects; BackupCoordinationFileInfos TSA_GUARDED_BY(file_infos_mutex) file_infos; + BackupCoordinationKeeperMapTables keeper_map_tables TSA_GUARDED_BY(keeper_map_tables_mutex); std::unordered_set TSA_GUARDED_BY(writing_files_mutex) writing_files; + + struct KeeperMapTableInfo + { + String table_id; + String data_path_in_backup; + }; + mutable std::mutex replicated_tables_mutex; mutable std::mutex replicated_access_mutex; mutable std::mutex replicated_sql_objects_mutex; mutable std::mutex file_infos_mutex; mutable std::mutex writing_files_mutex; + mutable std::mutex keeper_map_tables_mutex; }; } diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index e5fcbf26781..72fc2509089 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -666,6 +666,19 @@ void BackupCoordinationRemote::prepareReplicatedSQLObjects() const replicated_sql_objects->addDirectory(std::move(directory)); } +void BackupCoordinationRemote::addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) +{ + std::lock_guard lock(keeper_map_tables_mutex); + keeper_map_tables.addTable(table_zookeeper_root_path, table_id, data_path_in_backup); +} + +String BackupCoordinationRemote::getKeeperMapDataPath(const String & table_zookeeper_root_path) const +{ + std::lock_guard lock(keeper_map_tables_mutex); + return keeper_map_tables.getDataPath(table_zookeeper_root_path); +} + + void BackupCoordinationRemote::addFileInfos(BackupFileInfos && file_infos_) { { diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index c1c7a40fc44..28c24c574a6 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -7,6 +7,7 @@ #include #include #include +#include "Backups/BackupCoordinationKeeperMapTables.h" namespace DB @@ -63,6 +64,9 @@ public: void addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) override; Strings getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const override; + void addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) override; + String getKeeperMapDataPath(const String & table_zookeeper_root_path) const override; + void addFileInfos(BackupFileInfos && file_infos) override; BackupFileInfos getFileInfos() const override; BackupFileInfos getFileInfosForAllHosts() const override; @@ -108,12 +112,21 @@ private: mutable std::optional TSA_GUARDED_BY(file_infos_mutex) file_infos; std::unordered_set TSA_GUARDED_BY(writing_files_mutex) writing_files; + struct KeeperMapTableInfo + { + String table_id; + String data_path_in_backup; + }; + + mutable BackupCoordinationKeeperMapTables keeper_map_tables TSA_GUARDED_BY(keeper_map_tables_mutex); + mutable std::mutex zookeeper_mutex; mutable std::mutex replicated_tables_mutex; mutable std::mutex replicated_access_mutex; mutable std::mutex replicated_sql_objects_mutex; mutable std::mutex file_infos_mutex; mutable std::mutex writing_files_mutex; + mutable std::mutex keeper_map_tables_mutex; }; } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index da814dcbc08..b19135c5cba 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -58,16 +58,7 @@ namespace auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - BackupCoordinationRemote::BackupKeeperSettings keeper_settings - { - .keeper_max_retries = context->getSettingsRef().backup_restore_keeper_max_retries, - .keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms, - .keeper_retry_max_backoff_ms = context->getSettingsRef().backup_restore_keeper_retry_max_backoff_ms, - .batch_size_for_keeper_multiread = context->getSettingsRef().backup_restore_batch_size_for_keeper_multiread, - .keeper_fault_injection_probability = context->getSettingsRef().backup_restore_keeper_fault_injection_probability, - .keeper_fault_injection_seed = context->getSettingsRef().backup_restore_keeper_fault_injection_seed, - .keeper_value_max_size = context->getSettingsRef().backup_restore_keeper_value_max_size, - }; + BackupCoordinationRemote::BackupKeeperSettings keeper_settings = WithRetries::KeeperSettings::fromContext(context); auto all_hosts = BackupSettings::Util::filterHostIDs( backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 75d9202374b..f80b5dee883 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -56,6 +56,12 @@ public: /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). virtual std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const = 0; + /// Adds information about KeeperMap tables + virtual void addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) = 0; + + /// KeeperMap tables use shared storage without local data so only one table should backup the data + virtual String getKeeperMapDataPath(const String & table_zookeeper_root_path) const = 0; + /// Adds a data path in backup for a replicated table. /// Multiple replicas of the replicated table call this function and then all the added paths can be returned by call of the function /// getReplicatedDataPaths(). diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index fd6f014c326..489292cb88f 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -41,6 +41,10 @@ public: /// The function returns false if user-defined function at a specified zk path are being already restored by another replica. virtual bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) = 0; + /// Sets that this table is going to restore data into Keeper for all KeeperMap tables defined on root_zk_path. + /// The function returns false if data for this specific root path is already being restored by another table. + virtual bool acquireInsertingDataForKeeperMap(const String & root_zk_path) = 0; + /// Generates a new UUID for a table. The same UUID must be used for a replicated table on each replica, /// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly). virtual void generateUUIDForTable(ASTCreateQuery & create_query) = 0; diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 1bd2f8e2ed1..d32625c2c51 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -52,6 +52,12 @@ bool RestoreCoordinationLocal::acquireReplicatedSQLObjects(const String &, UserD return true; } +bool RestoreCoordinationLocal::acquireInsertingDataForKeeperMap(const String & root_zk_path) +{ + std::lock_guard lock{mutex}; + return acquired_data_in_keeper_map_tables.emplace(root_zk_path).second; +} + void RestoreCoordinationLocal::generateUUIDForTable(ASTCreateQuery & create_query) { String query_str = serializeAST(create_query); diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index 339b754fca5..93fbdb79d9f 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -40,6 +40,10 @@ public: /// The function returns false if user-defined function at a specified zk path are being already restored by another replica. bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) override; + /// Sets that this table is going to restore data into Keeper for all KeeperMap tables defined on root_zk_path. + /// The function returns false if data for this specific root path is already being restored by another table. + bool acquireInsertingDataForKeeperMap(const String & root_zk_path) override; + /// Generates a new UUID for a table. The same UUID must be used for a replicated table on each replica, /// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly). void generateUUIDForTable(ASTCreateQuery & create_query) override; @@ -52,6 +56,7 @@ private: std::set> acquired_tables_in_replicated_databases; std::unordered_set acquired_data_in_replicated_tables; std::unordered_map create_query_uuids; + std::unordered_set acquired_data_in_keeper_map_tables; mutable std::mutex mutex; }; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index c71466ad8f4..7e059b8d9cc 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -234,6 +234,11 @@ bool RestoreCoordinationRemote::acquireReplicatedSQLObjects(const String & loade return result; } +bool RestoreCoordinationRemote::acquireInsertingDataForKeeperMap(const String & /*root_zk_path*/) +{ + return true; +} + void RestoreCoordinationRemote::generateUUIDForTable(ASTCreateQuery & create_query) { String query_str = serializeAST(create_query); diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 22d0c0ed6df..7d3ae4ceec9 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -46,6 +46,10 @@ public: /// The function returns false if user-defined function at a specified zk path are being already restored by another replica. bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) override; + /// Sets that this table is going to restore data into Keeper for all KeeperMap tables defined on root_zk_path. + /// The function returns false if data for this specific root path is already being restored by another table. + bool acquireInsertingDataForKeeperMap(const String & root_zk_path) override; + /// Generates a new UUID for a table. The same UUID must be used for a replicated table on each replica, /// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly). void generateUUIDForTable(ASTCreateQuery & create_query) override; diff --git a/src/Backups/WithRetries.cpp b/src/Backups/WithRetries.cpp index 0893c65d8fd..d1612a7da4f 100644 --- a/src/Backups/WithRetries.cpp +++ b/src/Backups/WithRetries.cpp @@ -5,6 +5,21 @@ namespace DB { +WithRetries::KeeperSettings WithRetries::KeeperSettings::fromContext(ContextPtr context) +{ + return + { + .keeper_max_retries = context->getSettingsRef().backup_restore_keeper_max_retries, + .keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms, + .keeper_retry_max_backoff_ms = context->getSettingsRef().backup_restore_keeper_retry_max_backoff_ms, + .batch_size_for_keeper_multiread = context->getSettingsRef().backup_restore_batch_size_for_keeper_multiread, + .keeper_fault_injection_probability = context->getSettingsRef().backup_restore_keeper_fault_injection_probability, + .keeper_fault_injection_seed = context->getSettingsRef().backup_restore_keeper_fault_injection_seed, + .keeper_value_max_size = context->getSettingsRef().backup_restore_keeper_value_max_size, + .batch_size_for_keeper_multi = context->getSettingsRef().backup_restore_batch_size_for_keeper_multi, + }; +} + WithRetries::WithRetries(Poco::Logger * log_, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings_, RenewerCallback callback_) : log(log_) , get_zookeeper(get_zookeeper_) @@ -42,6 +57,11 @@ void WithRetries::renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const } } +const WithRetries::KeeperSettings & WithRetries::getKeeperSettings() const +{ + return settings; +} + WithRetries::FaultyKeeper WithRetries::getFaultyZooKeeper() const { /// We need to create new instance of ZooKeeperWithFaultInjection each time a copy a pointer to ZooKeeper client there diff --git a/src/Backups/WithRetries.h b/src/Backups/WithRetries.h index 3955682be94..8f4a730e6a1 100644 --- a/src/Backups/WithRetries.h +++ b/src/Backups/WithRetries.h @@ -26,6 +26,9 @@ public: Float64 keeper_fault_injection_probability{0}; UInt64 keeper_fault_injection_seed{42}; UInt64 keeper_value_max_size{1048576}; + UInt64 batch_size_for_keeper_multi{1000}; + + static KeeperSettings fromContext(ContextPtr context); }; /// For simplicity a separate ZooKeeperRetriesInfo and a faulty [Zoo]Keeper client @@ -53,6 +56,8 @@ public: /// Used to re-establish new connection inside a retry loop. void renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const; + + const KeeperSettings & getKeeperSettings() const; private: /// This will provide a special wrapper which is useful for testing FaultyKeeper getFaultyZooKeeper() const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index aa5c8569be6..bab9005a22c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -465,6 +465,7 @@ class IColumn; M(UInt64, backup_restore_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ 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, 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/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index f98728c012e..33a97af53f1 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -13,6 +14,9 @@ #include #include +#include +#include + #include #include #include @@ -38,6 +42,16 @@ #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include + #include #include @@ -54,6 +68,7 @@ namespace ErrorCodes extern const int KEEPER_EXCEPTION; extern const int LOGICAL_ERROR; extern const int LIMIT_EXCEEDED; + extern const int CANNOT_RESTORE_TABLE; } namespace @@ -296,13 +311,13 @@ StorageKeeperMap::StorageKeeperMap( const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - const std::string & root_path_, + const std::string & zk_root_path_, UInt64 keys_limit_) : IStorage(table_id) , WithContext(context_->getGlobalContext()) - , root_path(zkutil::extractZooKeeperPath(root_path_, false)) + , zk_root_path(zkutil::extractZooKeeperPath(zk_root_path_, false)) , primary_key(primary_key_) - , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) + , zookeeper_name(zkutil::extractZooKeeperName(zk_root_path_)) , keys_limit(keys_limit_) , log(&Poco::Logger::get(fmt::format("StorageKeeperMap ({})", table_id.getNameForLogs()))) { @@ -320,10 +335,10 @@ StorageKeeperMap::StorageKeeperMap( << "primary key: " << formattedAST(metadata.getPrimaryKey().expression_list_ast) << "\n"; metadata_string = out.str(); - if (root_path.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "root_path should not be empty"); - if (!root_path.starts_with('/')) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "root_path should start with '/'"); + if (zk_root_path.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "zk_root_path should not be empty"); + if (!zk_root_path.starts_with('/')) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "zk_root_path should start with '/'"); auto config_keys_limit = context_->getConfigRef().getUInt64("keeper_map_keys_limit", 0); if (config_keys_limit != 0 && (keys_limit == 0 || keys_limit > config_keys_limit)) @@ -341,20 +356,20 @@ StorageKeeperMap::StorageKeeperMap( LOG_INFO(log, "Keys limit will be set to {}", keys_limit); } - auto root_path_fs = fs::path(path_prefix) / std::string_view{root_path}.substr(1); - root_path = root_path_fs.generic_string(); + auto zk_root_path_fs = fs::path(path_prefix) / std::string_view{zk_root_path}.substr(1); + zk_root_path = zk_root_path_fs.generic_string(); - data_path = root_path_fs / "data"; + zk_data_path = zk_root_path_fs / "data"; - auto metadata_path_fs = root_path_fs / "metadata"; - metadata_path = metadata_path_fs; - tables_path = metadata_path_fs / "tables"; + auto metadata_path_fs = zk_root_path_fs / "metadata"; + zk_metadata_path = metadata_path_fs; + zk_tables_path = metadata_path_fs / "tables"; auto table_unique_id = toString(table_id.uuid) + toString(ServerUUID::get()); - table_path = fs::path(tables_path) / table_unique_id; + zk_table_path = fs::path(zk_tables_path) / table_unique_id; - dropped_path = metadata_path_fs / "dropped"; - dropped_lock_path = fs::path(dropped_path) / "lock"; + zk_dropped_path = metadata_path_fs / "dropped"; + zk_dropped_lock_path = fs::path(zk_dropped_path) / "lock"; if (attach) { @@ -364,17 +379,17 @@ StorageKeeperMap::StorageKeeperMap( auto client = getClient(); - if (root_path != "/" && !client->exists(root_path)) + if (zk_root_path != "/" && !client->exists(zk_root_path)) { - LOG_TRACE(log, "Creating root path {}", root_path); - client->createAncestors(root_path); - client->createIfNotExists(root_path, ""); + LOG_TRACE(log, "Creating root path {}", zk_root_path); + client->createAncestors(zk_root_path); + client->createIfNotExists(zk_root_path, ""); } for (size_t i = 0; i < 1000; ++i) { std::string stored_metadata_string; - auto exists = client->tryGet(metadata_path, stored_metadata_string); + auto exists = client->tryGet(zk_metadata_path, stored_metadata_string); if (exists) { @@ -384,10 +399,10 @@ StorageKeeperMap::StorageKeeperMap( throw Exception( ErrorCodes::BAD_ARGUMENTS, "Path {} is already used but the stored table definition doesn't match. Stored metadata: {}", - root_path, + zk_root_path, stored_metadata_string); - auto code = client->tryCreate(table_path, "", zkutil::CreateMode::Persistent); + auto code = client->tryCreate(zk_table_path, "", zkutil::CreateMode::Persistent); // tables_path was removed with drop if (code == Coordination::Error::ZNONODE) @@ -397,16 +412,16 @@ StorageKeeperMap::StorageKeeperMap( } else if (code != Coordination::Error::ZOK) { - throw zkutil::KeeperException(code, "Failed to create table on path {} because a table with same UUID already exists", root_path); + throw zkutil::KeeperException(code, "Failed to create table on path {} because a table with same UUID already exists", zk_root_path); } return; } - if (client->exists(dropped_path)) + if (client->exists(zk_dropped_path)) { LOG_INFO(log, "Removing leftover nodes"); - auto code = client->tryCreate(dropped_lock_path, "", zkutil::CreateMode::Ephemeral); + auto code = client->tryCreate(zk_dropped_lock_path, "", zkutil::CreateMode::Ephemeral); if (code == Coordination::Error::ZNONODE) { @@ -419,11 +434,11 @@ StorageKeeperMap::StorageKeeperMap( } else if (code != Coordination::Error::ZOK) { - throw Coordination::Exception::fromPath(code, dropped_lock_path); + throw Coordination::Exception::fromPath(code, zk_dropped_lock_path); } else { - auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); + auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(zk_dropped_lock_path, *client); if (!dropTable(client, metadata_drop_lock)) continue; } @@ -431,17 +446,17 @@ StorageKeeperMap::StorageKeeperMap( Coordination::Requests create_requests { - zkutil::makeCreateRequest(metadata_path, metadata_string, zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(data_path, metadata_string, zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(tables_path, "", zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(table_path, "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_metadata_path, metadata_string, zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_data_path, metadata_string, zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_tables_path, "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_table_path, "", zkutil::CreateMode::Persistent), }; Coordination::Responses create_responses; auto code = client->tryMulti(create_requests, create_responses); if (code == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "It looks like a table on path {} was created by another server at the same moment, will retry", root_path); + LOG_INFO(log, "It looks like a table on path {} was created by another server at the same moment, will retry", zk_root_path); continue; } else if (code != Coordination::Error::ZOK) @@ -456,7 +471,7 @@ StorageKeeperMap::StorageKeeperMap( throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, because it is removed concurrently or because " - "of wrong root_path ({})", root_path); + "of wrong zk_root_path ({})", zk_root_path); } @@ -519,7 +534,7 @@ Pipe StorageKeeperMap::read( auto client = getClient(); if (all_scan) - return process_keys(std::make_shared>(client->getChildren(data_path))); + return process_keys(std::make_shared>(client->getChildren(zk_data_path))); return process_keys(std::move(filtered_keys)); } @@ -534,19 +549,19 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont { checkTable(); auto client = getClient(); - client->tryRemoveChildrenRecursive(data_path, true); + client->tryRemoveChildrenRecursive(zk_data_path, true); } bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) { - zookeeper->removeChildrenRecursive(data_path); + zookeeper->removeChildrenRecursive(zk_data_path); bool completely_removed = false; Coordination::Requests ops; ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1)); - ops.emplace_back(zkutil::makeRemoveRequest(dropped_path, -1)); - ops.emplace_back(zkutil::makeRemoveRequest(data_path, -1)); - ops.emplace_back(zkutil::makeRemoveRequest(metadata_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(zk_dropped_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(zk_data_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(zk_metadata_path, -1)); Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); @@ -557,7 +572,7 @@ bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::E { metadata_drop_lock->setAlreadyRemoved(); completely_removed = true; - LOG_INFO(log, "Metadata ({}) and data ({}) was successfully removed from ZooKeeper", metadata_path, data_path); + LOG_INFO(log, "Metadata ({}) and data ({}) was successfully removed from ZooKeeper", zk_metadata_path, zk_data_path); break; } case ZNONODE: @@ -578,25 +593,25 @@ void StorageKeeperMap::drop() auto client = getClient(); // we allow ZNONODE in case we got hardware error on previous drop - if (auto code = client->tryRemove(table_path); code == Coordination::Error::ZNOTEMPTY) + if (auto code = client->tryRemove(zk_table_path); code == Coordination::Error::ZNOTEMPTY) { throw zkutil::KeeperException( - code, "{} contains children which shouldn't happen. Please DETACH the table if you want to delete it", table_path); + code, "{} contains children which shouldn't happen. Please DETACH the table if you want to delete it", zk_table_path); } std::vector children; // if the tables_path is not found, some other table removed it // if there are children, some other tables are still using this path as storage - if (auto code = client->tryGetChildren(tables_path, children); + if (auto code = client->tryGetChildren(zk_tables_path, children); code != Coordination::Error::ZOK || !children.empty()) return; Coordination::Requests ops; Coordination::Responses responses; - ops.emplace_back(zkutil::makeRemoveRequest(tables_path, -1)); - ops.emplace_back(zkutil::makeCreateRequest(dropped_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(dropped_lock_path, "", zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeRemoveRequest(zk_tables_path, -1)); + ops.emplace_back(zkutil::makeCreateRequest(zk_dropped_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_dropped_lock_path, "", zkutil::CreateMode::Ephemeral)); auto code = client->tryMulti(ops, responses); @@ -613,7 +628,7 @@ void StorageKeeperMap::drop() else if (code != Coordination::Error::ZOK) zkutil::KeeperMultiException::check(code, ops, responses); - auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); + auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(zk_dropped_lock_path, *client); dropTable(client, metadata_drop_lock); } @@ -623,6 +638,285 @@ NamesAndTypesList StorageKeeperMap::getVirtuals() const {std::string{version_column_name}, std::make_shared()}}; } +namespace +{ + +constexpr std::string_view backup_data_filename = "data.bin"; +constexpr std::string_view backup_data_location_filename = "data_location.bin"; + +class KeeperMapBackup : public IBackupEntriesLazyBatch, boost::noncopyable +{ +public: + KeeperMapBackup( + const std::string & data_zookeeper_path_, + const std::string & data_path_in_backup, + const DiskPtr & temp_disk_, + UInt64 max_compress_block_size_, + std::shared_ptr with_retries_) + : data_zookeeper_path(data_zookeeper_path_) + , temp_disk(temp_disk_) + , max_compress_block_size(max_compress_block_size_) + , with_retries(std::move(with_retries_)) + { + file_path = fs::path(data_path_in_backup) / backup_data_filename; + } + +private: + size_t getSize() const override + { + return 1; + } + + const String & getName(size_t i) const override + { + chassert(i == 0); + return file_path; + } + + BackupEntries generate() override + { + temp_dir_owner.emplace(temp_disk); + fs::path temp_dir = temp_dir_owner->getRelativePath(); + temp_disk->createDirectories(temp_dir); + + auto data_file_path = temp_dir / fs::path{file_path}.filename(); + auto data_out_compressed = temp_disk->writeFile(data_file_path); + auto data_out = std::make_unique(*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size); + std::vector data_children; + { + auto holder = with_retries->createRetriesControlHolder("getKeeperMapDataKeys"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries->renewZooKeeper(zk); + data_children = zk->getChildren(data_zookeeper_path); + }); + } + LOG_INFO(&Poco::Logger::get("BACKUPER"), "Got {} children", data_children.size()); + + const auto write_rows = [&](std::span keys) + { + std::vector keys_full_path; + keys_full_path.reserve(data_children.size()); + + for (const auto & key : data_children) + keys_full_path.push_back(data_zookeeper_path / key); + + zkutil::ZooKeeper::MultiGetResponse data; + auto holder = with_retries->createRetriesControlHolder("getKeeperMapDataKeys"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper] + { + with_retries->renewZooKeeper(zk); + data = zk->get(keys_full_path); + }); + + for (size_t i = 0; i < keys.size(); ++i) + { + auto & child_data = data[i]; + if (child_data.error != Coordination::Error::ZOK) + continue; + + writeStringBinary(keys[i], *data_out); + writeStringBinary(child_data.data, *data_out); + } + }; + + auto max_multiread_size = with_retries->getKeeperSettings().batch_size_for_keeper_multiread; + + auto keys_it = data_children.begin(); + while (keys_it != data_children.end()) + { + auto step = std::min(static_cast(std::distance(keys_it, data_children.end())), max_multiread_size); + write_rows(std::span{keys_it, keys_it + step}); + keys_it = keys_it + step; + } + + data_out->finalize(); + data_out.reset(); + data_out_compressed->finalize(); + data_out_compressed.reset(); + + return {{file_path, std::make_shared(temp_disk, data_file_path)}}; + } + + fs::path data_zookeeper_path; + DiskPtr temp_disk; + std::optional temp_dir_owner; + UInt64 max_compress_block_size; + String file_path; + std::shared_ptr with_retries; +}; +} + +void StorageKeeperMap::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /*partitions*/) +{ + auto table_id = toString(getStorageID().uuid); + + std::cout << "Backing up for path " << zk_root_path << " table id " << table_id << std::endl; + auto coordination = backup_entries_collector.getBackupCoordination(); + coordination->addKeeperMapTable(zk_root_path, table_id, data_path_in_backup); + + /// This task will be executed after all tables have registered their root zk path and the coordination is ready to + /// assign each path to a single table only. + auto post_collecting_task = [my_table_id = std::move(table_id), coordination, &backup_entries_collector, my_data_path_in_backup = data_path_in_backup, this] + { + auto path_with_data = coordination->getKeeperMapDataPath(zk_root_path); + if (path_with_data == my_data_path_in_backup) + { + std::cout << "Will be backing up data for path " << zk_root_path << " table id " << my_table_id << std::endl; + + auto temp_disk = backup_entries_collector.getContext()->getGlobalTemporaryVolume()->getDisk(0); + auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size; + + auto with_retries = std::make_shared + ( + &Poco::Logger::get(fmt::format("StorageKeeperMapBackup ({})", getStorageID().getNameForLogs())), + [&] { return getClient(); }, + WithRetries::KeeperSettings::fromContext(backup_entries_collector.getContext()), + [](WithRetries::FaultyKeeper &) {} + ); + + backup_entries_collector.addBackupEntries( + std::make_shared(this->zk_data_path, path_with_data, temp_disk, max_compress_block_size, std::move(with_retries)) + ->getBackupEntries()); + return; + } + + std::cout << "Not backing up data for path " << zk_root_path << " table id " << my_table_id << " writing only path with data " << path_with_data << std::endl; + auto file_path = fs::path(my_data_path_in_backup) / backup_data_location_filename; + backup_entries_collector.addBackupEntries({{file_path, std::make_shared(path_with_data)}}); + }; + + backup_entries_collector.addPostTask(post_collecting_task); +} + +void StorageKeeperMap::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & /*partitions*/) +{ + auto backup = restorer.getBackup(); + if (!backup->hasFiles(data_path_in_backup)) + return; + + if (!restorer.getRestoreCoordination()->acquireInsertingDataForKeeperMap(zk_root_path)) + { + /// Other table is already restoring the data for this Keeper path. + /// Tables defined on the same path share data + return; + } + + auto with_retries = std::make_shared + ( + &Poco::Logger::get(fmt::format("StorageKeeperMapRestore ({})", getStorageID().getNameForLogs())), + [&] { return getClient(); }, + WithRetries::KeeperSettings::fromContext(restorer.getContext()), + [](WithRetries::FaultyKeeper &) {} + ); + + bool allow_non_empty_tables = restorer.isNonEmptyTableAllowed(); + if (!allow_non_empty_tables) + { + Coordination::Stat data_stats; + + auto holder = with_retries->createRetriesControlHolder("checkKeeperMapData"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries->renewZooKeeper(zk); + zk->get(zk_data_path, &data_stats); + }); + + if (data_stats.numChildren != 0) + RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); + } + + /// TODO: Should we backup and verify the table structure? + + //auto temp_disk = restorer.getContext()->getGlobalTemporaryVolume()->getDisk(0); + /// only 1 table should restore data for a single path + restorer.addDataRestoreTask( + [storage = std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, with_retries, allow_non_empty_tables] + { storage->restoreDataImpl(backup, data_path_in_backup, with_retries, allow_non_empty_tables); }); +} + +void StorageKeeperMap::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, std::shared_ptr with_retries, bool allow_non_empty_tables) +{ + auto table_id = toString(getStorageID().uuid); + + std::cout << "Restoring into " << zk_root_path << " table id " << table_id << std::endl; + + fs::path data_path_in_backup_fs = data_path_in_backup; + + String data_file = data_path_in_backup_fs / backup_data_filename; + + if (!backup->fileExists(data_file)) + { + String data_location_file = data_path_in_backup_fs / "data_location.bin"; + if (!backup->fileExists(data_location_file)) + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Files {} or {} in backup are required to restore table", data_file, data_location_file); + + auto in = backup->readFile(data_location_file); + readStringUntilEOF(data_file, *in); + + data_file = fs::path(data_file) / backup_data_filename; + + if (!backup->fileExists(data_file)) + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", data_file); + } + + /// should we store locally in temp file? + auto in = backup->readFile(data_file); + CompressedReadBuffer compressed_in{*in}; + fs::path data_path_fs(zk_data_path); + + auto max_multi_size = with_retries->getKeeperSettings().batch_size_for_keeper_multi; + + Coordination::Requests create_requests; + const auto flush_create_requests = [&] + { + auto holder = with_retries->createRetriesControlHolder("addKeeperMapData"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries->renewZooKeeper(zk); + zk->multi(create_requests); + }); + }; + + while (!in->eof()) + { + std::string key; + std::string value; + readStringBinary(key, compressed_in); + readStringBinary(value, compressed_in); + + /// if a table can be non empty we can have conflicting keys so we need to do single create for each row + if (allow_non_empty_tables) + { + auto holder = with_retries->createRetriesControlHolder("addKeeperMapData"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries->renewZooKeeper(zk); + zk->tryCreate(data_path_fs / key, value, zkutil::CreateMode::Persistent); + }); + } + /// otherwise we can do multi requests + else + { + create_requests.push_back(zkutil::makeCreateRequest(data_path_fs / key, value, zkutil::CreateMode::Persistent)); + + if (create_requests.size() == max_multi_size) + { + flush_create_requests(); + create_requests.clear(); + } + } + } + + if (!create_requests.empty()) + flush_create_requests(); +} + zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const { std::lock_guard lock{zookeeper_mutex}; @@ -634,7 +928,7 @@ zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const else zookeeper_client = getContext()->getAuxiliaryZooKeeper(zookeeper_name); - zookeeper_client->sync(root_path); + zookeeper_client->sync(zk_root_path); } return zookeeper_client; @@ -642,12 +936,12 @@ zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const const std::string & StorageKeeperMap::dataPath() const { - return data_path; + return zk_data_path; } std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const { - return fs::path(data_path) / key; + return fs::path(zk_data_path) / key; } UInt64 StorageKeeperMap::keysLimit() const @@ -668,7 +962,7 @@ std::optional StorageKeeperMap::isTableValid() const auto client = getClient(); Coordination::Stat metadata_stat; - auto stored_metadata_string = client->get(metadata_path, &metadata_stat); + auto stored_metadata_string = client->get(zk_metadata_path, &metadata_stat); if (metadata_stat.numChildren == 0) { @@ -681,7 +975,7 @@ std::optional StorageKeeperMap::isTableValid() const LOG_ERROR( log, "Table definition does not match to the one stored in the path {}. Stored definition: {}", - root_path, + zk_root_path, stored_metadata_string); table_is_valid = false; return; @@ -689,9 +983,9 @@ std::optional StorageKeeperMap::isTableValid() const // validate all metadata and data nodes are present Coordination::Requests requests; - requests.push_back(zkutil::makeCheckRequest(table_path, -1)); - requests.push_back(zkutil::makeCheckRequest(data_path, -1)); - requests.push_back(zkutil::makeCheckRequest(dropped_path, -1)); + requests.push_back(zkutil::makeCheckRequest(zk_table_path, -1)); + requests.push_back(zkutil::makeCheckRequest(zk_data_path, -1)); + requests.push_back(zkutil::makeCheckRequest(zk_dropped_path, -1)); Coordination::Responses responses; client->tryMulti(requests, responses); @@ -699,19 +993,19 @@ std::optional StorageKeeperMap::isTableValid() const table_is_valid = false; if (responses[0]->error != Coordination::Error::ZOK) { - LOG_ERROR(log, "Table node ({}) is missing", table_path); + LOG_ERROR(log, "Table node ({}) is missing", zk_table_path); return; } if (responses[1]->error != Coordination::Error::ZOK) { - LOG_ERROR(log, "Data node ({}) is missing", data_path); + LOG_ERROR(log, "Data node ({}) is missing", zk_data_path); return; } if (responses[2]->error == Coordination::Error::ZOK) { - LOG_ERROR(log, "Tables with root node {} are being dropped", root_path); + LOG_ERROR(log, "Tables with root node {} are being dropped", zk_root_path); return; } @@ -962,11 +1256,11 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage KeeperMap requires 1-3 arguments:\n" - "root_path: path in the Keeper where the values will be stored (required)\n" + "zk_root_path: path in the Keeper where the values will be stored (required)\n" "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)"); - const auto root_path_node = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext()); - auto root_path = checkAndGetLiteralArgument(root_path_node, "root_path"); + const auto zk_root_path_node = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext()); + auto zk_root_path = checkAndGetLiteralArgument(zk_root_path_node, "zk_root_path"); UInt64 keys_limit = 0; if (engine_args.size() > 1) @@ -985,7 +1279,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageKeeperMap requires one column in primary key"); return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, keys_limit); + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], zk_root_path, keys_limit); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index ad7b719e972..94b02ca0242 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -10,6 +10,9 @@ #include #include +#include +#include + #include namespace DB @@ -72,6 +75,9 @@ public: } bool supportsDelete() const override { return true; } + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + zkutil::ZooKeeperPtr getClient() const; const std::string & dataPath() const; std::string fullPathForKey(std::string_view key) const; @@ -114,18 +120,20 @@ private: std::optional isTableValid() const; - std::string root_path; + void restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, std::shared_ptr with_retries, bool allow_non_empty_tables); + + std::string zk_root_path; std::string primary_key; - std::string data_path; + std::string zk_data_path; - std::string metadata_path; + std::string zk_metadata_path; - std::string tables_path; - std::string table_path; + std::string zk_tables_path; + std::string zk_table_path; - std::string dropped_path; - std::string dropped_lock_path; + std::string zk_dropped_path; + std::string zk_dropped_lock_path; std::string zookeeper_name; From 399d61b222b2a9b9a082569a2dc29c64c8a14c87 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 8 Nov 2023 11:34:49 +0000 Subject: [PATCH 151/813] Fix transfer query to MySQL compatible query --- .../transformQueryForExternalDatabase.cpp | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 84a696a1e9c..7999584ce0e 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -75,6 +75,33 @@ public: } }; +struct ReplaceLiteralToExprVisitorData +{ + using TypeToVisit = ASTFunction; + + void visit(ASTFunction & func, ASTPtr &) const + { + if (func.name == "and" || func.name == "or") + { + for (auto & argument : func.arguments->children) + { + auto * literal_expr = typeid_cast(argument.get()); + UInt64 value; + if (literal_expr && literal_expr->value.tryGet(value) && (value == 0 || value == 1)) + { + /// 1 -> 1=1, 0 -> 1=0. + if (value) + argument = makeASTFunction("equals", std::make_shared(1), std::make_shared(1)); + else + argument = makeASTFunction("equals", std::make_shared(1), std::make_shared(0)); + } + } + } + } +}; + +using ReplaceLiteralToExprVisitor = InDepthNodeVisitor, true>; + class DropAliasesMatcher { public: @@ -288,6 +315,10 @@ String transformQueryForExternalDatabaseImpl( { replaceConstantExpressions(original_where, context, available_columns); + /// Replace like WHERE 1 AND 1 to WHRE 1 = 1 AND 1 = 1 + ReplaceLiteralToExprVisitor::Data replace_literal_to_expr_data; + ReplaceLiteralToExprVisitor(replace_literal_to_expr_data).visit(original_where); + if (isCompatible(original_where)) { select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where)); From 18a5eeec38663bca5a894bb01f0cf84ad39e0b64 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 8 Nov 2023 13:14:09 +0000 Subject: [PATCH 152/813] Make on cluster backup/restore work --- .../BackupCoordinationKeeperMapTables.h | 4 +- src/Backups/BackupCoordinationLocal.h | 4 +- src/Backups/BackupCoordinationRemote.cpp | 65 ++++++++++++++++++- src/Backups/BackupCoordinationRemote.h | 5 +- src/Backups/RestoreCoordinationRemote.cpp | 39 ++++++++++- src/Storages/StorageKeeperMap.cpp | 8 +-- 6 files changed, 107 insertions(+), 18 deletions(-) diff --git a/src/Backups/BackupCoordinationKeeperMapTables.h b/src/Backups/BackupCoordinationKeeperMapTables.h index 28894bb9c6e..a642903cfae 100644 --- a/src/Backups/BackupCoordinationKeeperMapTables.h +++ b/src/Backups/BackupCoordinationKeeperMapTables.h @@ -10,13 +10,13 @@ struct BackupCoordinationKeeperMapTables { void addTable(const std::string & table_zookeeper_root_path, const std::string & table_id, const std::string & data_path_in_backup); std::string getDataPath(const std::string & table_zookeeper_root_path) const; -private: + struct KeeperMapTableInfo { std::string table_id; std::string data_path_in_backup; }; - +private: std::unordered_map tables_with_info; }; diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 1fecf30c51c..6f8e750697c 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -5,8 +5,8 @@ #include #include #include +#include #include -#include "Backups/BackupCoordinationKeeperMapTables.h" #include #include #include @@ -65,7 +65,7 @@ private: BackupCoordinationFileInfos TSA_GUARDED_BY(file_infos_mutex) file_infos; BackupCoordinationKeeperMapTables keeper_map_tables TSA_GUARDED_BY(keeper_map_tables_mutex); std::unordered_set TSA_GUARDED_BY(writing_files_mutex) writing_files; - + struct KeeperMapTableInfo { String table_id; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 72fc2509089..309cbc8be6a 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -230,6 +230,7 @@ void BackupCoordinationRemote::createRootNodes() ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_data_paths", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_access", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_sql_objects", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/keeper_map_tables", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/file_infos", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/writing_files", "", zkutil::CreateMode::Persistent)); zk->tryMulti(ops, responses); @@ -668,14 +669,72 @@ void BackupCoordinationRemote::prepareReplicatedSQLObjects() const void BackupCoordinationRemote::addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) { - std::lock_guard lock(keeper_map_tables_mutex); - keeper_map_tables.addTable(table_zookeeper_root_path, table_id, data_path_in_backup); + { + std::lock_guard lock{keeper_map_tables_mutex}; + if (keeper_map_tables) + throw Exception(ErrorCodes::LOGICAL_ERROR, "addKeeperMapTable() must not be called after preparing"); + } + + auto holder = with_retries.createRetriesControlHolder("addKeeperMapTable"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + String path = zookeeper_path + "/keeper_map_tables/" + escapeForFileName(table_id); + zk->create(path, fmt::format("{}\n{}", table_zookeeper_root_path, data_path_in_backup), zkutil::CreateMode::Persistent); + }); +} + +void BackupCoordinationRemote::prepareKeeperMapTables() const +{ + if (keeper_map_tables) + return; + + std::vector> keeper_map_table_infos; + auto holder = with_retries.createRetriesControlHolder("prepareKeeperMapTables"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + keeper_map_table_infos.clear(); + + with_retries.renewZooKeeper(zk); + + fs::path tables_path = fs::path(zookeeper_path) / "keeper_map_tables"; + + auto tables = zk->getChildren(tables_path); + keeper_map_table_infos.reserve(tables.size()); + + for (auto & table : tables) + table = tables_path / table; + + auto tables_info = zk->get(tables); + for (size_t i = 0; i < tables_info.size(); ++i) + { + const auto & table_info = tables_info[i]; + + if (table_info.error != Coordination::Error::ZOK) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Path in Keeper {} is unexpectedly missing", tables[i]); + + std::vector data; + boost::split(data, table_info.data, [](char c) { return c == '\n'; }); + keeper_map_table_infos.emplace_back( + std::move(data[0]), + BackupCoordinationKeeperMapTables::KeeperMapTableInfo{ + .table_id = fs::path(tables[i]).filename(), .data_path_in_backup = std::move(data[1])}); + } + }); + + keeper_map_tables.emplace(); + for (const auto & [zk_root_path, table_info] : keeper_map_table_infos) + keeper_map_tables->addTable(zk_root_path, table_info.table_id, table_info.data_path_in_backup); + } String BackupCoordinationRemote::getKeeperMapDataPath(const String & table_zookeeper_root_path) const { std::lock_guard lock(keeper_map_tables_mutex); - return keeper_map_tables.getDataPath(table_zookeeper_root_path); + prepareKeeperMapTables(); + return keeper_map_tables->getDataPath(table_zookeeper_root_path); } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 28c24c574a6..a0a9224bf71 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -5,9 +5,9 @@ #include #include #include +#include #include #include -#include "Backups/BackupCoordinationKeeperMapTables.h" namespace DB @@ -89,6 +89,7 @@ private: void prepareReplicatedTables() const TSA_REQUIRES(replicated_tables_mutex); void prepareReplicatedAccess() const TSA_REQUIRES(replicated_access_mutex); void prepareReplicatedSQLObjects() const TSA_REQUIRES(replicated_sql_objects_mutex); + void prepareKeeperMapTables() const TSA_REQUIRES(keeper_map_tables_mutex); void prepareFileInfos() const TSA_REQUIRES(file_infos_mutex); const String root_zookeeper_path; @@ -110,6 +111,7 @@ private: mutable std::optional TSA_GUARDED_BY(replicated_access_mutex) replicated_access; mutable std::optional TSA_GUARDED_BY(replicated_sql_objects_mutex) replicated_sql_objects; mutable std::optional TSA_GUARDED_BY(file_infos_mutex) file_infos; + mutable std::optional keeper_map_tables TSA_GUARDED_BY(keeper_map_tables_mutex); std::unordered_set TSA_GUARDED_BY(writing_files_mutex) writing_files; struct KeeperMapTableInfo @@ -118,7 +120,6 @@ private: String data_path_in_backup; }; - mutable BackupCoordinationKeeperMapTables keeper_map_tables TSA_GUARDED_BY(keeper_map_tables_mutex); mutable std::mutex zookeeper_mutex; mutable std::mutex replicated_tables_mutex; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 7e059b8d9cc..12a67d2a55d 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -89,6 +89,7 @@ void RestoreCoordinationRemote::createRootNodes() ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_tables_data_acquired", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_access_storages_acquired", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_sql_objects_acquired", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/keeper_map_tables", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/table_uuids", "", zkutil::CreateMode::Persistent)); zk->tryMulti(ops, responses); }); @@ -234,9 +235,43 @@ bool RestoreCoordinationRemote::acquireReplicatedSQLObjects(const String & loade return result; } -bool RestoreCoordinationRemote::acquireInsertingDataForKeeperMap(const String & /*root_zk_path*/) +bool RestoreCoordinationRemote::acquireInsertingDataForKeeperMap(const String & root_zk_path) { - return true; + bool result = false; + auto holder = with_retries.createRetriesControlHolder("acquireInsertingDataForKeeperMap"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + fs::path base_path = fs::path(zookeeper_path) / "keeper_map_tables" / root_zk_path; + zk->createAncestors(base_path); + std::string restore_lock_path = base_path / "restore_lock"; + result = zk->tryCreate(restore_lock_path, "restorelock", zkutil::CreateMode::Persistent) == Coordination::Error::ZOK; + + if (result) + return; + + /// there can be an edge case where a path contains `/restore_lock/ in the middle of it + /// to differentiate that case from lock we also set the data + for (size_t i = 0; i < 1000; ++i) + { + Coordination::Stat lock_stat; + auto data = zk->get(restore_lock_path, &lock_stat); + if (data == "restorelock") + return; + + if (auto set_result = zk->trySet(restore_lock_path, "restorelock", lock_stat.version); + set_result == Coordination::Error::ZOK) + { + result = true; + return; + } + else if (set_result == Coordination::Error::ZNONODE) + throw zkutil::KeeperException::fromPath(set_result, restore_lock_path); + } + }); + return result; } void RestoreCoordinationRemote::generateUUIDForTable(ASTCreateQuery & create_query) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 33a97af53f1..c583a693035 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -753,7 +753,6 @@ void StorageKeeperMap::backupData(BackupEntriesCollector & backup_entries_collec { auto table_id = toString(getStorageID().uuid); - std::cout << "Backing up for path " << zk_root_path << " table id " << table_id << std::endl; auto coordination = backup_entries_collector.getBackupCoordination(); coordination->addKeeperMapTable(zk_root_path, table_id, data_path_in_backup); @@ -764,8 +763,6 @@ void StorageKeeperMap::backupData(BackupEntriesCollector & backup_entries_collec auto path_with_data = coordination->getKeeperMapDataPath(zk_root_path); if (path_with_data == my_data_path_in_backup) { - std::cout << "Will be backing up data for path " << zk_root_path << " table id " << my_table_id << std::endl; - auto temp_disk = backup_entries_collector.getContext()->getGlobalTemporaryVolume()->getDisk(0); auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size; @@ -783,7 +780,6 @@ void StorageKeeperMap::backupData(BackupEntriesCollector & backup_entries_collec return; } - std::cout << "Not backing up data for path " << zk_root_path << " table id " << my_table_id << " writing only path with data " << path_with_data << std::endl; auto file_path = fs::path(my_data_path_in_backup) / backup_data_location_filename; backup_entries_collector.addBackupEntries({{file_path, std::make_shared(path_with_data)}}); }; @@ -842,8 +838,6 @@ void StorageKeeperMap::restoreDataImpl(const BackupPtr & backup, const String & { auto table_id = toString(getStorageID().uuid); - std::cout << "Restoring into " << zk_root_path << " table id " << table_id << std::endl; - fs::path data_path_in_backup_fs = data_path_in_backup; String data_file = data_path_in_backup_fs / backup_data_filename; @@ -882,7 +876,7 @@ void StorageKeeperMap::restoreDataImpl(const BackupPtr & backup, const String & }); }; - while (!in->eof()) + while (!compressed_in.eof()) { std::string key; std::string value; From 4438c2f70aeadd6ec6ab221f2980210351b7d1c5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 8 Nov 2023 14:36:39 +0000 Subject: [PATCH 153/813] Remove unnecassary log --- src/Storages/StorageKeeperMap.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index c583a693035..74c1905cd61 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -692,14 +692,13 @@ private: data_children = zk->getChildren(data_zookeeper_path); }); } - LOG_INFO(&Poco::Logger::get("BACKUPER"), "Got {} children", data_children.size()); const auto write_rows = [&](std::span keys) { std::vector keys_full_path; - keys_full_path.reserve(data_children.size()); + keys_full_path.reserve(keys.size()); - for (const auto & key : data_children) + for (const auto & key : keys) keys_full_path.push_back(data_zookeeper_path / key); zkutil::ZooKeeper::MultiGetResponse data; From d0ba561faacdd97f22f8486b92cc2b29e7236677 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 8 Nov 2023 23:20:12 +0800 Subject: [PATCH 154/813] Update src/Storages/transformQueryForExternalDatabase.cpp Co-authored-by: Sema Checherinda <104093494+CheSema@users.noreply.github.com> --- src/Storages/transformQueryForExternalDatabase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 7999584ce0e..fd3c47e0e3f 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -315,7 +315,7 @@ String transformQueryForExternalDatabaseImpl( { replaceConstantExpressions(original_where, context, available_columns); - /// Replace like WHERE 1 AND 1 to WHRE 1 = 1 AND 1 = 1 + /// Replace like WHERE 1 AND 1 to WHERE 1 = 1 AND 1 = 1 ReplaceLiteralToExprVisitor::Data replace_literal_to_expr_data; ReplaceLiteralToExprVisitor(replace_literal_to_expr_data).visit(original_where); From 6e189b17c01cc45c3c8da1079cf8749daff0a2e0 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 8 Nov 2023 18:55:19 +0300 Subject: [PATCH 155/813] merge_row_policy: processAliases introduced --- src/Storages/StorageMerge.cpp | 279 ++++++++++++++++------ src/Storages/StorageMerge.h | 16 +- src/TableFunctions/TableFunctionMerge.cpp | 2 +- 3 files changed, 213 insertions(+), 84 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 8002a5f4b8f..d9460190677 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -385,7 +385,7 @@ public: /// Add to data stream columns that are needed only for row policies /// SELECT x from T if T has row policy y=42 /// required y in data pipeline - void extendNames(Names &); + void extendNames(Names &, bool alias_allowed = true); /// Use storage facilities to filter data /// optimization @@ -401,6 +401,7 @@ private: std::string filter_column_name; // complex filter, may contain logic operations ActionsDAGPtr actions_dag; ExpressionActionsPtr filter_actions; + StorageMetadataPtr storage_metadata_snapshot; }; // using RowPolicyDataPtr = std::unique_ptr; @@ -472,7 +473,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu query_info.input_order_info = input_sorting_info; } - auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); + // auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); std::vector> pipelines; QueryPlanResourceHolder resources; @@ -494,98 +495,100 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu if (sampling_requested && !storage->supportsSampling()) throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: table doesn't support sampling"); - const auto & [database_name, _, _b, table_name] = table; + // const auto & [database_name, _, _b, table_name] = table; - std::unique_ptr row_policy_data_ptr; + // std::unique_ptr row_policy_data_ptr; - auto row_policy_filter_ptr = context->getRowPolicyFilter( - database_name, - table_name, - RowPolicyFilterType::SELECT_FILTER); - if (row_policy_filter_ptr) - { - row_policy_data_ptr = std::make_unique(row_policy_filter_ptr, storage, context); - row_policy_data_ptr->extendNames(column_names); - } + // auto row_policy_filter_ptr = context->getRowPolicyFilter( + // database_name, + // table_name, + // RowPolicyFilterType::SELECT_FILTER); + // if (row_policy_filter_ptr) + // { + // row_policy_data_ptr = std::make_unique(row_policy_filter_ptr, storage, context); + // row_policy_data_ptr->extendNames(column_names); + // } - Aliases aliases; - auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, context); + // Aliases aliases; + // auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + // auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, context); - auto modified_query_info = getModifiedQueryInfo(query_info, context, table, nested_storage_snaphsot); - Names column_names_as_aliases; + // auto modified_query_info = getModifiedQueryInfo(query_info, context, table, nested_storage_snaphsot); + // Names column_names_as_aliases; - if (!context->getSettingsRef().allow_experimental_analyzer) - { - auto storage_columns = storage_metadata_snapshot->getColumns(); - auto syntax_result = TreeRewriter(context).analyzeSelect( - modified_query_info.query, TreeRewriterResult({}, storage, nested_storage_snaphsot)); + // if (!context->getSettingsRef().allow_experimental_analyzer) + // { + // auto storage_columns = storage_metadata_snapshot->getColumns(); + // auto syntax_result = TreeRewriter(context).analyzeSelect( + // modified_query_info.query, TreeRewriterResult({}, storage, nested_storage_snaphsot)); - bool with_aliases = common_processed_stage == QueryProcessingStage::FetchColumns && !storage_columns.getAliases().empty(); - if (with_aliases) - { - ASTPtr required_columns_expr_list = std::make_shared(); - ASTPtr column_expr; + // bool with_aliases = common_processed_stage == QueryProcessingStage::FetchColumns && !storage_columns.getAliases().empty(); + // if (with_aliases) + // { + // ASTPtr required_columns_expr_list = std::make_shared(); + // ASTPtr column_expr; - for (const auto & column : column_names) - { - const auto column_default = storage_columns.getDefault(column); - bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias; + // for (const auto & column : column_names) + // { + // const auto column_default = storage_columns.getDefault(column); + // bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias; - if (is_alias) - { - column_expr = column_default->expression->clone(); - replaceAliasColumnsInQuery(column_expr, storage_metadata_snapshot->getColumns(), - syntax_result->array_join_result_to_source, context); + // if (is_alias) + // { + // column_expr = column_default->expression->clone(); + // replaceAliasColumnsInQuery(column_expr, storage_metadata_snapshot->getColumns(), + // syntax_result->array_join_result_to_source, context); - const auto & column_description = storage_columns.get(column); - column_expr = addTypeConversionToAST(std::move(column_expr), column_description.type->getName(), - storage_metadata_snapshot->getColumns().getAll(), context); - column_expr = setAlias(column_expr, column); + // const auto & column_description = storage_columns.get(column); + // column_expr = addTypeConversionToAST(std::move(column_expr), column_description.type->getName(), + // storage_metadata_snapshot->getColumns().getAll(), context); + // column_expr = setAlias(column_expr, column); - auto type = sample_block.getByName(column).type; - aliases.push_back({ .name = column, .type = type, .expression = column_expr->clone() }); + // auto type = sample_block.getByName(column).type; + // aliases.push_back({ .name = column, .type = type, .expression = column_expr->clone() }); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), - "adding new alias name {}, expression {}", - column, column_expr->formatForLogging()); + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), + // "adding new alias name {}, expression {}", + // column, column_expr->formatForLogging()); - } - else - column_expr = std::make_shared(column); + // } + // else + // column_expr = std::make_shared(column); - required_columns_expr_list->children.emplace_back(std::move(column_expr)); - } + // required_columns_expr_list->children.emplace_back(std::move(column_expr)); + // } - syntax_result = TreeRewriter(context).analyze( - required_columns_expr_list, storage_columns.getAllPhysical(), storage, storage->getStorageSnapshot(storage_metadata_snapshot, context)); + // syntax_result = TreeRewriter(context).analyze( + // required_columns_expr_list, storage_columns.getAllPhysical(), storage, storage->getStorageSnapshot(storage_metadata_snapshot, context)); - auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); + // auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); - column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), - "alias_actions->getRequiredColumns: {}", alias_actions->getRequiredColumns().toString()); + // column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); + // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), + // "alias_actions->getRequiredColumns: {}", alias_actions->getRequiredColumns().toString()); - // if (row_policy_data_ptr) - // row_policy_data_ptr->extendNames(column_names_as_aliases); + // // if (row_policy_data_ptr) + // // row_policy_data_ptr->extendNames(column_names_as_aliases, false /* alias_allowed */); - if (column_names_as_aliases.empty()) - column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); - } - } + // if (column_names_as_aliases.empty()) + // column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); + // } + // } auto source_pipeline = createSources( - nested_storage_snaphsot, - modified_query_info, + // nested_storage_snaphsot, + // modified_query_info, common_processed_stage, required_max_block_size, common_header, - aliases, + // aliases, table, - column_names_as_aliases.empty() ? column_names : column_names_as_aliases, - std::move(row_policy_data_ptr), + // column_names_as_aliases.empty() ? column_names : column_names_as_aliases, + column_names, + merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(), + // std::move(row_policy_data_ptr), context, current_streams); @@ -678,23 +681,128 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer return modified_query_info; } +void ReadFromMerge::processAliases( + Names & real_column_names, + const StorageWithLockAndName & storage_with_lock, + Aliases & aliases, + const Block & sample_block, + ContextMutablePtr modified_context) +{ + auto storage = std::get<1>(storage_with_lock); + auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, modified_context); + + auto modified_query_info = getModifiedQueryInfo(query_info, context, storage_with_lock, nested_storage_snaphsot); + Names column_names_as_aliases; + + if (!context->getSettingsRef().allow_experimental_analyzer) + { + auto storage_columns = storage_metadata_snapshot->getColumns(); + auto syntax_result = TreeRewriter(context).analyzeSelect( + modified_query_info.query, TreeRewriterResult({}, storage, nested_storage_snaphsot)); + + bool with_aliases = common_processed_stage == QueryProcessingStage::FetchColumns && !storage_columns.getAliases().empty(); + if (with_aliases) + { + ASTPtr required_columns_expr_list = std::make_shared(); + ASTPtr column_expr; + + for (const auto & column : real_column_names) + { + const auto column_default = storage_columns.getDefault(column); + bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias; + + if (is_alias) + { + column_expr = column_default->expression->clone(); + replaceAliasColumnsInQuery(column_expr, storage_metadata_snapshot->getColumns(), + syntax_result->array_join_result_to_source, context); + + const auto & column_description = storage_columns.get(column); + column_expr = addTypeConversionToAST(std::move(column_expr), column_description.type->getName(), + storage_metadata_snapshot->getColumns().getAll(), context); + column_expr = setAlias(column_expr, column); + + auto type = sample_block.getByName(column).type; + aliases.push_back({ .name = column, .type = type, .expression = column_expr->clone() }); + + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), + "adding new alias name {}, expression {}", + column, column_expr->formatForLogging()); + + } + else + column_expr = std::make_shared(column); + + required_columns_expr_list->children.emplace_back(std::move(column_expr)); + } + + syntax_result = TreeRewriter(context).analyze( + required_columns_expr_list, storage_columns.getAllPhysical(), storage, storage->getStorageSnapshot(storage_metadata_snapshot, context)); + + auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); + + column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::processAliases"), + "alias_actions->getRequiredColumns: {}", alias_actions->getRequiredColumns().toString()); + + // if (row_policy_data_ptr) + // row_policy_data_ptr->extendNames(column_names_as_aliases, false /* alias_allowed */); + + if (column_names_as_aliases.empty()) + column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); + } + } + if (!column_names_as_aliases.empty()) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::processAliases"), + "substitute real_column_names by column_names_as_aliases"); + real_column_names = column_names_as_aliases; + } +} + + QueryPipelineBuilderPtr ReadFromMerge::createSources( - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & modified_query_info, + // const StorageSnapshotPtr & storage_snapshot, + // SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, const Block & header, - const Aliases & aliases, const StorageWithLockAndName & storage_with_lock, Names real_column_names, - std::unique_ptr row_policy_data_ptr, + const Block & sample_block, + // std::unique_ptr row_policy_data_ptr, ContextMutablePtr modified_context, size_t streams_num, bool concat_streams) { const auto & [database_name, storage, _, table_name] = storage_with_lock; + auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + auto storage_snapshot = storage->getStorageSnapshot(storage_metadata_snapshot, context); + auto modified_query_info = getModifiedQueryInfo(query_info, context, storage_with_lock, storage_snapshot); + auto & modified_select = modified_query_info.query->as(); + std::unique_ptr row_policy_data_ptr; + + auto row_policy_filter_ptr = context->getRowPolicyFilter( + database_name, + table_name, + RowPolicyFilterType::SELECT_FILTER); + if (row_policy_filter_ptr) + { + row_policy_data_ptr = std::make_unique(row_policy_filter_ptr, storage, context); + row_policy_data_ptr->extendNames(real_column_names); + } + + + Aliases aliases; + processAliases(real_column_names, storage_with_lock, aliases, sample_block, modified_context); + // if (row_policy_data_ptr) + // { + // row_policy_data_ptr->extendNames(real_column_names); + // } + QueryPipelineBuilderPtr builder; if (!InterpreterSelectQuery::isQueryWithFinal(modified_query_info) && storage->needRewriteQueryWithFinal(real_column_names)) { @@ -883,7 +991,10 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter ContextPtr local_context) : row_policy_filter_ptr(row_policy_filter_ptr_) { - auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData ctor"), + "storage {}", storage->getName()); + + storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_columns = storage_metadata_snapshot->getColumns(); auto needed_columns = storage_columns.getAll/*Physical*/(); @@ -912,7 +1023,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter filter_column_name = added.getNames().front(); } -void ReadFromMerge::RowPolicyData::extendNames(Names & names) +void ReadFromMerge::RowPolicyData::extendNames(Names & names, bool alias_allowed) { std::sort(names.begin(), names.end()); NameSet added_names; @@ -921,6 +1032,16 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) { if (!std::binary_search(names.begin(), names.end(), req_column)) { + if (!alias_allowed) + { + auto storage_columns = storage_metadata_snapshot->getColumns(); + const auto column_default = storage_columns.getDefault(req_column); + bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias; + if (is_alias) + { + continue; + } + } added_names.insert(req_column); } } @@ -929,11 +1050,11 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) std::copy(added_names.begin(), added_names.end(), std::back_inserter(names)); LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), "{} names added", added_names.size()); - // for (const auto & added_name : added_names) - // { - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), - // " added name {}", added_name); - // } + for (const auto & added_name : added_names) + { + LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), + " added name {}", added_name); + } } } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 80c5790d319..350d871359a 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -184,16 +184,24 @@ private: const StorageWithLockAndName & storage_with_lock_and_name, const StorageSnapshotPtr & storage_snapshot); + void processAliases( + Names & real_column_names, + const StorageWithLockAndName & storage_with_lock, + Aliases & aliases, + const Block & sample_block, + ContextMutablePtr modified_context); + QueryPipelineBuilderPtr createSources( - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, + // const StorageSnapshotPtr & storage_snapshot, + // SelectQueryInfo & query_info, QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, const Block & header, - const Aliases & aliases, + // const Aliases & aliases, const StorageWithLockAndName & storage_with_lock, Names real_column_names, - std::unique_ptr row_policy_data_ptr, + const Block & sample_block, + // std::unique_ptr row_policy_data_ptr, ContextMutablePtr modified_context, size_t streams_num, bool concat_streams = false); diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 5231c651f17..d0457504144 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -126,7 +126,7 @@ ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr contex { auto storage = DatabaseCatalog::instance().tryGetTable(StorageID{db_with_tables.first, table}, context); if (storage) - return ColumnsDescription{storage->getInMemoryMetadataPtr()->getColumns().getAllPhysical()}; // !!! + return ColumnsDescription{storage->getInMemoryMetadataPtr()->getColumns().getAll/* Physical*/()}; // !!! } } From 93e22e85d1ce07fde446b68f5fcaa9d75cb2a090 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 8 Nov 2023 17:16:56 +0100 Subject: [PATCH 156/813] Better --- src/Interpreters/Cache/FileCache.cpp | 188 ++++++++++++++---- src/Interpreters/Cache/FileCache.h | 4 + src/Interpreters/Cache/FileSegment.cpp | 1 + .../tests/gtest_lru_file_cache.cpp | 2 +- 4 files changed, 154 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 31e9008c69a..1a09db7a3f0 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -156,7 +156,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: { auto file_segment = std::make_shared( locked_key.getKey(), range.left, range.size(), FileSegment::State::DETACHED); - return { file_segment }; + return {file_segment}; } if (locked_key.empty()) @@ -245,11 +245,34 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: return result; } +std::vector FileCache::splitRange(size_t offset, size_t size) +{ + assert(size > 0); + std::vector ranges; + + size_t current_pos = offset; + size_t end_pos_non_included = offset + size; + size_t remaining_size = size; + + FileSegments file_segments; + while (current_pos < end_pos_non_included) + { + auto current_file_segment_size = std::min(remaining_size, max_file_segment_size); + ranges.emplace_back(current_pos, current_pos + current_file_segment_size - 1); + + remaining_size -= current_file_segment_size; + current_pos += current_file_segment_size; + } + + return ranges; +} + FileSegments FileCache::splitRangeIntoFileSegments( LockedKey & locked_key, size_t offset, size_t size, FileSegment::State state, + size_t file_segments_limit, const CreateFileSegmentSettings & settings) { assert(size > 0); @@ -261,7 +284,7 @@ FileSegments FileCache::splitRangeIntoFileSegments( size_t remaining_size = size; FileSegments file_segments; - while (current_pos < end_pos_non_included) + while (current_pos < end_pos_non_included && (!file_segments_limit || file_segments.size() < file_segments_limit)) { current_file_segment_size = std::min(remaining_size, max_file_segment_size); remaining_size -= current_file_segment_size; @@ -273,7 +296,7 @@ FileSegments FileCache::splitRangeIntoFileSegments( current_pos += current_file_segment_size; } - assert(file_segments.empty() || offset + size - 1 == file_segments.back()->range().right); + assert(file_segments.empty() || file_segments_limit > 0 || offset + size - 1 == file_segments.back()->range().right); return file_segments; } @@ -298,6 +321,7 @@ void FileCache::fillHolesWithEmptyFileSegments( assert(!file_segments.empty()); auto it = file_segments.begin(); + size_t added = 0; auto segment_range = (*it)->range(); size_t current_pos; @@ -310,11 +334,12 @@ void FileCache::fillHolesWithEmptyFileSegments( current_pos = segment_range.right + 1; ++it; + ++added; } else current_pos = range.left; - while (current_pos <= range.right && it != file_segments.end()) + while (current_pos <= range.right && it != file_segments.end() && (!file_segments_limit || added < file_segments_limit)) { segment_range = (*it)->range(); @@ -322,6 +347,7 @@ void FileCache::fillHolesWithEmptyFileSegments( { current_pos = segment_range.right + 1; ++it; + ++added; continue; } @@ -338,17 +364,38 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto split = splitRangeIntoFileSegments( - locked_key, current_pos, hole_size, FileSegment::State::EMPTY, settings); - file_segments.splice(it, std::move(split)); + auto ranges = splitRange(current_pos, hole_size); + FileSegments hole; + for (const auto & r : ranges) + { + auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, settings, nullptr); + hole.push_back(metadata_it->second->file_segment); + ++added; + + if (file_segments_limit && added == file_segments_limit) + { + file_segments.splice(it, std::move(hole)); + file_segments.erase(it, file_segments.end()); + return; + } + } + file_segments.splice(it, std::move(hole)); } current_pos = segment_range.right + 1; ++it; + ++added; } - if (file_segments_limit && file_segments.size() >= file_segments_limit) + if (file_segments_limit && added == file_segments_limit) + { + chassert(file_segments.size() >= file_segments_limit); + file_segments.erase(it, file_segments.end()); + chassert(file_segments.size() == file_segments_limit); return; + } + + chassert(!file_segments_limit || file_segments.size() < file_segments_limit); if (current_pos <= range.right) { @@ -368,9 +415,22 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto split = splitRangeIntoFileSegments( - locked_key, current_pos, hole_size, FileSegment::State::EMPTY, settings); - file_segments.splice(file_segments.end(), std::move(split)); + auto ranges = splitRange(current_pos, hole_size); + FileSegments hole; + for (const auto & r : ranges) + { + auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, settings, nullptr); + hole.push_back(metadata_it->second->file_segment); + ++added; + + if (file_segments_limit && added == file_segments_limit) + { + file_segments.splice(it, std::move(hole)); + file_segments.erase(it, file_segments.end()); + return; + } + } + file_segments.splice(it, std::move(hole)); } } } @@ -400,7 +460,7 @@ FileSegmentsHolderPtr FileCache::set( else { file_segments = splitRangeIntoFileSegments( - *locked_key, offset, size, FileSegment::State::EMPTY, settings); + *locked_key, offset, size, FileSegment::State::EMPTY, /* file_segments_limit */0, settings); } return std::make_unique(std::move(file_segments)); @@ -419,77 +479,125 @@ FileCache::getOrSet( assertInitialized(); - const auto end_offset = offset + size - 1; - const auto aligned_offset = roundDownToMultiple(offset, boundary_alignment); - const auto aligned_end_offset = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size) - 1; - chassert(aligned_offset <= offset); + FileSegment::Range range(offset, offset + size - 1); + + const auto aligned_offset = roundDownToMultiple(range.left, boundary_alignment); + auto aligned_end_offset = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size) - 1; + + chassert(aligned_offset <= range.left); + chassert(aligned_end_offset >= range.right); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY); - /// Get all segments which intersect with the given range. - FileSegment::Range range(offset, end_offset); auto file_segments = getImpl(*locked_key, range, file_segments_limit); - if (aligned_offset < offset && (file_segments.empty() || offset < file_segments.front()->range().left)) + if (file_segments_limit) { - auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? offset - 1 : file_segments.front()->range().left - 1); + chassert(file_segments.size() <= file_segments_limit); + if (file_segments.size() == file_segments_limit) + range.right = aligned_end_offset = file_segments.back()->range().right; + } + + /// Check case if we have uncovered prefix, e.g. + /// + /// [_______________] + /// ^ ^ + /// range.left range.right + /// [___] [__________] <-- current cache (example) + /// [ ] + /// ^----^ + /// uncovered prefix. + const bool has_uncovered_prefix = file_segments.empty() || range.left < file_segments.front()->range().left; + + if (aligned_offset < range.left && has_uncovered_prefix) + { + auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? range.left - 1 : file_segments.front()->range().left - 1); auto prefix_file_segments = getImpl(*locked_key, prefix_range, /* file_segments_limit */0); if (prefix_file_segments.empty()) { + /// [____________________][_______________] + /// ^ ^ ^ + /// aligned_offset range.left range.right + /// [___] [__________] <-- current cache (example) range.left = aligned_offset; } else { - size_t last_right_offset = prefix_file_segments.back()->range().right; + /// [____________________][_______________] + /// ^ ^ ^ + /// aligned_offset range.left range.right + /// ____] [____] [___] [__________] <-- current cache (example) + /// ^ + /// prefix_file_segments.back().right - while (!prefix_file_segments.empty() && prefix_file_segments.front()->range().right < offset) - prefix_file_segments.pop_front(); + chassert(prefix_file_segments.back()->range().right < range.left); + chassert(prefix_file_segments.back()->range().right >= aligned_offset); - if (prefix_file_segments.empty()) - { - range.left = last_right_offset + 1; - } - else - { - file_segments.splice(file_segments.begin(), prefix_file_segments); - range.left = file_segments.front()->range().left; - } + range.left = prefix_file_segments.back()->range().right + 1; } } - if (end_offset < aligned_end_offset && (file_segments.empty() || file_segments.back()->range().right < end_offset)) + /// Check case if we have uncovered suffix. + /// + /// [___________________] + /// ^ ^ + /// range.left range.right + /// [___] [___] <-- current cache (example) + /// [___] + /// ^---^ + /// uncovered_suffix + const bool has_uncovered_suffix = file_segments.empty() || file_segments.back()->range().right < range.right; + + if (range.right < aligned_end_offset && has_uncovered_suffix) { - auto suffix_range = FileSegment::Range(end_offset, aligned_end_offset); - /// Get only 1 file segment. + auto suffix_range = FileSegment::Range(range.right, aligned_end_offset); + /// We need to get 1 file segment, so file_segments_limit = 1 here. auto suffix_file_segments = getImpl(*locked_key, suffix_range, /* file_segments_limit */1); if (suffix_file_segments.empty()) + { + /// [__________________][ ] + /// ^ ^ ^ + /// range.left range.right aligned_end_offset + /// [___] [___] <-- current cache (example) + range.right = aligned_end_offset; + } else + { + /// [__________________][ ] + /// ^ ^ ^ + /// range.left range.right aligned_end_offset + /// [___] [___] [_________] <-- current cache (example) + /// ^ + /// suffix_file_segments.front().left range.right = suffix_file_segments.front()->range().left - 1; + } } if (file_segments.empty()) { - file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, settings); + file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, file_segments_limit, settings); } else { - chassert(file_segments.front()->range().right >= offset); - chassert(file_segments.back()->range().left <= end_offset); + chassert(file_segments.front()->range().right >= range.left); + chassert(file_segments.back()->range().left <= range.right); fillHolesWithEmptyFileSegments( *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, settings); + chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); + if (!file_segments.front()->range().contains(offset)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " "(end offset: {}, aligned offset: {}, aligned end offset: {})", - file_segments.front()->range().toString(), offset, end_offset, aligned_offset, aligned_end_offset); + file_segments.front()->range().toString(), offset, range.right, aligned_offset, aligned_end_offset); } - chassert(file_segments_limit ? file_segments.back()->range().left <= end_offset : file_segments.back()->range().contains(end_offset)); + chassert(file_segments_limit ? file_segments.back()->range().left <= range.right : file_segments.back()->range().contains(range.right)); } while (file_segments_limit && file_segments.size() > file_segments_limit) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index f8fd9635cd5..523ff90e33e 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -216,13 +216,17 @@ private: void loadMetadataImpl(); void loadMetadataForKeys(const std::filesystem::path & keys_dir); + /// bool - if `file_segments_limit` reached or not. FileSegments getImpl(const LockedKey & locked_key, const FileSegment::Range & range, size_t file_segments_limit) const; + std::vector splitRange(size_t offset, size_t size); + FileSegments splitRangeIntoFileSegments( LockedKey & locked_key, size_t offset, size_t size, FileSegment::State state, + size_t file_segments_limit, const CreateFileSegmentSettings & create_settings); void fillHolesWithEmptyFileSegments( diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 362103f3e22..59134229997 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -926,6 +926,7 @@ void FileSegment::use() FileSegmentsHolder::FileSegmentsHolder(FileSegments && file_segments_) : file_segments(std::move(file_segments_)) +{ CurrentMetrics::add(CurrentMetrics::FilesystemCacheHoldFileSegments, file_segments.size()); ProfileEvents::increment(ProfileEvents::FilesystemCacheHoldFileSegments, file_segments.size()); } diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index ab2a128de34..e1db07958fc 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -242,7 +242,7 @@ TEST_F(FileCacheTest, get) settings.max_elements = 5; settings.boundary_alignment = 1; - const size_t file_size = -1; // the value doesn't really matter because boundary_alignment == 1. + const size_t file_size = INT_MAX; // the value doesn't really matter because boundary_alignment == 1. { std::cerr << "Step 1\n"; From ca18674f1af01bf6463fe683fb8dcb4835b08dd2 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 8 Nov 2023 16:21:16 +0000 Subject: [PATCH 157/813] update test --- .../gtest_transform_query_for_external_database.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 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 749a154c19d..c40c9b0f5bc 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -279,9 +279,13 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) { const State & state = State::instance(); - check(state, 1, {"column"}, - "SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(toString(column), 10) = RIGHT(toString(column), 10) AND column IN (1, 42) AND SUBSTRING(toString(column) FROM 1 FOR 2) = 'Hello' AND column != 4", - R"(SELECT "column" FROM "test"."table" WHERE 1 AND ("column" = 42) AND ("column" IN (1, 42)) AND ("column" != 4))"); + check( + state, + 1, + {"column"}, + "SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(toString(column), 10) = " + "RIGHT(toString(column), 10) AND column IN (1, 42) AND SUBSTRING(toString(column) FROM 1 FOR 2) = 'Hello' AND column != 4", + R"(SELECT "column" FROM "test"."table" WHERE 1 = 1 AND ("column" = 42) AND ("column" IN (1, 42)) AND ("column" != 4))"); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE toString(column) = '42' AND left(toString(column), 10) = RIGHT(toString(column), 10) AND column = 42", R"(SELECT "column" FROM "test"."table" WHERE "column" = 42)"); From f1af83447e548db2ac28447486e3b141bba1bd12 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 8 Nov 2023 23:07:08 +0300 Subject: [PATCH 158/813] merge_row_policy: cleanup, traces removed --- src/Storages/StorageMerge.cpp | 191 ++---------------- src/Storages/StorageMerge.h | 2 +- src/TableFunctions/TableFunctionMerge.cpp | 2 +- .../02763_row_policy_storage_merge.reference | 58 ++++++ .../02763_row_policy_storage_merge.sql.j2 | 38 ++-- 5 files changed, 100 insertions(+), 191 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index d9460190677..5412a87fa01 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -385,7 +385,7 @@ public: /// Add to data stream columns that are needed only for row policies /// SELECT x from T if T has row policy y=42 /// required y in data pipeline - void extendNames(Names &, bool alias_allowed = true); + void extendNames(Names &); /// Use storage facilities to filter data /// optimization @@ -495,100 +495,13 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu if (sampling_requested && !storage->supportsSampling()) throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: table doesn't support sampling"); - // const auto & [database_name, _, _b, table_name] = table; - - // std::unique_ptr row_policy_data_ptr; - - // auto row_policy_filter_ptr = context->getRowPolicyFilter( - // database_name, - // table_name, - // RowPolicyFilterType::SELECT_FILTER); - // if (row_policy_filter_ptr) - // { - // row_policy_data_ptr = std::make_unique(row_policy_filter_ptr, storage, context); - // row_policy_data_ptr->extendNames(column_names); - // } - - - - // Aliases aliases; - // auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - // auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, context); - - // auto modified_query_info = getModifiedQueryInfo(query_info, context, table, nested_storage_snaphsot); - // Names column_names_as_aliases; - - // if (!context->getSettingsRef().allow_experimental_analyzer) - // { - // auto storage_columns = storage_metadata_snapshot->getColumns(); - // auto syntax_result = TreeRewriter(context).analyzeSelect( - // modified_query_info.query, TreeRewriterResult({}, storage, nested_storage_snaphsot)); - - // bool with_aliases = common_processed_stage == QueryProcessingStage::FetchColumns && !storage_columns.getAliases().empty(); - // if (with_aliases) - // { - // ASTPtr required_columns_expr_list = std::make_shared(); - // ASTPtr column_expr; - - // for (const auto & column : column_names) - // { - // const auto column_default = storage_columns.getDefault(column); - // bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias; - - // if (is_alias) - // { - // column_expr = column_default->expression->clone(); - // replaceAliasColumnsInQuery(column_expr, storage_metadata_snapshot->getColumns(), - // syntax_result->array_join_result_to_source, context); - - // const auto & column_description = storage_columns.get(column); - // column_expr = addTypeConversionToAST(std::move(column_expr), column_description.type->getName(), - // storage_metadata_snapshot->getColumns().getAll(), context); - // column_expr = setAlias(column_expr, column); - - // auto type = sample_block.getByName(column).type; - // aliases.push_back({ .name = column, .type = type, .expression = column_expr->clone() }); - - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), - // "adding new alias name {}, expression {}", - // column, column_expr->formatForLogging()); - - // } - // else - // column_expr = std::make_shared(column); - - // required_columns_expr_list->children.emplace_back(std::move(column_expr)); - // } - - // syntax_result = TreeRewriter(context).analyze( - // required_columns_expr_list, storage_columns.getAllPhysical(), storage, storage->getStorageSnapshot(storage_metadata_snapshot, context)); - - // auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); - - // column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); - // LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), - // "alias_actions->getRequiredColumns: {}", alias_actions->getRequiredColumns().toString()); - - // // if (row_policy_data_ptr) - // // row_policy_data_ptr->extendNames(column_names_as_aliases, false /* alias_allowed */); - - // if (column_names_as_aliases.empty()) - // column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); - // } - // } - auto source_pipeline = createSources( - // nested_storage_snaphsot, - // modified_query_info, common_processed_stage, required_max_block_size, common_header, - // aliases, table, - // column_names_as_aliases.empty() ? column_names : column_names_as_aliases, column_names, merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(), - // std::move(row_policy_data_ptr), context, current_streams); @@ -723,13 +636,9 @@ void ReadFromMerge::processAliases( storage_metadata_snapshot->getColumns().getAll(), context); column_expr = setAlias(column_expr, column); - auto type = sample_block.getByName(column).type; + auto type = sample_block.has(column) ? sample_block.getByName(column).type : column_description.type; + aliases.push_back({ .name = column, .type = type, .expression = column_expr->clone() }); - - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::initializePipeline"), - "adding new alias name {}, expression {}", - column, column_expr->formatForLogging()); - } else column_expr = std::make_shared(column); @@ -743,11 +652,6 @@ void ReadFromMerge::processAliases( auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::processAliases"), - "alias_actions->getRequiredColumns: {}", alias_actions->getRequiredColumns().toString()); - - // if (row_policy_data_ptr) - // row_policy_data_ptr->extendNames(column_names_as_aliases, false /* alias_allowed */); if (column_names_as_aliases.empty()) column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); @@ -755,23 +659,18 @@ void ReadFromMerge::processAliases( } if (!column_names_as_aliases.empty()) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::processAliases"), - "substitute real_column_names by column_names_as_aliases"); real_column_names = column_names_as_aliases; } } QueryPipelineBuilderPtr ReadFromMerge::createSources( - // const StorageSnapshotPtr & storage_snapshot, - // SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, const Block & header, const StorageWithLockAndName & storage_with_lock, Names real_column_names, const Block & sample_block, - // std::unique_ptr row_policy_data_ptr, ContextMutablePtr modified_context, size_t streams_num, bool concat_streams) @@ -795,13 +694,8 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( row_policy_data_ptr->extendNames(real_column_names); } - Aliases aliases; processAliases(real_column_names, storage_with_lock, aliases, sample_block, modified_context); - // if (row_policy_data_ptr) - // { - // row_policy_data_ptr->extendNames(real_column_names); - // } QueryPipelineBuilderPtr builder; if (!InterpreterSelectQuery::isQueryWithFinal(modified_query_info) && storage->needRewriteQueryWithFinal(real_column_names)) @@ -818,8 +712,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage_snapshot, modified_query_info); - // std::optional row_policy_data; - if (processed_stage <= storage_stage || (allow_experimental_analyzer && processed_stage == QueryProcessingStage::FetchColumns)) { /// If there are only virtual columns in query, you must request at least one other column. @@ -831,16 +723,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( StorageView * view = dynamic_cast(storage.get()); if (!view || allow_experimental_analyzer) { - // auto row_policy_filter_ptr = modified_context->getRowPolicyFilter( - // database_name, - // table_name, - // RowPolicyFilterType::SELECT_FILTER); - // if (row_policy_data_ptr) - // { - // row_policy_data.emplace(row_policy_filter_ptr, storage, modified_context); - // row_policy_data_ptr->extendNames(real_column_names); - // } - storage->read(plan, real_column_names, storage_snapshot, @@ -972,15 +854,14 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( }); } - // if (row_policy_data_ptr) - // { - // row_policy_data_ptr->addFilterTransform(*builder); - // } - /// 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. - convertingSourceStream(header, storage_snapshot->metadata, aliases, std::move(row_policy_data_ptr), modified_context, *builder, processed_stage); - + convertAndFilterSourceStream(header, + storage_snapshot->metadata, + aliases, std::move(row_policy_data_ptr), + modified_context, + *builder, + processed_stage); } return builder; @@ -991,9 +872,6 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter ContextPtr local_context) : row_policy_filter_ptr(row_policy_filter_ptr_) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData ctor"), - "storage {}", storage->getName()); - storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_columns = storage_metadata_snapshot->getColumns(); auto needed_columns = storage_columns.getAll/*Physical*/(); @@ -1017,70 +895,41 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter if (!deleted.empty() || added.size() != 1) { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot determine row level filter"); + "Cannot determine row level filter; {} columns deleted, {} columns added", + deleted.size(), added.size()); } filter_column_name = added.getNames().front(); } -void ReadFromMerge::RowPolicyData::extendNames(Names & names, bool alias_allowed) +void ReadFromMerge::RowPolicyData::extendNames(Names & names) { - std::sort(names.begin(), names.end()); + boost::container::flat_set names_set(names.begin(), names.end()); NameSet added_names; for (const auto & req_column : filter_actions->getRequiredColumns()) { - if (!std::binary_search(names.begin(), names.end(), req_column)) + if (!names_set.contains(req_column)) { - if (!alias_allowed) - { - auto storage_columns = storage_metadata_snapshot->getColumns(); - const auto column_default = storage_columns.getDefault(req_column); - bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias; - if (is_alias) - { - continue; - } - } - added_names.insert(req_column); + added_names.emplace(req_column); } } + if (!added_names.empty()) { std::copy(added_names.begin(), added_names.end(), std::back_inserter(names)); - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), - "{} names added", added_names.size()); - for (const auto & added_name : added_names) - { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::extendNames"), - " added name {}", added_name); - } } } void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::addStorageFilter"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), - filter_actions->getActionsDAG().dumpDAG(), - filter_actions->getSampleBlock().dumpStructure()); - step->addFilter(actions_dag, filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & builder) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::addFilterTransform"), "filter_actions_dag: {},<> {}, <> {}", - filter_actions->getActionsDAG().dumpNames(), - filter_actions->getActionsDAG().dumpDAG(), - filter_actions->getSampleBlock().dumpStructure()); - builder.addSimpleTransform([&](const Block & stream_header) { - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::RowPolicyData::addFilterTransform"), - "stream_header.dumpNames {}", stream_header.dumpNames()); - - return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove filter column */); }); } @@ -1255,7 +1104,7 @@ void StorageMerge::alter( setInMemoryMetadata(storage_metadata); } -void ReadFromMerge::convertingSourceStream( +void ReadFromMerge::convertAndFilterSourceStream( const Block & header, const StorageMetadataPtr & metadata_snapshot, const Aliases & aliases, @@ -1293,12 +1142,6 @@ void ReadFromMerge::convertingSourceStream( if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns) convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; - LOG_TRACE(&Poco::Logger::get("ReadFromMerge::convertingSourceStream"), - "builder.getHeader(): {}, header.getColumnsWithTypeAndName: {}", - builder.getHeader().dumpStructure(), - header.dumpStructure()); - - if (row_policy_data_ptr) { row_policy_data_ptr->addFilterTransform(builder); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 350d871359a..f7c31ab3ef8 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -206,7 +206,7 @@ private: size_t streams_num, bool concat_streams = false); - static void convertingSourceStream( + static void convertAndFilterSourceStream( const Block & header, const StorageMetadataPtr & metadata_snapshot, const Aliases & aliases, diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index d0457504144..599953a1add 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -126,7 +126,7 @@ ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr contex { auto storage = DatabaseCatalog::instance().tryGetTable(StorageID{db_with_tables.first, table}, context); if (storage) - return ColumnsDescription{storage->getInMemoryMetadataPtr()->getColumns().getAll/* Physical*/()}; // !!! + return ColumnsDescription{storage->getInMemoryMetadataPtr()->getColumns().getAllPhysical()}; } } diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference index 9c91a1652c7..0b7664deb7d 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference @@ -49,6 +49,10 @@ SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2 3 13 4 14 4 14 +SELECT * FROM engine_merge_12 WHERE x>2 +3 13 +4 14 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge) 1 11 1 11 @@ -162,6 +166,31 @@ SELECT x, y from merge(currentDatabase(), 02763_merge 4 14 4 14 4 14 +02763_merge_aliases +x, y, z FROM 02763_a_merge +3 13 16 +4 14 18 +* FROM 02763_a_merge +3 13 16 +4 14 18 +x, y FROM 02763_a_merge +3 13 +4 14 +SELECT x, y FROM merge(currentDatabase(), 02763_alias) +3 13 +4 14 +SELECT x, y FROM merge(currentDatabase(), 02763_alias) +2 12 +3 13 +4 14 +SELECT x FROM merge(currentDatabase(), 02763_alias) +12 +13 +14 +SELECT y FROM merge(currentDatabase(), 02763_alias) +2 +3 +4 SETTINGS optimize_move_to_prewhere= 1 SELECT * FROM 02763_merge_log_1 3 13 @@ -191,6 +220,10 @@ SELECT * FROM merge(currentDatabase(), 02763_merge_merge) WHERE x>2 3 13 4 14 4 14 +SELECT * FROM engine_merge_12 WHERE x>2 +3 13 +4 14 +4 14 SELECT * FROM merge(currentDatabase(), 02763_merge) 1 11 1 11 @@ -304,3 +337,28 @@ SELECT x, y from merge(currentDatabase(), 02763_merge 4 14 4 14 4 14 +02763_merge_aliases +x, y, z FROM 02763_a_merge +3 13 16 +4 14 18 +* FROM 02763_a_merge +3 13 16 +4 14 18 +x, y FROM 02763_a_merge +3 13 +4 14 +SELECT x, y FROM merge(currentDatabase(), 02763_alias) +3 13 +4 14 +SELECT x, y FROM merge(currentDatabase(), 02763_alias) +2 12 +3 13 +4 14 +SELECT x FROM merge(currentDatabase(), 02763_alias) +12 +13 +14 +SELECT y FROM merge(currentDatabase(), 02763_alias) +2 +3 +4 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index 0bc1292d4a5..a22a9c5b641 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -116,19 +116,19 @@ INSERT INTO 02763_merge_fancycols (x, y) SELECT x, y from merge(currentDatabase CREATE ROW POLICY 02763_filter_5 ON 02763_merge_fancycols USING cnst<>42 AS permissive TO ALL; SELECT 'SELECT *'; -SELECT * from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT * from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT x, lc'; -SELECT x, lc from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT x, lc from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; CREATE ROW POLICY 02763_filter_6 ON 02763_merge_fancycols USING lc='111' AS permissive TO ALL; SELECT 'SELECT *'; -SELECT * from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT * from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT x, lc'; -SELECT x, lc from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT x, lc from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT x, lc, cnst'; -SELECT x, lc, cnst from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x; +SELECT x, lc, cnst from merge(currentDatabase(), '02763_merge_fancycols') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; SELECT 'SELECT x, y from merge(currentDatabase(), 02763_merge'; -SELECT x, y from merge(currentDatabase(), '02763_merge') ORDER BY x; +SELECT x, y from merge(currentDatabase(), '02763_merge') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; DROP TABLE 02763_merge_fancycols; @@ -138,19 +138,26 @@ INSERT INTO 02763_alias VALUES (1, 11), (2, 12), (3, 13), (4, 14); CREATE ROW POLICY 02763_filter_7 ON 02763_alias USING z>15 AS permissive TO ALL; --- SELECT 'SELECT * FROM 02763_alias ORDER BY x'; --- SELECT x, y, z FROM 02763_alias ORDER BY x; - CREATE TABLE 02763_a_merge (x UInt8, y UInt64, z UInt64) ENGINE = Merge(currentDatabase(), '02763_alias'); --- SELECT 'SELECT * FROM merge(currentDatabase(), 02763_merge_alias) ORDER BY x'; --- SELECT * FROM merge(currentDatabase(), '02763_merge_alias') ORDER BY x; +SELECT 'x, y, z FROM 02763_a_merge'; +SELECT x, y, z FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT '* FROM 02763_a_merge'; +SELECT * FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'x, y FROM 02763_a_merge'; +SELECT x, y FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT x, y FROM merge(currentDatabase(), 02763_alias)'; +SELECT x, y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; -SELECT 'x, y, z FROM 02763_a_merge ORDER BY x'; -SELECT x, y, z FROM 02763_a_merge ORDER BY x; --- SELECT 'SELECT x, y, z FROM merge(currentDatabase(), 02763_merge_alias) ORDER BY x'; --- SELECT x, y, z FROM merge(currentDatabase(), '02763_merge_alias') ORDER BY x; +CREATE ROW POLICY 02763_filter_8 ON 02763_alias USING y>11 AS permissive TO ALL; + +SELECT 'SELECT x, y FROM merge(currentDatabase(), 02763_alias)'; +SELECT x, y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT x FROM merge(currentDatabase(), 02763_alias)'; +SELECT y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT y FROM merge(currentDatabase(), 02763_alias)'; +SELECT x FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; DROP TABLE 02763_alias; DROP TABLE 02763_a_merge; @@ -165,5 +172,6 @@ DROP ROW POLICY 02763_filter_5 ON 02763_merge_fancycols; DROP ROW POLICY 02763_filter_6 ON 02763_merge_fancycols; DROP ROW POLICY 02763_filter_7 ON 02763_alias; +DROP ROW POLICY 02763_filter_8 ON 02763_alias; {% endfor %} From 9c5f9f7dcb0e8ab646c6712def0a7384074d522d Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 9 Nov 2023 00:51:57 +0300 Subject: [PATCH 159/813] merge_row_policy: style fix, cleanup --- src/Storages/StorageMerge.cpp | 9 ++------- src/Storages/StorageMerge.h | 4 ---- 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5412a87fa01..5d5c4b716ec 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -376,7 +376,8 @@ void StorageMerge::read( /// A transient object of this helper class is created /// when processing a Merge table data source (subordinary table) -/// to guarantee that row policies are applied +/// that has row policies +/// to guarantee that these row policies are applied class ReadFromMerge::RowPolicyData { public: @@ -404,7 +405,6 @@ private: StorageMetadataPtr storage_metadata_snapshot; }; -// using RowPolicyDataPtr = std::unique_ptr; ReadFromMerge::ReadFromMerge( Block common_header_, @@ -473,8 +473,6 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu query_info.input_order_info = input_sorting_info; } - // auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); - std::vector> pipelines; QueryPlanResourceHolder resources; @@ -1135,8 +1133,6 @@ void ReadFromMerge::convertAndFilterSourceStream( }); } - - ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns) @@ -1158,7 +1154,6 @@ void ReadFromMerge::convertAndFilterSourceStream( { return std::make_shared(stream_header, actions); }); - } bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index f7c31ab3ef8..71680b90072 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -192,16 +192,12 @@ private: ContextMutablePtr modified_context); QueryPipelineBuilderPtr createSources( - // const StorageSnapshotPtr & storage_snapshot, - // SelectQueryInfo & query_info, QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, const Block & header, - // const Aliases & aliases, const StorageWithLockAndName & storage_with_lock, Names real_column_names, const Block & sample_block, - // std::unique_ptr row_policy_data_ptr, ContextMutablePtr modified_context, size_t streams_num, bool concat_streams = false); From 86685685d36c4a07c631b84589fcd34004a3877f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 7 Nov 2023 00:16:38 +0000 Subject: [PATCH 160/813] Fix segfault during Kerberos initialization --- src/Access/KerberosInit.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/KerberosInit.cpp b/src/Access/KerberosInit.cpp index 58e4a46f2aa..772938ad9b2 100644 --- a/src/Access/KerberosInit.cpp +++ b/src/Access/KerberosInit.cpp @@ -44,7 +44,7 @@ private: krb5_ccache defcache = nullptr; krb5_get_init_creds_opt * options = nullptr; // Credentials structure including ticket, session key, and lifetime info. - krb5_creds my_creds; + krb5_creds my_creds {}; krb5_keytab keytab = nullptr; krb5_principal defcache_princ = nullptr; String fmtError(krb5_error_code code) const; From 44eb73980f1c4bb42e57915d633794715d55e3c4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 9 Nov 2023 04:31:49 +0000 Subject: [PATCH 161/813] Fix startup failure due to TTL dependency --- src/Databases/DDLLoadingDependencyVisitor.cpp | 9 +++++++ src/Databases/DDLLoadingDependencyVisitor.h | 1 + .../02908_table_ttl_dependency.reference | 0 .../0_stateless/02908_table_ttl_dependency.sh | 24 +++++++++++++++++++ 4 files changed, 34 insertions(+) create mode 100644 tests/queries/0_stateless/02908_table_ttl_dependency.reference create mode 100755 tests/queries/0_stateless/02908_table_ttl_dependency.sh diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index 99538fd801e..fc362dd8578 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -7,6 +8,7 @@ #include #include #include +#include #include @@ -22,6 +24,7 @@ TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, c data.default_database = global_context->getCurrentDatabase(); data.create_query = ast; data.global_context = global_context; + data.table_name = table; TableLoadingDependenciesVisitor visitor{data}; visitor.visit(ast); data.dependencies.erase(table); @@ -113,6 +116,12 @@ void DDLLoadingDependencyVisitor::visit(const ASTFunctionWithKeyValueArguments & void DDLLoadingDependencyVisitor::visit(const ASTStorage & storage, Data & data) { + if (storage.ttl_table) + { + auto ttl_dependensies = getDependenciesFromCreateQuery(data.global_context, data.table_name, storage.ttl_table->ptr()); + data.dependencies.merge(ttl_dependensies); + } + if (!storage.engine) return; diff --git a/src/Databases/DDLLoadingDependencyVisitor.h b/src/Databases/DDLLoadingDependencyVisitor.h index f173517f852..a9e9f4d7a53 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.h +++ b/src/Databases/DDLLoadingDependencyVisitor.h @@ -38,6 +38,7 @@ public: TableNamesSet dependencies; ContextPtr global_context; ASTPtr create_query; + QualifiedTableName table_name; }; using Visitor = ConstInDepthNodeVisitor; diff --git a/tests/queries/0_stateless/02908_table_ttl_dependency.reference b/tests/queries/0_stateless/02908_table_ttl_dependency.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02908_table_ttl_dependency.sh b/tests/queries/0_stateless/02908_table_ttl_dependency.sh new file mode 100755 index 00000000000..70136b4a42b --- /dev/null +++ b/tests/queries/0_stateless/02908_table_ttl_dependency.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database +# Tag no-ordinary-database: requires UUID + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " + DROP TABLE IF EXISTS 02908_dependent; + DROP TABLE IF EXISTS 02908_main; + + CREATE TABLE 02908_main (a UInt32) ENGINE = MergeTree ORDER BY a; + CREATE TABLE 02908_dependent (a UInt32, ts DateTime) ENGINE = MergeTree ORDER BY a TTL ts + 1 WHERE a IN (SELECT a FROM ${CLICKHOUSE_DATABASE}.02908_main); +" + +$CLICKHOUSE_CLIENT -nm -q " + DROP TABLE 02908_main; +" 2>&1 | grep -F -q "HAVE_DEPENDENT_OBJECTS" + +$CLICKHOUSE_CLIENT -nm -q " + DROP TABLE 02908_dependent; + DROP TABLE 02908_main; +" From 41cdd5dd1a3e520d85f36be71c8fd69967ffee57 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 9 Nov 2023 06:23:23 +0000 Subject: [PATCH 162/813] Fix --- .../tests/gtest_transform_query_for_external_database.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c40c9b0f5bc..1b2a4187c94 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -285,7 +285,7 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) {"column"}, "SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(toString(column), 10) = " "RIGHT(toString(column), 10) AND column IN (1, 42) AND SUBSTRING(toString(column) FROM 1 FOR 2) = 'Hello' AND column != 4", - R"(SELECT "column" FROM "test"."table" WHERE 1 = 1 AND ("column" = 42) AND ("column" IN (1, 42)) AND ("column" != 4))"); + R"(SELECT "column" FROM "test"."table" WHERE (1 = 1) AND ("column" = 42) AND ("column" IN (1, 42)) AND ("column" != 4))"); check(state, 1, {"column"}, "SELECT column FROM test.table WHERE toString(column) = '42' AND left(toString(column), 10) = RIGHT(toString(column), 10) AND column = 42", R"(SELECT "column" FROM "test"."table" WHERE "column" = 42)"); From 4ac3dcc417bb629c9731aa7447d030aabf56b5a3 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 9 Nov 2023 07:07:41 +0000 Subject: [PATCH 163/813] Fix ALTER COLUMN with ALIAS --- src/Storages/AlterCommands.cpp | 2 +- .../0_stateless/02908_alter_column_alias.reference | 1 + tests/queries/0_stateless/02908_alter_column_alias.sql | 8 ++++++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02908_alter_column_alias.reference create mode 100644 tests/queries/0_stateless/02908_alter_column_alias.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index c6fa17583b5..3d6f4b864a8 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1148,7 +1148,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const /// The change of data type to/from Object is broken, so disable it for now if (command.data_type) { - const GetColumnsOptions options(GetColumnsOptions::AllPhysical); + const GetColumnsOptions options(GetColumnsOptions::All); const auto old_data_type = all_columns.getColumn(options, column_name).type; if (command.data_type->getName().contains("Object") diff --git a/tests/queries/0_stateless/02908_alter_column_alias.reference b/tests/queries/0_stateless/02908_alter_column_alias.reference new file mode 100644 index 00000000000..e44df6e9ff6 --- /dev/null +++ b/tests/queries/0_stateless/02908_alter_column_alias.reference @@ -0,0 +1 @@ +CREATE TABLE default.t\n(\n `c0` DateTime,\n `c1` DateTime,\n `a` DateTime ALIAS c1\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02908_alter_column_alias.sql b/tests/queries/0_stateless/02908_alter_column_alias.sql new file mode 100644 index 00000000000..fd98339e8b5 --- /dev/null +++ b/tests/queries/0_stateless/02908_alter_column_alias.sql @@ -0,0 +1,8 @@ +CREATE TABLE t ( + c0 DateTime, + c1 DateTime, + a DateTime alias toStartOfFifteenMinutes(c0) +) ENGINE = MergeTree() ORDER BY tuple(); + +ALTER TABLE t MODIFY COLUMN a DateTime ALIAS c1; +SHOW CREATE t; From 10ac68517e4303e3bf25697d45ff37f1d3c4eee7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 9 Nov 2023 07:59:50 +0000 Subject: [PATCH 164/813] Fix empty NAMED COLLECTIONs --- src/Common/NamedCollections/NamedCollectionUtils.cpp | 6 ++++++ src/Parsers/ASTAlterNamedCollectionQuery.cpp | 2 +- .../0_stateless/02908_empty_named_collection.reference | 0 tests/queries/0_stateless/02908_empty_named_collection.sql | 5 +++++ 4 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02908_empty_named_collection.reference create mode 100644 tests/queries/0_stateless/02908_empty_named_collection.sql diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index cab844d6213..c535c851464 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -217,6 +217,12 @@ public: for (const auto & [name, value] : result_changes_map) create_query.changes.emplace_back(name, value); + if (create_query.changes.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Named collection cannot be empty (collection name: {})", + query.collection_name); + writeCreateQueryToMetadata( create_query, getMetadataPath(query.collection_name), diff --git a/src/Parsers/ASTAlterNamedCollectionQuery.cpp b/src/Parsers/ASTAlterNamedCollectionQuery.cpp index 6363a7306bd..00f073b320a 100644 --- a/src/Parsers/ASTAlterNamedCollectionQuery.cpp +++ b/src/Parsers/ASTAlterNamedCollectionQuery.cpp @@ -14,7 +14,7 @@ ASTPtr ASTAlterNamedCollectionQuery::clone() const void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "Alter NAMED COLLECTION "; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ALTER NAMED COLLECTION "; if (if_exists) settings.ostr << "IF EXISTS "; settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : ""); diff --git a/tests/queries/0_stateless/02908_empty_named_collection.reference b/tests/queries/0_stateless/02908_empty_named_collection.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02908_empty_named_collection.sql b/tests/queries/0_stateless/02908_empty_named_collection.sql new file mode 100644 index 00000000000..6aab83858e8 --- /dev/null +++ b/tests/queries/0_stateless/02908_empty_named_collection.sql @@ -0,0 +1,5 @@ +-- Tags: no-parallel + +CREATE NAMED COLLECTION foobar03 AS a = 1; +ALTER NAMED COLLECTION foobar03 DELETE b; -- { serverError BAD_ARGUMENTS } +DROP NAMED COLLECTION foobar03; From f9895ab37b2133a36296b67b8904d251ffdaf3e4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 9 Nov 2023 15:56:57 +0000 Subject: [PATCH 165/813] Small fixes and add test --- src/Backups/BackupCoordinationRemote.cpp | 5 +- src/Backups/RestoreCoordinationRemote.cpp | 7 +- src/Common/ZooKeeper/ZooKeeper.cpp | 1 + src/Storages/StorageKeeperMap.cpp | 43 +++++-- src/Storages/StorageKeeperMap.h | 7 +- .../__init__.py | 0 .../configs/backups_disk.xml | 13 ++ .../configs/keeper_map_path_prefix.xml | 3 + .../configs/remote_servers.xml | 22 ++++ .../configs/zookeeper_retries.xml | 11 ++ .../test_backup_restore_keeper_map/test.py | 111 ++++++++++++++++++ .../02911_backup_restore_keeper_map.reference | 13 ++ .../02911_backup_restore_keeper_map.sh | 47 ++++++++ 13 files changed, 269 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_backup_restore_keeper_map/__init__.py create mode 100644 tests/integration/test_backup_restore_keeper_map/configs/backups_disk.xml create mode 100644 tests/integration/test_backup_restore_keeper_map/configs/keeper_map_path_prefix.xml create mode 100644 tests/integration/test_backup_restore_keeper_map/configs/remote_servers.xml create mode 100644 tests/integration/test_backup_restore_keeper_map/configs/zookeeper_retries.xml create mode 100644 tests/integration/test_backup_restore_keeper_map/test.py create mode 100644 tests/queries/0_stateless/02911_backup_restore_keeper_map.reference create mode 100755 tests/queries/0_stateless/02911_backup_restore_keeper_map.sh diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 309cbc8be6a..064e0599f6e 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -681,7 +681,10 @@ void BackupCoordinationRemote::addKeeperMapTable(const String & table_zookeeper_ { with_retries.renewZooKeeper(zk); String path = zookeeper_path + "/keeper_map_tables/" + escapeForFileName(table_id); - zk->create(path, fmt::format("{}\n{}", table_zookeeper_root_path, data_path_in_backup), zkutil::CreateMode::Persistent); + if (auto res + = zk->tryCreate(path, fmt::format("{}\n{}", table_zookeeper_root_path, data_path_in_backup), zkutil::CreateMode::Persistent); + res != Coordination::Error::ZOK && res != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperException(res); }); } diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 12a67d2a55d..1b814c2889e 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -244,9 +244,10 @@ bool RestoreCoordinationRemote::acquireInsertingDataForKeeperMap(const String & { with_retries.renewZooKeeper(zk); - fs::path base_path = fs::path(zookeeper_path) / "keeper_map_tables" / root_zk_path; - zk->createAncestors(base_path); - std::string restore_lock_path = base_path / "restore_lock"; + /// we need to remove leading '/' from root_zk_path + auto normalized_root_zk_path = std::string_view{root_zk_path}.substr(1); + std::string restore_lock_path = fs::path(zookeeper_path) / "keeper_map_tables" / normalized_root_zk_path / "restore_lock"; + zk->createAncestors(restore_lock_path); result = zk->tryCreate(restore_lock_path, "restorelock", zkutil::CreateMode::Persistent) == Coordination::Error::ZOK; if (result) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 436a4e14f14..8a97362aa96 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -385,6 +385,7 @@ void ZooKeeper::createAncestors(const std::string & path) size_t last_pos = path.rfind('/'); if (last_pos == std::string::npos || last_pos == 0) return; + std::string current_node = path.substr(0, last_pos); while (true) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 74c1905cd61..15ebc4d92d1 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -15,7 +15,7 @@ #include #include -#include +#include #include #include @@ -52,6 +52,8 @@ #include #include +#include + #include #include @@ -824,16 +826,24 @@ void StorageKeeperMap::restoreDataFromBackup(RestorerFromBackup & restorer, cons RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); } - /// TODO: Should we backup and verify the table structure? + auto temp_disk = restorer.getContext()->getGlobalTemporaryVolume()->getDisk(0); - //auto temp_disk = restorer.getContext()->getGlobalTemporaryVolume()->getDisk(0); /// only 1 table should restore data for a single path restorer.addDataRestoreTask( - [storage = std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, with_retries, allow_non_empty_tables] - { storage->restoreDataImpl(backup, data_path_in_backup, with_retries, allow_non_empty_tables); }); + [storage = std::static_pointer_cast(shared_from_this()), + backup, + data_path_in_backup, + with_retries, + allow_non_empty_tables, + temp_disk] { storage->restoreDataImpl(backup, data_path_in_backup, with_retries, allow_non_empty_tables, temp_disk); }); } -void StorageKeeperMap::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, std::shared_ptr with_retries, bool allow_non_empty_tables) +void StorageKeeperMap::restoreDataImpl( + const BackupPtr & backup, + const String & data_path_in_backup, + std::shared_ptr with_retries, + bool allow_non_empty_tables, + const DiskPtr & temporary_disk) { auto table_id = toString(getStorageID().uuid); @@ -858,7 +868,17 @@ void StorageKeeperMap::restoreDataImpl(const BackupPtr & backup, const String & /// should we store locally in temp file? auto in = backup->readFile(data_file); - CompressedReadBuffer compressed_in{*in}; + std::optional temp_data_file; + if (!dynamic_cast(in.get())) + { + temp_data_file.emplace(temporary_disk); + auto out = std::make_unique(temp_data_file->getAbsolutePath()); + copyData(*in, *out); + out.reset(); + in = createReadBufferFromFileBase(temp_data_file->getAbsolutePath(), {}); + } + std::unique_ptr in_from_file{static_cast(in.release())}; + CompressedReadBufferFromFile compressed_in{std::move(in_from_file)}; fs::path data_path_fs(zk_data_path); auto max_multi_size = with_retries->getKeeperSettings().batch_size_for_keeper_multi; @@ -871,7 +891,10 @@ void StorageKeeperMap::restoreDataImpl(const BackupPtr & backup, const String & [&, &zk = holder.faulty_zookeeper]() { with_retries->renewZooKeeper(zk); - zk->multi(create_requests); + Coordination::Responses create_responses; + if (auto res = zk->tryMulti(create_requests, create_responses); + res != Coordination::Error::ZOK && res != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperMultiException(res, create_requests, create_responses); }); }; @@ -890,7 +913,9 @@ void StorageKeeperMap::restoreDataImpl(const BackupPtr & backup, const String & [&, &zk = holder.faulty_zookeeper]() { with_retries->renewZooKeeper(zk); - zk->tryCreate(data_path_fs / key, value, zkutil::CreateMode::Persistent); + if (auto res = zk->tryCreate(data_path_fs / key, value, zkutil::CreateMode::Persistent); + res != Coordination::Error::ZOK && res != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperException::fromPath(res, data_path_fs / key); }); } /// otherwise we can do multi requests diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 94b02ca0242..10eebdd0129 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -120,7 +120,12 @@ private: std::optional isTableValid() const; - void restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, std::shared_ptr with_retries, bool allow_non_empty_tables); + void restoreDataImpl( + const BackupPtr & backup, + const String & data_path_in_backup, + std::shared_ptr with_retries, + bool allow_non_empty_tables, + const DiskPtr & temporary_disk); std::string zk_root_path; std::string primary_key; diff --git a/tests/integration/test_backup_restore_keeper_map/__init__.py b/tests/integration/test_backup_restore_keeper_map/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_backup_restore_keeper_map/configs/backups_disk.xml b/tests/integration/test_backup_restore_keeper_map/configs/backups_disk.xml new file mode 100644 index 00000000000..b99a51cd56d --- /dev/null +++ b/tests/integration/test_backup_restore_keeper_map/configs/backups_disk.xml @@ -0,0 +1,13 @@ + + + + + local + /backups/ + + + + + backups + + diff --git a/tests/integration/test_backup_restore_keeper_map/configs/keeper_map_path_prefix.xml b/tests/integration/test_backup_restore_keeper_map/configs/keeper_map_path_prefix.xml new file mode 100644 index 00000000000..91d7b9d3f8f --- /dev/null +++ b/tests/integration/test_backup_restore_keeper_map/configs/keeper_map_path_prefix.xml @@ -0,0 +1,3 @@ + + /keeper_map_tables + diff --git a/tests/integration/test_backup_restore_keeper_map/configs/remote_servers.xml b/tests/integration/test_backup_restore_keeper_map/configs/remote_servers.xml new file mode 100644 index 00000000000..5cf07c69fd6 --- /dev/null +++ b/tests/integration/test_backup_restore_keeper_map/configs/remote_servers.xml @@ -0,0 +1,22 @@ + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + node3 + 9000 + + + + + diff --git a/tests/integration/test_backup_restore_keeper_map/configs/zookeeper_retries.xml b/tests/integration/test_backup_restore_keeper_map/configs/zookeeper_retries.xml new file mode 100644 index 00000000000..1283f28a8cb --- /dev/null +++ b/tests/integration/test_backup_restore_keeper_map/configs/zookeeper_retries.xml @@ -0,0 +1,11 @@ + + + + 1000 + 1 + 1 + 42 + 0.002 + + + diff --git a/tests/integration/test_backup_restore_keeper_map/test.py b/tests/integration/test_backup_restore_keeper_map/test.py new file mode 100644 index 00000000000..95e8a8b3027 --- /dev/null +++ b/tests/integration/test_backup_restore_keeper_map/test.py @@ -0,0 +1,111 @@ +from time import sleep +import pytest +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) + +main_configs = [ + "configs/remote_servers.xml", + "configs/backups_disk.xml", + "configs/keeper_map_path_prefix.xml", +] + +user_configs = [ + "configs/zookeeper_retries.xml", +] + +node1 = cluster.add_instance( + "node1", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "node1", "shard": "shard1"}, + with_zookeeper=True, + stay_alive=True, +) + +node2 = cluster.add_instance( + "node2", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "node2", "shard": "shard1"}, + with_zookeeper=True, + stay_alive=True, +) + + +node3 = cluster.add_instance( + "node3", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "node3", "shard": "shard2"}, + with_zookeeper=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +backup_id_counter = 0 + +def new_backup_name(base_name): + global backup_id_counter + backup_id_counter += 1 + return f"Disk('backups', '{base_name}{backup_id_counter}')" + +def test_on_cluster(): + node1.query_with_retry("CREATE DATABASE keeper_backup ON CLUSTER cluster") + node1.query_with_retry("CREATE TABLE keeper_backup.keeper1 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster1') PRIMARY KEY key") + node1.query_with_retry("CREATE TABLE keeper_backup.keeper2 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster1') PRIMARY KEY key") + node1.query_with_retry("CREATE TABLE keeper_backup.keeper3 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster2') PRIMARY KEY key") + node1.query_with_retry("INSERT INTO keeper_backup.keeper2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5") + node1.query_with_retry("INSERT INTO keeper_backup.keeper3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5") + + expected_result = ''.join(f'{i}\ttest{i}\n' for i in range(5)) + + def verify_data(): + for node in [node1, node2, node3]: + for i in range(1, 4): + result = node.query_with_retry(f'SELECT key, value FROM keeper_backup.keeper{i} ORDER BY key FORMAT TSV') + assert result == expected_result + + verify_data() + + backup_name = new_backup_name('test_on_cluster') + node1.query(f"BACKUP DATABASE keeper_backup ON CLUSTER cluster TO {backup_name} SETTINGS async = false;") + + node1.query("DROP DATABASE keeper_backup ON CLUSTER cluster SYNC;") + + def apply_for_all_nodes(f): + for node in [node1, node2, node3]: + f(node) + + def change_keeper_map_prefix(node): + node.replace_config( + "/etc/clickhouse-server/config.d/keeper_map_path_prefix.xml", """ + + /different_path/keeper_map + +""") + + apply_for_all_nodes(lambda node: node.stop_clickhouse()) + apply_for_all_nodes(change_keeper_map_prefix) + apply_for_all_nodes(lambda node: node.start_clickhouse()) + + node1.query(f"RESTORE DATABASE keeper_backup ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;") + + verify_data() + + node1.query("DROP TABLE keeper_backup.keeper3 ON CLUSTER cluster SYNC;") + node1.query(f"RESTORE TABLE keeper_backup.keeper3 ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;") + + verify_data() \ No newline at end of file diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.reference b/tests/queries/0_stateless/02911_backup_restore_keeper_map.reference new file mode 100644 index 00000000000..e58335de67c --- /dev/null +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.reference @@ -0,0 +1,13 @@ +5000 +5000 +3000 +OK +OK +OK +5000 +5000 +3000 +OK +5000 +5000 +3000 diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh new file mode 100755 index 00000000000..6c463beb221 --- /dev/null +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " + DROP DATABASE IF EXISTS 02911_keeper_map; + CREATE DATABASE 02911_keeper_map; + CREATE TABLE 02911_keeper_map.02911_backup_restore_keeper_map1 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; + CREATE TABLE 02911_keeper_map.02911_backup_restore_keeper_map2 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; + CREATE TABLE 02911_keeper_map.02911_backup_restore_keeper_map3 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911_different') PRIMARY KEY key; + + INSERT INTO 02911_keeper_map.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000; + INSERT INTO 02911_keeper_map.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000; +" + +backup_path="$CLICKHOUSE_DATABASE/02911_keeper_map" +for i in $(seq 1 3); do + $CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map$i;" +done + +$CLICKHOUSE_CLIENT -q "BACKUP DATABASE 02911_keeper_map TO Disk('backups', '$backup_path');" > /dev/null + +$CLICKHOUSE_CLIENT -q "DROP DATABASE 02911_keeper_map SYNC;" + +for i in $(seq 1 3); do + $CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map$i;" 2>&1 | grep -Fq "UNKNOWN_DATABASE" && echo 'OK' || echo 'ERROR' +done + +$CLICKHOUSE_CLIENT -q "RESTORE DATABASE 02911_keeper_map FROM Disk('backups', '$backup_path');" > /dev/null + +for i in $(seq 1 3); do + $CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map$i;" +done + +$CLICKHOUSE_CLIENT -q "DROP TABLE 02911_keeper_map.02911_backup_restore_keeper_map3 SYNC;" + +$CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map3;" 2>&1 | grep -Fq "UNKNOWN_TABLE" && echo 'OK' || echo 'ERROR' + +$CLICKHOUSE_CLIENT -q "RESTORE TABLE 02911_keeper_map.02911_backup_restore_keeper_map3 FROM Disk('backups', '$backup_path');" > /dev/null + +for i in $(seq 1 3); do + $CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map$i;" +done + +$CLICKHOUSE_CLIENT -q "DROP DATABASE 02911_keeper_map SYNC;" \ No newline at end of file From ae09b16701624825da053a1b0501c20243e71fd3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 9 Nov 2023 17:05:11 +0100 Subject: [PATCH 166/813] Debug logging --- src/Interpreters/Cache/FileCache.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 1a09db7a3f0..ed8cc547fbc 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -395,6 +395,14 @@ void FileCache::fillHolesWithEmptyFileSegments( return; } + if (file_segments_limit && file_segments.size() >= file_segments_limit) + { + std::string res; + for (const auto & f : file_segments) + res += " - " + f->range().toString(); + LOG_ERROR(log, "Limit: {}, file segments: {}, added: {}, range: {}, file_segments: {}", + file_segments_limit, file_segments.size(), added, range.toString(), res); + } chassert(!file_segments_limit || file_segments.size() < file_segments_limit); if (current_pos <= range.right) From 188a88fa3391cd2044fce96d00da12c22b654319 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 9 Nov 2023 16:15:14 +0000 Subject: [PATCH 167/813] Automatic style fix --- .../test_backup_restore_keeper_map/test.py | 59 +++++++++++++------ 1 file changed, 41 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_backup_restore_keeper_map/test.py b/tests/integration/test_backup_restore_keeper_map/test.py index 95e8a8b3027..8343ad3177f 100644 --- a/tests/integration/test_backup_restore_keeper_map/test.py +++ b/tests/integration/test_backup_restore_keeper_map/test.py @@ -55,33 +55,50 @@ def start_cluster(): finally: cluster.shutdown() + backup_id_counter = 0 + def new_backup_name(base_name): global backup_id_counter backup_id_counter += 1 return f"Disk('backups', '{base_name}{backup_id_counter}')" + def test_on_cluster(): node1.query_with_retry("CREATE DATABASE keeper_backup ON CLUSTER cluster") - node1.query_with_retry("CREATE TABLE keeper_backup.keeper1 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster1') PRIMARY KEY key") - node1.query_with_retry("CREATE TABLE keeper_backup.keeper2 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster1') PRIMARY KEY key") - node1.query_with_retry("CREATE TABLE keeper_backup.keeper3 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster2') PRIMARY KEY key") - node1.query_with_retry("INSERT INTO keeper_backup.keeper2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5") - node1.query_with_retry("INSERT INTO keeper_backup.keeper3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5") + node1.query_with_retry( + "CREATE TABLE keeper_backup.keeper1 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster1') PRIMARY KEY key" + ) + node1.query_with_retry( + "CREATE TABLE keeper_backup.keeper2 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster1') PRIMARY KEY key" + ) + node1.query_with_retry( + "CREATE TABLE keeper_backup.keeper3 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster2') PRIMARY KEY key" + ) + node1.query_with_retry( + "INSERT INTO keeper_backup.keeper2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5" + ) + node1.query_with_retry( + "INSERT INTO keeper_backup.keeper3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5" + ) - expected_result = ''.join(f'{i}\ttest{i}\n' for i in range(5)) + expected_result = "".join(f"{i}\ttest{i}\n" for i in range(5)) def verify_data(): for node in [node1, node2, node3]: for i in range(1, 4): - result = node.query_with_retry(f'SELECT key, value FROM keeper_backup.keeper{i} ORDER BY key FORMAT TSV') + result = node.query_with_retry( + f"SELECT key, value FROM keeper_backup.keeper{i} ORDER BY key FORMAT TSV" + ) assert result == expected_result verify_data() - backup_name = new_backup_name('test_on_cluster') - node1.query(f"BACKUP DATABASE keeper_backup ON CLUSTER cluster TO {backup_name} SETTINGS async = false;") + backup_name = new_backup_name("test_on_cluster") + node1.query( + f"BACKUP DATABASE keeper_backup ON CLUSTER cluster TO {backup_name} SETTINGS async = false;" + ) node1.query("DROP DATABASE keeper_backup ON CLUSTER cluster SYNC;") @@ -91,21 +108,27 @@ def test_on_cluster(): def change_keeper_map_prefix(node): node.replace_config( - "/etc/clickhouse-server/config.d/keeper_map_path_prefix.xml", """ + "/etc/clickhouse-server/config.d/keeper_map_path_prefix.xml", + """ /different_path/keeper_map -""") +""", + ) apply_for_all_nodes(lambda node: node.stop_clickhouse()) apply_for_all_nodes(change_keeper_map_prefix) apply_for_all_nodes(lambda node: node.start_clickhouse()) - node1.query(f"RESTORE DATABASE keeper_backup ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;") - - verify_data() - - node1.query("DROP TABLE keeper_backup.keeper3 ON CLUSTER cluster SYNC;") - node1.query(f"RESTORE TABLE keeper_backup.keeper3 ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;") + node1.query( + f"RESTORE DATABASE keeper_backup ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;" + ) - verify_data() \ No newline at end of file + verify_data() + + node1.query("DROP TABLE keeper_backup.keeper3 ON CLUSTER cluster SYNC;") + node1.query( + f"RESTORE TABLE keeper_backup.keeper3 ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;" + ) + + verify_data() From 65ed229400800e9910c8590153baa7f65ad45c16 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Nov 2023 16:24:17 +0000 Subject: [PATCH 168/813] Add a test with MOVE PARTITION and inactive replica. --- ..._move_partition_inactive_replica.reference | 1 + .../02915_move_partition_inactive_replica.sql | 57 +++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 tests/queries/0_stateless/02915_move_partition_inactive_replica.reference create mode 100644 tests/queries/0_stateless/02915_move_partition_inactive_replica.sql diff --git a/tests/queries/0_stateless/02915_move_partition_inactive_replica.reference b/tests/queries/0_stateless/02915_move_partition_inactive_replica.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02915_move_partition_inactive_replica.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02915_move_partition_inactive_replica.sql b/tests/queries/0_stateless/02915_move_partition_inactive_replica.sql new file mode 100644 index 00000000000..3b30a2b6c2c --- /dev/null +++ b/tests/queries/0_stateless/02915_move_partition_inactive_replica.sql @@ -0,0 +1,57 @@ +-- Tags: no-parallel + +create database if not exists shard_0; +create database if not exists shard_1; + +drop table if exists shard_0.from_0; +drop table if exists shard_1.from_0; +drop table if exists shard_0.from_1; +drop table if exists shard_1.from_1; +drop table if exists shard_0.to; +drop table if exists shard_1.to; + +create table shard_0.from_0 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_0_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; +create table shard_1.from_0 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_0_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; + +create table shard_0.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; +create table shard_1.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; + +insert into shard_0.from_0 select number from numbers(10); +insert into shard_0.from_0 select number + 10 from numbers(10); + +insert into shard_0.from_1 select number + 20 from numbers(10); +insert into shard_0.from_1 select number + 30 from numbers(10); + +system sync replica shard_1.from_0; +system sync replica shard_1.from_1; + + +create table shard_0.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; + +create table shard_1.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; + +detach table shard_1.to; + +alter table shard_0.from_0 on cluster test_cluster_two_shards_different_databases move partition tuple() to table shard_0.to format Null settings distributed_ddl_output_mode='never_throw', distributed_ddl_task_timeout = 1; + +alter table shard_0.from_1 on cluster test_cluster_two_shards_different_databases move partition tuple() to table shard_0.to format Null settings distributed_ddl_output_mode='never_throw', distributed_ddl_task_timeout = 1; + +OPTIMIZE TABLE shard_0.from_0; +OPTIMIZE TABLE shard_1.from_0; +OPTIMIZE TABLE shard_0.from_1; +OPTIMIZE TABLE shard_1.from_1; +OPTIMIZE TABLE shard_0.to; + +system restart replica shard_0.to; + +select sleep(2); + +attach table shard_1.to; + +drop table if exists shard_0.from_0; +drop table if exists shard_1.from_0; +drop table if exists shard_0.from_1; +drop table if exists shard_1.from_1; +drop table if exists shard_0.to; +drop table if exists shard_1.to; + From 6fb7d44b62277c7c51e540c68c0c54a87bc24423 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 9 Nov 2023 17:30:53 +0300 Subject: [PATCH 169/813] Analyzer support EXPLAIN ESTIMATE --- src/Interpreters/InterpreterExplainQuery.cpp | 18 ++++++++++++++---- src/Storages/StorageDistributed.cpp | 3 --- .../02911_analyzer_explain_estimate.reference | 0 .../02911_analyzer_explain_estimate.sql | 5 +++++ 4 files changed, 19 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02911_analyzer_explain_estimate.reference create mode 100644 tests/queries/0_stateless/02911_analyzer_explain_estimate.sql diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 39cc4df5c2d..7ec17292055 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -536,11 +536,21 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; - ContextPtr context; + ContextPtr context = getContext(); + + if (context->getSettingsRef().allow_experimental_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); + plan = std::move(interpreter).extractQueryPlan(); + context = interpreter.getContext(); + } + else + { + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); + interpreter.buildQueryPlan(plan); + context = interpreter.getContext(); + } - InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); - interpreter.buildQueryPlan(plan); - context = interpreter.getContext(); // Collect the selected marks, rows, parts during build query pipeline. // Hold on to the returned QueryPipelineBuilderPtr because `plan` may have pointers into // it (through QueryPlanResourceHolder). diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 7705d0f193f..812ce40eee3 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -532,9 +532,6 @@ std::optional StorageDistributed::getOptimizedQueryP const auto & query_node = query_info.query_tree->as(); - // std::cerr << query_node.dumpTree() << std::endl; - // std::cerr << query_info.table_expression->dumpTree() << std::endl; - auto expr_contains_sharding_key = [&](const ListNode & exprs) -> bool { std::unordered_set expr_columns; diff --git a/tests/queries/0_stateless/02911_analyzer_explain_estimate.reference b/tests/queries/0_stateless/02911_analyzer_explain_estimate.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02911_analyzer_explain_estimate.sql b/tests/queries/0_stateless/02911_analyzer_explain_estimate.sql new file mode 100644 index 00000000000..b082f2f33b2 --- /dev/null +++ b/tests/queries/0_stateless/02911_analyzer_explain_estimate.sql @@ -0,0 +1,5 @@ +-- Tags: distributed + +SET allow_experimental_analyzer = 1; + +EXPLAIN ESTIMATE SELECT 0 = 1048577, NULL, groupBitmapOr(bitmapBuild([toInt32(65537)])) FROM cluster(test_cluster_two_shards) WHERE NULL = 1048575; From 124af73f1d51dd52b26ac6cb697c34a548dff29e Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Thu, 9 Nov 2023 17:54:45 +0100 Subject: [PATCH 170/813] Add support of arbitrary types to concat --- src/Functions/concat.cpp | 35 ++++++++++++++------------- src/Functions/concatWithSeparator.cpp | 2 +- src/Functions/formatString.cpp | 2 +- src/Functions/formatString.h | 2 +- 4 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 9eb222d8c09..350cbee58a3 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -56,18 +57,6 @@ public: getName(), arguments.size()); - for (const auto arg_idx : collections::range(0, arguments.size())) - { - const auto * arg = arguments[arg_idx].get(); - if (!isStringOrFixedString(arg)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument {} of function {}", - arg->getName(), - arg_idx + 1, - getName()); - } - return std::make_shared(); } @@ -76,7 +65,7 @@ public: /// Format function is not proven to be faster for two arguments. /// Actually there is overhead of 2 to 5 extra instructions for each string for checking empty strings in FormatImpl. /// Though, benchmarks are really close, for most examples we saw executeBinary is slightly faster (0-3%). - /// For 3 and more arguments FormatImpl is much faster (up to 50-60%). + /// For 3 and more arguments FormatStringImpl is much faster (up to 50-60%). if (arguments.size() == 2) return executeBinary(arguments, input_rows_count); else @@ -107,6 +96,7 @@ private: else { /// Fallback: use generic implementation for not very important cases. + /// Concat of arbitrary types also goes here. return executeFormatImpl(arguments, input_rows_count); } @@ -145,8 +135,18 @@ private: constant_strings[i] = const_col->getValue(); } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - column->getName(), getName()); + { + // An arbitrary type argument: converting it to a StringColumn as if `toString` was called + ColumnsWithTypeAndName args; + args.emplace_back(column, arguments[i].type, "tmp"); + const ColumnPtr converted_col_ptr = ConvertImplGenericToString::execute( + args, std::make_shared(), column->size()); + const ColumnString * converted_col_str = assert_cast(converted_col_ptr.get()); + // Same as the normal `ColumnString` branch + has_column_string = true; + data[i] = &converted_col_str->getChars(); + offsets[i] = &converted_col_str->getOffsets(); + } } String pattern; @@ -155,7 +155,7 @@ private: for (size_t i = 0; i < num_arguments; ++i) pattern += "{}"; - FormatImpl::formatExecute( + FormatStringImpl::formatExecute( has_column_string, has_column_fixed_string, std::move(pattern), @@ -185,7 +185,8 @@ using FunctionConcat = ConcatImpl; using FunctionConcatAssumeInjective = ConcatImpl; -/// Also works with arrays. +/// Works with arrays via `arrayConcat`, maps via `mapConcat`, and tuples via `tupleConcat`. +/// Additionally, allows concatenation of arbitrary types that can be cast to string using the corresponding default serialization. class ConcatOverloadResolver : public IFunctionOverloadResolver { public: diff --git a/src/Functions/concatWithSeparator.cpp b/src/Functions/concatWithSeparator.cpp index bfd1bc392db..f0a7afbbaa7 100644 --- a/src/Functions/concatWithSeparator.cpp +++ b/src/Functions/concatWithSeparator.cpp @@ -122,7 +122,7 @@ public: for (size_t i = 0; i < num_args; ++i) pattern += "{}"; - FormatImpl::formatExecute( + FormatStringImpl::formatExecute( has_column_string, has_column_fixed_string, std::move(pattern), diff --git a/src/Functions/formatString.cpp b/src/Functions/formatString.cpp index ee6e26b775a..8e0b3a238cb 100644 --- a/src/Functions/formatString.cpp +++ b/src/Functions/formatString.cpp @@ -110,7 +110,7 @@ public: column->getName(), getName()); } - FormatImpl::formatExecute( + FormatStringImpl::formatExecute( has_column_string, has_column_fixed_string, std::move(pattern), diff --git a/src/Functions/formatString.h b/src/Functions/formatString.h index 44fbdac9378..30149e9a5b0 100644 --- a/src/Functions/formatString.h +++ b/src/Functions/formatString.h @@ -18,7 +18,7 @@ namespace DB { -struct FormatImpl +struct FormatStringImpl { static constexpr size_t right_padding = 15; From c28c1ac4a2932fd652280b8bbe087a33e4d7ff2c Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 9 Nov 2023 18:22:24 +0100 Subject: [PATCH 171/813] Allow getting cache configuration from named collection --- .../Cached/registerDiskCache.cpp | 7 ++- src/Interpreters/Cache/FileCacheSettings.cpp | 43 ++++++++++++++++--- src/Interpreters/Cache/FileCacheSettings.h | 2 + tests/config/config.d/named_collection.xml | 4 ++ ...8_filesystem_cache_as_collection.reference | 2 + .../02908_filesystem_cache_as_collection.sql | 8 ++++ 6 files changed, 60 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference create mode 100644 tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql diff --git a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp index 2f80b4c9efd..182326bbdc3 100644 --- a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp +++ b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -39,7 +40,11 @@ void registerDiskCache(DiskFactory & factory, bool /* global_skip_access_check * } FileCacheSettings file_cache_settings; - file_cache_settings.loadFromConfig(config, config_prefix); + auto predefined_configuration = config.has("cache_name") ? NamedCollectionFactory::instance().tryGet(config.getString("cache_name")) : nullptr; + if (predefined_configuration) + file_cache_settings.loadFromCollection(*predefined_configuration); + else + file_cache_settings.loadFromConfig(config, config_prefix); auto config_fs_caches_dir = context->getFilesystemCachesPath(); if (config_fs_caches_dir.empty()) diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 6f2f8c4b778..e81bd9ddc35 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -26,10 +27,6 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & if (max_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected non-zero size for cache configuration"); - auto path = config.getString(config_prefix + ".path", ""); - if (path.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk Cache requires non-empty `path` field (cache base path) in config"); - max_elements = config.getUInt64(config_prefix + ".max_elements", FILECACHE_DEFAULT_MAX_ELEMENTS); if (config.has(config_prefix + ".max_file_segment_size")) @@ -38,7 +35,6 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & cache_on_write_operations = config.getUInt64(config_prefix + ".cache_on_write_operations", false); enable_filesystem_query_cache_limit = config.getUInt64(config_prefix + ".enable_filesystem_query_cache_limit", false); cache_hits_threshold = config.getUInt64(config_prefix + ".cache_hits_threshold", FILECACHE_DEFAULT_HITS_THRESHOLD); - enable_bypass_cache_with_threshold = config.getUInt64(config_prefix + ".enable_bypass_cache_with_threshold", false); if (config.has(config_prefix + ".bypass_cache_threshold")) @@ -54,4 +50,41 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & load_metadata_threads = config.getUInt(config_prefix + ".load_metadata_threads"); } +void FileCacheSettings::loadFromCollection(const NamedCollection & collection) +{ + if (!collection.has("path")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected cache path (`path`) in configuration"); + + base_path = collection.get("path"); + + if (!collection.has("max_size")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected cache size (`max_size`) in configuration"); + + max_size = parseWithSizeSuffix(collection.get("max_size")); + if (max_size == 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected non-zero size for cache configuration"); + + max_elements = collection.getOrDefault("max_elements", FILECACHE_DEFAULT_MAX_ELEMENTS); + + if (collection.has("max_file_segment_size")) + max_file_segment_size = parseWithSizeSuffix(collection.get("max_file_segment_size")); + + cache_on_write_operations = collection.getOrDefault("cache_on_write_operations", false); + enable_filesystem_query_cache_limit = collection.getOrDefault("enable_filesystem_query_cache_limit", false); + cache_hits_threshold = collection.getOrDefault("cache_hits_threshold", FILECACHE_DEFAULT_HITS_THRESHOLD); + enable_bypass_cache_with_threshold = collection.getOrDefault("enable_bypass_cache_with_threshold", false); + + if (collection.has("bypass_cache_threshold")) + bypass_cache_threshold = parseWithSizeSuffix(collection.get("bypass_cache_threshold")); + + if (collection.has("boundary_alignment")) + boundary_alignment = parseWithSizeSuffix(collection.get("boundary_alignment")); + + if (collection.has("background_download_threads")) + background_download_threads = collection.get("background_download_threads"); + + if (collection.has("load_metadata_threads")) + load_metadata_threads = collection.get("load_metadata_threads"); +} + } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index 9888b814a0b..9d15a32d3bf 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -8,6 +8,7 @@ namespace Poco { namespace Util { class AbstractConfiguration; } } // NOLINT(cpp namespace DB { +class NamedCollection; struct FileCacheSettings { @@ -31,6 +32,7 @@ struct FileCacheSettings size_t load_metadata_threads = FILECACHE_DEFAULT_LOAD_METADATA_THREADS; void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); + void loadFromCollection(const NamedCollection & collection); }; } diff --git a/tests/config/config.d/named_collection.xml b/tests/config/config.d/named_collection.xml index 5b716a7b8da..2c4b349a183 100644 --- a/tests/config/config.d/named_collection.xml +++ b/tests/config/config.d/named_collection.xml @@ -37,5 +37,9 @@ test testtest + + 1Mi + collection + diff --git a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference new file mode 100644 index 00000000000..f5e0af6d507 --- /dev/null +++ b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference @@ -0,0 +1,2 @@ +1048576 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection_sql 2 0 1 +1048576 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection 2 0 1 diff --git a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql new file mode 100644 index 00000000000..e878727359b --- /dev/null +++ b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql @@ -0,0 +1,8 @@ +CREATE NAMED COLLECTION IF NOT EXISTS cache_collection_sql AS path = 'collection_sql', max_size = '1Mi'; +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int32, b String) +ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 's3_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME', cache_name='cache_collection_sql'); +DESCRIBE FILESYSTEM CACHE '$CLICHOUSE_TEST_UNIQUE_NAME'; +CREATE TABLE test2 (a Int32, b String) +ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 's3_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME_2', cache_name='cache_collection'); +DESCRIBE FILESYSTEM CACHE '$CLICHOUSE_TEST_UNIQUE_NAME_2'; From 96f73139b63d38a92e9e35db49b1c0158a1f164f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 10 Nov 2023 06:13:55 +0100 Subject: [PATCH 172/813] Check for large translation units --- CMakeLists.txt | 7 +++++++ programs/CMakeLists.txt | 5 +++++ utils/check-style/check-large-objects.sh | 10 ++++++++++ 3 files changed, 22 insertions(+) create mode 100755 utils/check-style/check-large-objects.sh diff --git a/CMakeLists.txt b/CMakeLists.txt index 0d1ef22b2aa..ef97c13fa1c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -622,3 +622,10 @@ if (NATIVE_BUILD_TARGETS COMMAND ${CMAKE_COMMAND} --build "${NATIVE_BUILD_DIR}" --target ${NATIVE_BUILD_TARGETS} COMMAND_ECHO STDOUT) endif () + +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") + set(CHECK_LARGE_OBJECT_SIZES_DEFAULT ON) +else () + set(CHECK_LARGE_OBJECT_SIZES_DEFAULT OFF) +endif () +option(CHECK_LARGE_OBJECT_SIZES "Check that there are no large object files after build." ${CHECK_LARGE_OBJECT_SIZES_DEFAULT}) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index eb4a898d472..8496452e6ea 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -432,6 +432,11 @@ if (USE_BINARY_HASH) add_custom_command(TARGET clickhouse POST_BUILD COMMAND ./clickhouse hash-binary > hash && ${OBJCOPY_PATH} --add-section .clickhouse.hash=hash clickhouse COMMENT "Adding section '.clickhouse.hash' to clickhouse binary" VERBATIM) endif() +if (CHECK_LARGE_OBJECT_SIZES) + add_custom_command(TARGET clickhouse POST_BUILD + COMMAND "${CMAKE_SOURCE_DIR}/utils/check-style/check-large-objects.sh" "${CMAKE_BINARY_DIR}") +endif () + if (SPLIT_DEBUG_SYMBOLS) clickhouse_split_debug_symbols(TARGET clickhouse DESTINATION_DIR ${CMAKE_CURRENT_BINARY_DIR}/${SPLITTED_DEBUG_SYMBOLS_DIR} BINARY_PATH clickhouse) else() diff --git a/utils/check-style/check-large-objects.sh b/utils/check-style/check-large-objects.sh new file mode 100755 index 00000000000..c598ff0e99c --- /dev/null +++ b/utils/check-style/check-large-objects.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +# Check that there are no new translation units compiled to an object file larger than a certain size. + +if find $1 -name '*.o' | xargs wc -c | grep -v total | sort -rn | awk '{ if ($1 > 50000000) print }' \ + | grep -v -P 'CastOverloadResolver|AggregateFunctionMax|AggregateFunctionMin|RangeHashedDictionary|Aggregator|AggregateFunctionUniq' +then + echo "^ It's not allowed to have so large translation units." + exit 1 +fi From 2544a15c94bff918f4a9aab934d771f9e2c604d8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 10 Nov 2023 05:59:24 +0000 Subject: [PATCH 173/813] optimization, comment --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 19 +++++++++++-------- .../InterpreterSelectQueryAnalyzer.cpp | 3 ++- tests/broken_tests.txt | 1 + 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b7bd606b6cf..4e9e3e5f350 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4724,17 +4724,20 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi else { /// Replace storage with values storage of insertion block - if (auto * query_node = in_second_argument->as()) + if (StoragePtr storage = scope.context->getViewSource()) { - auto table_expression = extractLeftTableExpression(query_node->getJoinTree()); - if (auto * query_table_node = table_expression->as()) + if (auto * query_node = in_second_argument->as()) { - if (StoragePtr storage = scope.context->getViewSource(); storage && query_table_node->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullTableName()) + auto table_expression = extractLeftTableExpression(query_node->getJoinTree()); + if (auto * query_table_node = table_expression->as()) { - auto replacement_table_expression = std::make_shared(storage, scope.context); - if (std::optional table_expression_modifiers = query_table_node->getTableExpressionModifiers()) - replacement_table_expression->setTableExpressionModifiers(*table_expression_modifiers); - in_second_argument = in_second_argument->cloneAndReplace(table_expression, std::move(replacement_table_expression)); + if (query_table_node->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullNameNotQuoted()) + { + auto replacement_table_expression = std::make_shared(storage, scope.context); + if (std::optional table_expression_modifiers = query_table_node->getTableExpressionModifiers()) + replacement_table_expression->setTableExpressionModifiers(*table_expression_modifiers); + in_second_argument = in_second_argument->cloneAndReplace(table_expression, std::move(replacement_table_expression)); + } } } } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 9e675f0f25a..93073a0e10d 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -109,7 +109,8 @@ void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr & } } - if (auto * table_node = table_expression_to_replace->as(); table_node && table_node->getStorageID().getFullNameNotQuoted() != storage->getStorageID().getFullTableName()) + /// Don't replace storage if table name differs + if (auto * table_node = table_expression_to_replace->as(); table_node && table_node->getStorageID().getFullNameNotQuoted() != storage->getStorageID().getFullNameNotQuoted()) return; auto replacement_table_expression = std::make_shared(storage, context); diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index 38a51194279..faee1c5b295 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -33,6 +33,7 @@ 01232_extremes 01244_optimize_distributed_group_by_sharding_key 01247_optimize_distributed_group_by_sharding_key_dist_on_dist +01268_mv_scalars 01268_shard_avgweighted 01270_optimize_skip_unused_shards_low_cardinality 01319_optimize_skip_unused_shards_nesting From 5fea9f9dc6fd33baf332affc458d2bbbd82f4d0f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 10 Nov 2023 09:25:57 +0000 Subject: [PATCH 174/813] Small fixes --- src/Common/ZooKeeper/ZooKeeper.h | 19 +++++++++ .../ZooKeeper/ZooKeeperWithFaultInjection.h | 5 +++ src/Storages/StorageKeeperMap.cpp | 5 ++- .../02911_backup_restore_keeper_map.sh | 39 ++++++++++--------- 4 files changed, 47 insertions(+), 21 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index c41d1d8dbab..785842b94bd 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -135,6 +135,16 @@ struct MultiReadResponses responses); } + /// If Keeper/ZooKeeper doesn't support MultiRead feature we will dispatch + /// asynchronously all the read requests separately + /// Sometimes it's important to process all requests instantly + /// e.g. we want to trigger exceptions while we are in the ZK client retry loop + void waitForResponses() + { + if (auto * responses_with_futures = std::get_if(&responses)) + responses_with_futures->waitForResponses(); + } + private: using RegularResponses = std::vector; using FutureResponses = std::vector>; @@ -158,6 +168,15 @@ private: return *cached_responses[index]; } + void waitForResponses() + { + for (size_t i = 0; i < size(); ++i) + { + if (!cached_responses[i].has_value()) + cached_responses[i] = future_responses[i].get(); + } + } + size_t size() const { return future_responses.size(); } }; diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index 4887e896e9b..be4642c2988 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -242,6 +242,11 @@ public: return access("get", !paths.empty() ? paths.front() : "", [&]() { return keeper->get(paths); }); } + zkutil::ZooKeeper::MultiTryGetResponse tryGet(const std::vector & paths) + { + return access("tryGet", !paths.empty() ? paths.front() : "", [&]() { return keeper->tryGet(paths); }); + } + bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) { return access("exists", path, [&]() { return keeper->exists(path, stat, watch); }); diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 15ebc4d92d1..3032973c411 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -703,13 +703,14 @@ private: for (const auto & key : keys) keys_full_path.push_back(data_zookeeper_path / key); - zkutil::ZooKeeper::MultiGetResponse data; + zkutil::ZooKeeper::MultiTryGetResponse data; auto holder = with_retries->createRetriesControlHolder("getKeeperMapDataKeys"); holder.retries_ctl.retryLoop( [&, &zk = holder.faulty_zookeeper] { with_retries->renewZooKeeper(zk); - data = zk->get(keys_full_path); + data = zk->tryGet(keys_full_path); + data.waitForResponses(); }); for (size_t i = 0; i < keys.size(); ++i) diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh index 6c463beb221..ae7c22f6820 100755 --- a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh @@ -4,44 +4,45 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +database_name="$CLICKHOUSE_DATABASE"_02911_keeper_map $CLICKHOUSE_CLIENT -nm -q " - DROP DATABASE IF EXISTS 02911_keeper_map; - CREATE DATABASE 02911_keeper_map; - CREATE TABLE 02911_keeper_map.02911_backup_restore_keeper_map1 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; - CREATE TABLE 02911_keeper_map.02911_backup_restore_keeper_map2 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; - CREATE TABLE 02911_keeper_map.02911_backup_restore_keeper_map3 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911_different') PRIMARY KEY key; + DROP DATABASE IF EXISTS $database_name; + CREATE DATABASE $database_name; + CREATE TABLE $database_name.02911_backup_restore_keeper_map1 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; + CREATE TABLE $database_name.02911_backup_restore_keeper_map2 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; + CREATE TABLE $database_name.02911_backup_restore_keeper_map3 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911_different') PRIMARY KEY key; - INSERT INTO 02911_keeper_map.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000; - INSERT INTO 02911_keeper_map.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000; + INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000; + INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000; " -backup_path="$CLICKHOUSE_DATABASE/02911_keeper_map" +backup_path="$database_name" for i in $(seq 1 3); do - $CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map$i;" + $CLICKHOUSE_CLIENT -q "SELECT count() FROM $database_name.02911_backup_restore_keeper_map$i;" done -$CLICKHOUSE_CLIENT -q "BACKUP DATABASE 02911_keeper_map TO Disk('backups', '$backup_path');" > /dev/null +$CLICKHOUSE_CLIENT -q "BACKUP DATABASE $database_name TO Disk('backups', '$backup_path');" > /dev/null -$CLICKHOUSE_CLIENT -q "DROP DATABASE 02911_keeper_map SYNC;" +$CLICKHOUSE_CLIENT -q "DROP DATABASE $database_name SYNC;" for i in $(seq 1 3); do - $CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map$i;" 2>&1 | grep -Fq "UNKNOWN_DATABASE" && echo 'OK' || echo 'ERROR' + $CLICKHOUSE_CLIENT -q "SELECT count() FROM $database_name.02911_backup_restore_keeper_map$i;" 2>&1 | grep -Fq "UNKNOWN_DATABASE" && echo 'OK' || echo 'ERROR' done -$CLICKHOUSE_CLIENT -q "RESTORE DATABASE 02911_keeper_map FROM Disk('backups', '$backup_path');" > /dev/null +$CLICKHOUSE_CLIENT -q "RESTORE DATABASE $database_name FROM Disk('backups', '$backup_path');" > /dev/null for i in $(seq 1 3); do - $CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map$i;" + $CLICKHOUSE_CLIENT -q "SELECT count() FROM $database_name.02911_backup_restore_keeper_map$i;" done -$CLICKHOUSE_CLIENT -q "DROP TABLE 02911_keeper_map.02911_backup_restore_keeper_map3 SYNC;" +$CLICKHOUSE_CLIENT -q "DROP TABLE $database_name.02911_backup_restore_keeper_map3 SYNC;" -$CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map3;" 2>&1 | grep -Fq "UNKNOWN_TABLE" && echo 'OK' || echo 'ERROR' +$CLICKHOUSE_CLIENT -q "SELECT count() FROM $database_name.02911_backup_restore_keeper_map3;" 2>&1 | grep -Fq "UNKNOWN_TABLE" && echo 'OK' || echo 'ERROR' -$CLICKHOUSE_CLIENT -q "RESTORE TABLE 02911_keeper_map.02911_backup_restore_keeper_map3 FROM Disk('backups', '$backup_path');" > /dev/null +$CLICKHOUSE_CLIENT -q "RESTORE TABLE $database_name.02911_backup_restore_keeper_map3 FROM Disk('backups', '$backup_path');" > /dev/null for i in $(seq 1 3); do - $CLICKHOUSE_CLIENT -q "SELECT count() FROM 02911_keeper_map.02911_backup_restore_keeper_map$i;" + $CLICKHOUSE_CLIENT -q "SELECT count() FROM $database_name.02911_backup_restore_keeper_map$i;" done -$CLICKHOUSE_CLIENT -q "DROP DATABASE 02911_keeper_map SYNC;" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "DROP DATABASE $database_name SYNC;" \ No newline at end of file From b0addc66a837bb087cd9f507f4a84d32cef32908 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 10 Nov 2023 12:16:32 +0100 Subject: [PATCH 175/813] Update 02908_filesystem_cache_as_collection.sql --- .../0_stateless/02908_filesystem_cache_as_collection.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql index e878727359b..08fb8d76e71 100644 --- a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql +++ b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql @@ -1,8 +1,8 @@ CREATE NAMED COLLECTION IF NOT EXISTS cache_collection_sql AS path = 'collection_sql', max_size = '1Mi'; DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) -ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 's3_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME', cache_name='cache_collection_sql'); +ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 'local_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME', cache_name='cache_collection_sql'); DESCRIBE FILESYSTEM CACHE '$CLICHOUSE_TEST_UNIQUE_NAME'; CREATE TABLE test2 (a Int32, b String) -ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 's3_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME_2', cache_name='cache_collection'); +ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 'local_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME_2', cache_name='cache_collection'); DESCRIBE FILESYSTEM CACHE '$CLICHOUSE_TEST_UNIQUE_NAME_2'; From 687eaeb18644092809d9edf58c601513bd4b2264 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Nov 2023 12:34:25 +0100 Subject: [PATCH 176/813] Update InterpreterExplainQuery.cpp --- src/Interpreters/InterpreterExplainQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 7ec17292055..8f4b6b13739 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -541,14 +541,14 @@ QueryPipeline InterpreterExplainQuery::executeImpl() if (context->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); - plan = std::move(interpreter).extractQueryPlan(); context = interpreter.getContext(); + plan = std::move(interpreter).extractQueryPlan(); } else { InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); - interpreter.buildQueryPlan(plan); context = interpreter.getContext(); + interpreter.buildQueryPlan(plan); } // Collect the selected marks, rows, parts during build query pipeline. From 214ac112a75703470c6dc607299f02d58a14f093 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Nov 2023 12:48:03 +0100 Subject: [PATCH 177/813] Looks fixed --- src/Interpreters/Cache/FileCache.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index ed8cc547fbc..914292c003d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -361,6 +361,7 @@ void FileCache::fillHolesWithEmptyFileSegments( locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, settings); file_segments.insert(it, file_segment); + ++added; } else { @@ -395,14 +396,6 @@ void FileCache::fillHolesWithEmptyFileSegments( return; } - if (file_segments_limit && file_segments.size() >= file_segments_limit) - { - std::string res; - for (const auto & f : file_segments) - res += " - " + f->range().toString(); - LOG_ERROR(log, "Limit: {}, file segments: {}, added: {}, range: {}, file_segments: {}", - file_segments_limit, file_segments.size(), added, range.toString(), res); - } chassert(!file_segments_limit || file_segments.size() < file_segments_limit); if (current_pos <= range.right) From 2d46340803c3b5a5a4ba33e29430fc6a6547e4cb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 10 Nov 2023 13:27:28 +0100 Subject: [PATCH 178/813] Update 02908_filesystem_cache_as_collection.sql --- .../0_stateless/02908_filesystem_cache_as_collection.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql index 08fb8d76e71..ed9041a0800 100644 --- a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql +++ b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + CREATE NAMED COLLECTION IF NOT EXISTS cache_collection_sql AS path = 'collection_sql', max_size = '1Mi'; DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) From a7fb6a30f8ac8751ce63972e22d6e829d3ecdb5d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Nov 2023 13:29:01 +0100 Subject: [PATCH 179/813] Better --- src/Interpreters/Cache/FileCache.cpp | 77 +++++++++++++------------- src/Interpreters/Cache/FileCache.h | 16 +++++- src/Interpreters/Cache/FileSegment.cpp | 1 - 3 files changed, 53 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 914292c003d..e998d2a3639 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -156,7 +156,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: { auto file_segment = std::make_shared( locked_key.getKey(), range.left, range.size(), FileSegment::State::DETACHED); - return {file_segment}; + return { file_segment }; } if (locked_key.empty()) @@ -296,7 +296,6 @@ FileSegments FileCache::splitRangeIntoFileSegments( current_pos += current_file_segment_size; } - assert(file_segments.empty() || file_segments_limit > 0 || offset + size - 1 == file_segments.back()->range().right); return file_segments; } @@ -321,7 +320,7 @@ void FileCache::fillHolesWithEmptyFileSegments( assert(!file_segments.empty()); auto it = file_segments.begin(); - size_t added = 0; + size_t processed_count = 0; auto segment_range = (*it)->range(); size_t current_pos; @@ -334,12 +333,17 @@ void FileCache::fillHolesWithEmptyFileSegments( current_pos = segment_range.right + 1; ++it; - ++added; + ++processed_count; } else current_pos = range.left; - while (current_pos <= range.right && it != file_segments.end() && (!file_segments_limit || added < file_segments_limit)) + auto is_limit_reached = [&]() -> bool + { + return file_segments_limit && processed_count >= file_segments_limit; + }; + + while (current_pos <= range.right && it != file_segments.end() && !is_limit_reached()) { segment_range = (*it)->range(); @@ -347,7 +351,7 @@ void FileCache::fillHolesWithEmptyFileSegments( { current_pos = segment_range.right + 1; ++it; - ++added; + ++processed_count; continue; } @@ -361,7 +365,7 @@ void FileCache::fillHolesWithEmptyFileSegments( locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, settings); file_segments.insert(it, file_segment); - ++added; + ++processed_count; } else { @@ -371,28 +375,32 @@ void FileCache::fillHolesWithEmptyFileSegments( { auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, settings, nullptr); hole.push_back(metadata_it->second->file_segment); - ++added; + ++processed_count; - if (file_segments_limit && added == file_segments_limit) - { - file_segments.splice(it, std::move(hole)); - file_segments.erase(it, file_segments.end()); - return; - } + if (is_limit_reached()) + break; } file_segments.splice(it, std::move(hole)); } + if (is_limit_reached()) + break; + current_pos = segment_range.right + 1; ++it; - ++added; + ++processed_count; } - if (file_segments_limit && added == file_segments_limit) + auto erase_unprocessed = [&]() { chassert(file_segments.size() >= file_segments_limit); file_segments.erase(it, file_segments.end()); chassert(file_segments.size() == file_segments_limit); + }; + + if (is_limit_reached()) + { + erase_unprocessed(); return; } @@ -422,16 +430,15 @@ void FileCache::fillHolesWithEmptyFileSegments( { auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, settings, nullptr); hole.push_back(metadata_it->second->file_segment); - ++added; + ++processed_count; - if (file_segments_limit && added == file_segments_limit) - { - file_segments.splice(it, std::move(hole)); - file_segments.erase(it, file_segments.end()); - return; - } + if (is_limit_reached()) + break; } file_segments.splice(it, std::move(hole)); + + if (is_limit_reached()) + erase_unprocessed(); } } } @@ -589,23 +596,16 @@ FileCache::getOrSet( fillHolesWithEmptyFileSegments( *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, settings); - chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); - if (!file_segments.front()->range().contains(offset)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " "(end offset: {}, aligned offset: {}, aligned end offset: {})", file_segments.front()->range().toString(), offset, range.right, aligned_offset, aligned_end_offset); } - - chassert(file_segments_limit ? file_segments.back()->range().left <= range.right : file_segments.back()->range().contains(range.right)); } - while (file_segments_limit && file_segments.size() > file_segments_limit) - file_segments.pop_back(); - - if (file_segments.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of file segments for offset {}, size {} (file size: {})", offset, size, file_size); + chassert(file_segments_limit ? file_segments.back()->range().left <= range.right : file_segments.back()->range().contains(range.right)); + chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); } @@ -625,14 +625,17 @@ FileSegmentsHolderPtr FileCache::get(const Key & key, size_t offset, size_t size auto file_segments = getImpl(*locked_key, range, file_segments_limit); if (!file_segments.empty()) { + if (file_segments_limit) + { + chassert(file_segments.size() <= file_segments_limit); + if (file_segments.size() == file_segments_limit) + range.right = file_segments.back()->range().right; + } + fillHolesWithEmptyFileSegments( *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */true, CreateFileSegmentSettings{}); - if (file_segments_limit) - { - while (file_segments.size() > file_segments_limit) - file_segments.pop_back(); - } + chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); } } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 523ff90e33e..7485c7d2b90 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -209,18 +209,28 @@ private: std::unique_ptr cleanup_thread; void assertInitialized() const; - void assertCacheCorrectness(); void loadMetadata(); void loadMetadataImpl(); void loadMetadataForKeys(const std::filesystem::path & keys_dir); - /// bool - if `file_segments_limit` reached or not. - FileSegments getImpl(const LockedKey & locked_key, const FileSegment::Range & range, size_t file_segments_limit) const; + /// Get all file segments from cache which intersect with `range`. + /// If `file_segments_limit` > 0, return no more than first file_segments_limit + /// file segments. + FileSegments getImpl( + const LockedKey & locked_key, + const FileSegment::Range & range, + size_t file_segments_limit) const; + /// Split range into subranges by max_file_segment_size, + /// each subrange size must be less or equal to max_file_segment_size. std::vector splitRange(size_t offset, size_t size); + /// Split range into subranges by max_file_segment_size (same as in splitRange()) + /// and create a new file segment for each subrange. + /// If `file_segments_limit` > 0, create no more than first file_segments_limit + /// file segments. FileSegments splitRangeIntoFileSegments( LockedKey & locked_key, size_t offset, diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 59134229997..794dd663bee 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -933,7 +933,6 @@ FileSegmentsHolder::FileSegmentsHolder(FileSegments && file_segments_) FileSegmentsHolder::~FileSegmentsHolder() { - ProfileEvents::increment(ProfileEvents::FilesystemCacheUnusedHoldFileSegments, file_segments.size()); ProfileEventTimeIncrement watch(ProfileEvents::FileSegmentHolderCompleteMicroseconds); ProfileEvents::increment(ProfileEvents::FilesystemCacheUnusedHoldFileSegments, file_segments.size()); From b898e90bc157b7ce69b3ff2da7e887b68f039d26 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Nov 2023 14:31:00 +0100 Subject: [PATCH 180/813] Better --- src/Interpreters/Cache/FileCacheSettings.cpp | 95 +++++++++----------- src/Interpreters/Cache/FileCacheSettings.h | 6 ++ 2 files changed, 48 insertions(+), 53 deletions(-) diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index e81bd9ddc35..2ac54006a8e 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -13,78 +13,67 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetString get_string) { - if (!config.has(config_prefix + ".path")) + auto config_parse_size = [&](std::string_view key) { return parseWithSizeSuffix(get_string(key)); }; + + if (!has("path")) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected cache path (`path`) in configuration"); - base_path = config.getString(config_prefix + ".path"); + base_path = get_string("path"); - if (!config.has(config_prefix + ".max_size")) + if (!has("max_size")) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected cache size (`max_size`) in configuration"); - max_size = parseWithSizeSuffix(config.getString(config_prefix + ".max_size")); + max_size = config_parse_size("max_size"); if (max_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected non-zero size for cache configuration"); - max_elements = config.getUInt64(config_prefix + ".max_elements", FILECACHE_DEFAULT_MAX_ELEMENTS); + if (has("max_elements")) + max_elements = get_uint("max_elements"); - if (config.has(config_prefix + ".max_file_segment_size")) - max_file_segment_size = parseWithSizeSuffix(config.getString(config_prefix + ".max_file_segment_size")); + if (has("max_file_segment_size")) + max_file_segment_size = config_parse_size("max_file_segment_size"); - cache_on_write_operations = config.getUInt64(config_prefix + ".cache_on_write_operations", false); - enable_filesystem_query_cache_limit = config.getUInt64(config_prefix + ".enable_filesystem_query_cache_limit", false); - cache_hits_threshold = config.getUInt64(config_prefix + ".cache_hits_threshold", FILECACHE_DEFAULT_HITS_THRESHOLD); - enable_bypass_cache_with_threshold = config.getUInt64(config_prefix + ".enable_bypass_cache_with_threshold", false); + if (has("cache_on_write_operations")) + cache_on_write_operations = get_uint("cache_on_write_operations"); - if (config.has(config_prefix + ".bypass_cache_threshold")) - bypass_cache_threshold = parseWithSizeSuffix(config.getString(config_prefix + ".bypass_cache_threshold")); + if (has("enable_filesystem_query_cache_limit")) + enable_filesystem_query_cache_limit = get_uint("enable_filesystem_query_cache_limit"); - if (config.has(config_prefix + ".boundary_alignment")) - boundary_alignment = parseWithSizeSuffix(config.getString(config_prefix + ".boundary_alignment")); + if (has("cache_hits_threshold")) + cache_hits_threshold = get_uint("cache_hits_threshold"); - if (config.has(config_prefix + ".background_download_threads")) - background_download_threads = config.getUInt(config_prefix + ".background_download_threads"); + if (has("enable_bypass_cache_with_threshold")) + enable_bypass_cache_with_threshold = get_uint("enable_bypass_cache_with_threshold"); - if (config.has(config_prefix + ".load_metadata_threads")) - load_metadata_threads = config.getUInt(config_prefix + ".load_metadata_threads"); + if (has("bypass_cache_threshold")) + bypass_cache_threshold = config_parse_size("bypass_cache_threshold"); + + if (has("boundary_alignment")) + boundary_alignment = config_parse_size("boundary_alignment"); + + if (has("background_download_threads")) + background_download_threads = get_uint("background_download_threads"); + + if (has("load_metadata_threads")) + load_metadata_threads = get_uint("load_metadata_threads"); +} + +void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +{ + auto config_has = [&](std::string_view key) { return config.has(fmt::format("{}.{}", config_prefix, key)); }; + auto config_get_uint = [&](std::string_view key) { return config.getUInt(fmt::format("{}.{}", config_prefix, key)); }; + auto config_get_string = [&](std::string_view key) { return config.getString(fmt::format("{}.{}", config_prefix, key)); }; + loadImpl(std::move(config_has), std::move(config_get_uint), std::move(config_get_string)); } void FileCacheSettings::loadFromCollection(const NamedCollection & collection) { - if (!collection.has("path")) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected cache path (`path`) in configuration"); - - base_path = collection.get("path"); - - if (!collection.has("max_size")) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected cache size (`max_size`) in configuration"); - - max_size = parseWithSizeSuffix(collection.get("max_size")); - if (max_size == 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected non-zero size for cache configuration"); - - max_elements = collection.getOrDefault("max_elements", FILECACHE_DEFAULT_MAX_ELEMENTS); - - if (collection.has("max_file_segment_size")) - max_file_segment_size = parseWithSizeSuffix(collection.get("max_file_segment_size")); - - cache_on_write_operations = collection.getOrDefault("cache_on_write_operations", false); - enable_filesystem_query_cache_limit = collection.getOrDefault("enable_filesystem_query_cache_limit", false); - cache_hits_threshold = collection.getOrDefault("cache_hits_threshold", FILECACHE_DEFAULT_HITS_THRESHOLD); - enable_bypass_cache_with_threshold = collection.getOrDefault("enable_bypass_cache_with_threshold", false); - - if (collection.has("bypass_cache_threshold")) - bypass_cache_threshold = parseWithSizeSuffix(collection.get("bypass_cache_threshold")); - - if (collection.has("boundary_alignment")) - boundary_alignment = parseWithSizeSuffix(collection.get("boundary_alignment")); - - if (collection.has("background_download_threads")) - background_download_threads = collection.get("background_download_threads"); - - if (collection.has("load_metadata_threads")) - load_metadata_threads = collection.get("load_metadata_threads"); + auto config_has = [&](std::string_view key) { return collection.has(std::string(key)); }; + auto config_get_uint = [&](std::string_view key) { return collection.get(std::string(key)); }; + auto config_get_string = [&](std::string_view key) { return collection.get(std::string(key)); }; + loadImpl(std::move(config_has), std::move(config_get_uint), std::move(config_get_string)); } } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index 9d15a32d3bf..bf0dd492bc9 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -33,6 +33,12 @@ struct FileCacheSettings void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); void loadFromCollection(const NamedCollection & collection); + +private: + using FuncHas = std::function; + using FuncGetUInt = std::function; + using FuncGetString = std::function; + void loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetString get_string); }; } From 4859db79e5144f9121eba1febf31288410927d13 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Nov 2023 17:13:29 +0000 Subject: [PATCH 181/813] Fix 02554_fix_grouping_sets_predicate_push_down with analyzer. --- ...rouping_sets_predicate_push_down.reference | 24 +++++++++ ..._fix_grouping_sets_predicate_push_down.sql | 51 +++++++++++++++++-- 2 files changed, 72 insertions(+), 3 deletions(-) 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 794fe5944cd..7e43f249a74 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 @@ -41,6 +41,20 @@ ExpressionTransform × 2 ExpressionTransform (ReadFromMergeTree) MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 +(Expression) +ExpressionTransform × 2 + (Filter) + FilterTransform × 2 + (Aggregating) + ExpressionTransform × 2 + AggregatingTransform × 2 + Copy 1 → 2 + (Expression) + ExpressionTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 ---Result--- 2023-01-05 all @@ -60,3 +74,13 @@ ExpressionTransform × 2 FilterTransform (ReadFromMergeTree) MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 +(Expression) +ExpressionTransform × 2 + (Aggregating) + ExpressionTransform × 2 + AggregatingTransform × 2 + Copy 1 → 2 + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 diff --git a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql index 9a970674890..3d98096f52e 100644 --- a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql +++ b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql @@ -26,7 +26,7 @@ FROM SELECT day_, type_1 - FROM test_grouping_sets_predicate + FROM test_grouping_sets_predicate WHERE day_ = '2023-01-05' GROUP BY GROUPING SETS ( @@ -58,7 +58,30 @@ FROM (day_)) ) AS t ) -WHERE type_1 = 'all'; +WHERE type_1 = 'all' settings allow_experimental_analyzer=0; + +-- Query plan with analyzer has less Filter steps (which is more optimal) +EXPLAIN PIPELINE +SELECT * +FROM +( + SELECT + day_, + if(type_1 = '', 'all', type_1) AS type_1 + FROM + ( + SELECT + day_, + type_1 + FROM test_grouping_sets_predicate + WHERE day_ = '2023-01-05' + GROUP BY + GROUPING SETS ( + (day_, type_1), + (day_)) + ) AS t +) +WHERE type_1 = 'all' settings allow_experimental_analyzer=1; SELECT ''; SELECT '---Result---'; @@ -104,6 +127,28 @@ FROM (day_)) ) AS t ) -WHERE day_ = '2023-01-05'; +WHERE day_ = '2023-01-05' settings allow_experimental_analyzer=0; + +-- Query plan with analyzer has less Filter steps (which is more optimal) +EXPLAIN PIPELINE +SELECT * +FROM +( + SELECT + day_, + if(type_1 = '', 'all', type_1) AS type_1 + FROM + ( + SELECT + day_, + type_1 + FROM test_grouping_sets_predicate + GROUP BY + GROUPING SETS ( + (day_, type_1), + (day_)) + ) AS t +) +WHERE day_ = '2023-01-05' settings allow_experimental_analyzer=1; DROP TABLE test_grouping_sets_predicate; From 047f87a3e0676e16c5128d74f330c3417e9e2a08 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Nov 2023 17:14:50 +0000 Subject: [PATCH 182/813] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 018482c2301..33b972f7e18 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -40,7 +40,6 @@ 02458_use_structure_from_insertion_table 02479_race_condition_between_insert_and_droppin_mv 02493_inconsistent_hex_and_binary_number -02554_fix_grouping_sets_predicate_push_down 02575_merge_prewhere_different_default_kind 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET 01009_global_array_join_names From 78f0a2a61d6ab089c475d3f47926353139d5f2aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 10 Nov 2023 17:20:17 +0000 Subject: [PATCH 183/813] Revert "Merge pull request #56591 from ClickHouse/revert-56500-function-sleep-exception-message-fix" This reverts commit e8875d881de0f0aaafaa306814ecc33b29ddff49, reversing changes made to a5b1474b6a8de71cdfa3b579efbb76b25c5741bd. --- src/Functions/sleep.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index fba8293e5ff..b7d4a1ab921 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -112,13 +112,15 @@ public: if (size > 0) { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. + UInt64 microseconds = static_cast(seconds * 1e6); if (max_microseconds && seconds * 1e6 > max_microseconds) - throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is {} microseconds. Requested: {}", max_microseconds, seconds); + throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is {} microseconds. Requested: {} microseconds", + max_microseconds, microseconds); if (!dry_run) { UInt64 count = (variant == FunctionSleepVariant::PerBlock ? 1 : size); - UInt64 microseconds = static_cast(seconds * count * 1e6); + microseconds = static_cast(seconds * count * 1e6); if (max_microseconds && microseconds > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, From 72fae1057d487312608b3da59bca83dcb9b39164 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 10 Nov 2023 17:23:56 +0000 Subject: [PATCH 184/813] Fix ubsan in sleep and add sanity checks --- src/Functions/sleep.h | 11 ++++++----- .../0_stateless/02915_sleep_large_uint.reference | 0 tests/queries/0_stateless/02915_sleep_large_uint.sql | 4 ++++ 3 files changed, 10 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02915_sleep_large_uint.reference create mode 100644 tests/queries/0_stateless/02915_sleep_large_uint.sql diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index b7d4a1ab921..024ea757b9e 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -50,7 +50,8 @@ public: return std::make_shared>(context->getSettingsRef().function_sleep_max_microseconds_per_block); } - FunctionSleep(UInt64 max_microseconds_) : max_microseconds(max_microseconds_) + FunctionSleep(UInt64 max_microseconds_) + : max_microseconds(std::min(max_microseconds_, static_cast(std::numeric_limits::max()))) { } @@ -103,8 +104,8 @@ public: Float64 seconds = applyVisitor(FieldVisitorConvertToNumber(), assert_cast(*col).getField()); - if (seconds < 0 || !std::isfinite(seconds)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot sleep infinite or negative amount of time (not implemented)"); + if (seconds < 0 || !std::isfinite(seconds) || seconds > static_cast(std::numeric_limits::max())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot sleep infinite, very large or negative amount of time (not implemented)"); size_t size = col->size(); @@ -112,7 +113,7 @@ public: if (size > 0) { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. - UInt64 microseconds = static_cast(seconds * 1e6); + UInt64 microseconds = static_cast(seconds) * 1000000ull; if (max_microseconds && seconds * 1e6 > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is {} microseconds. Requested: {} microseconds", max_microseconds, microseconds); @@ -120,7 +121,7 @@ public: if (!dry_run) { UInt64 count = (variant == FunctionSleepVariant::PerBlock ? 1 : size); - microseconds = static_cast(seconds * count * 1e6); + microseconds = static_cast(seconds) * count * 1000000ull; if (max_microseconds && microseconds > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, diff --git a/tests/queries/0_stateless/02915_sleep_large_uint.reference b/tests/queries/0_stateless/02915_sleep_large_uint.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02915_sleep_large_uint.sql b/tests/queries/0_stateless/02915_sleep_large_uint.sql new file mode 100644 index 00000000000..9fff629fb14 --- /dev/null +++ b/tests/queries/0_stateless/02915_sleep_large_uint.sql @@ -0,0 +1,4 @@ +SELECT sleep(3.40282e+44); -- { serverError BAD_ARGUMENTS } +SELECT sleep((pow(2, 64) / 1000000) - 1); -- { serverError BAD_ARGUMENTS } +SELECT sleepEachRow(184467440737095516) from numbers(10000); -- { serverError BAD_ARGUMENTS } +SELECT sleepEachRow(pow(2, 31)) from numbers(9007199254740992) settings function_sleep_max_microseconds_per_block = 8589934592000000000; -- { serverError TOO_SLOW } From 62daff0db823b602a028dc93979ede28dbf227bb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 10 Nov 2023 18:41:41 +0100 Subject: [PATCH 185/813] Update 02908_filesystem_cache_as_collection.sql --- .../0_stateless/02908_filesystem_cache_as_collection.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql index ed9041a0800..c7216833bc9 100644 --- a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql +++ b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-replicated-database CREATE NAMED COLLECTION IF NOT EXISTS cache_collection_sql AS path = 'collection_sql', max_size = '1Mi'; DROP TABLE IF EXISTS test; From 0d01438c48282051f8978e5fe30b34db9f2d10fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 00:44:46 +0100 Subject: [PATCH 186/813] Fix error --- CMakeLists.txt | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index ef97c13fa1c..9c8952aea96 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -554,6 +554,13 @@ if (ENABLE_RUST) endif() endif() +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") + set(CHECK_LARGE_OBJECT_SIZES_DEFAULT ON) +else () + set(CHECK_LARGE_OBJECT_SIZES_DEFAULT OFF) +endif () +option(CHECK_LARGE_OBJECT_SIZES "Check that there are no large object files after build." ${CHECK_LARGE_OBJECT_SIZES_DEFAULT}) + add_subdirectory (base) add_subdirectory (src) add_subdirectory (programs) @@ -622,10 +629,3 @@ if (NATIVE_BUILD_TARGETS COMMAND ${CMAKE_COMMAND} --build "${NATIVE_BUILD_DIR}" --target ${NATIVE_BUILD_TARGETS} COMMAND_ECHO STDOUT) endif () - -if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") - set(CHECK_LARGE_OBJECT_SIZES_DEFAULT ON) -else () - set(CHECK_LARGE_OBJECT_SIZES_DEFAULT OFF) -endif () -option(CHECK_LARGE_OBJECT_SIZES "Check that there are no large object files after build." ${CHECK_LARGE_OBJECT_SIZES_DEFAULT}) From 356ae52e9b122545ea9a26a18ccce0e5311d4df5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Nov 2023 08:46:10 +0100 Subject: [PATCH 187/813] Enable access and named collections control by default --- .../config/users.d/perf-comparison-tweaks-users.xml | 5 ----- programs/server/embedded.xml | 2 ++ programs/server/users.xml | 5 ++++- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml index cb591f1a184..e780a99ecde 100644 --- a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml +++ b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml @@ -34,9 +34,4 @@ 0 - - - 1 - - diff --git a/programs/server/embedded.xml b/programs/server/embedded.xml index c2336e0d582..9311749a173 100644 --- a/programs/server/embedded.xml +++ b/programs/server/embedded.xml @@ -23,7 +23,9 @@ default default + 1 + 1 diff --git a/programs/server/users.xml b/programs/server/users.xml index fbb5a2c228f..57bc6309a54 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -85,7 +85,10 @@ default - + 1 + + + 1 10 - - 0 From 993f7529001db2872394da5b2b36e8e45fc69d64 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 13 Nov 2023 12:53:09 +0000 Subject: [PATCH 246/813] Revert "Merge pull request #56543 from rschu1ze/bump-grpc" This reverts commit 213c7cffb5eae1951f67b8531ec69262696c7e3d, reversing changes made to 9ed47749dee12e900875ff4c6214a177fa07a94c. --- contrib/grpc | 2 +- contrib/sparse-checkout/update-grpc.sh | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/contrib/grpc b/contrib/grpc index b723ecae099..267af8c3a1e 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit b723ecae0991bb873fe87a595dfb187178733fde +Subproject commit 267af8c3a1ea4a5a4d9e5a070ad2d1ac7c701923 diff --git a/contrib/sparse-checkout/update-grpc.sh b/contrib/sparse-checkout/update-grpc.sh index 21628ce8dd1..4571bd6307d 100755 --- a/contrib/sparse-checkout/update-grpc.sh +++ b/contrib/sparse-checkout/update-grpc.sh @@ -7,7 +7,6 @@ echo '/*' > $FILES_TO_CHECKOUT echo '!/test/*' >> $FILES_TO_CHECKOUT echo '/test/build/*' >> $FILES_TO_CHECKOUT echo '/test/core/tsi/alts/fake_handshaker/*' >> $FILES_TO_CHECKOUT -echo '/test/core/event_engine/fuzzing_event_engine/*' >> $FILES_TO_CHECKOUT echo '!/tools/*' >> $FILES_TO_CHECKOUT echo '/tools/codegen/*' >> $FILES_TO_CHECKOUT echo '!/examples/*' >> $FILES_TO_CHECKOUT From 51e8c58a38687698eeb63af7bf18f48c9f1ed854 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Nov 2023 14:12:16 +0000 Subject: [PATCH 247/813] Revert "Revert "Merge pull request #56543 from rschu1ze/bump-grpc"" This reverts commit d4840adfd4708729ed8f969e61116f58ffe2fb18. --- contrib/grpc | 2 +- contrib/sparse-checkout/update-grpc.sh | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 267af8c3a1e..b723ecae099 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 267af8c3a1ea4a5a4d9e5a070ad2d1ac7c701923 +Subproject commit b723ecae0991bb873fe87a595dfb187178733fde diff --git a/contrib/sparse-checkout/update-grpc.sh b/contrib/sparse-checkout/update-grpc.sh index 4571bd6307d..21628ce8dd1 100755 --- a/contrib/sparse-checkout/update-grpc.sh +++ b/contrib/sparse-checkout/update-grpc.sh @@ -7,6 +7,7 @@ echo '/*' > $FILES_TO_CHECKOUT echo '!/test/*' >> $FILES_TO_CHECKOUT echo '/test/build/*' >> $FILES_TO_CHECKOUT echo '/test/core/tsi/alts/fake_handshaker/*' >> $FILES_TO_CHECKOUT +echo '/test/core/event_engine/fuzzing_event_engine/*' >> $FILES_TO_CHECKOUT echo '!/tools/*' >> $FILES_TO_CHECKOUT echo '/tools/codegen/*' >> $FILES_TO_CHECKOUT echo '!/examples/*' >> $FILES_TO_CHECKOUT From ed7bfeff6999e69e749acacd1d9e20e2828ccb12 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Nov 2023 14:14:13 +0000 Subject: [PATCH 248/813] Update gRPC to not include systemd --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index b723ecae099..6e5e645de7c 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit b723ecae0991bb873fe87a595dfb187178733fde +Subproject commit 6e5e645de7cb0604e3ad4ba55abff2eca38c1803 From 73d058967a94cab6ec52152bea9c7fe50bc0d2ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 13 Nov 2023 12:07:43 +0000 Subject: [PATCH 249/813] Add back flaky tests to analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 6021e7ccf38..8cb921bdd19 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -11,6 +11,7 @@ 01244_optimize_distributed_group_by_sharding_key 01268_mv_scalars 01268_shard_avgweighted +01287_max_execution_speed 01455_shard_leaf_max_rows_bytes_to_read 01495_subqueries_in_with_statement 01560_merge_distributed_join @@ -42,6 +43,8 @@ 00917_multiple_joins_denny_crane 00636_partition_key_parts_pruning 01825_type_json_multiple_files +02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET +02404_memory_bound_merging 02725_agg_projection_resprect_PK 02721_url_cluster 02534_s3_cluster_insert_select_schema_inference From 2ad98a58c9a744ccd43651ed1c14bcf4f07182b2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Nov 2023 16:51:01 +0100 Subject: [PATCH 250/813] Better caching of arrow build (#56657) * Better caching of Arrow build * Fix error * Remove useless files --- contrib/arrow-cmake/CMakeLists.txt | 9 ++- .../arrow-cmake/cpp/src/arrow/util/config.h | 61 +++++++++++++++++++ .../cpp/src/orc/c++/include/orc/orc-config.hh | 38 ++++++++++++ src/Processors/examples/CMakeLists.txt | 6 -- src/Processors/examples/native_orc.cpp | 36 ----------- .../examples/processors_test_aggregation.cpp | 0 ...rocessors_test_merge_sorting_transform.cpp | 0 7 files changed, 103 insertions(+), 47 deletions(-) create mode 100644 contrib/arrow-cmake/cpp/src/arrow/util/config.h create mode 100644 contrib/arrow-cmake/cpp/src/orc/c++/include/orc/orc-config.hh delete mode 100644 src/Processors/examples/native_orc.cpp delete mode 100644 src/Processors/examples/processors_test_aggregation.cpp delete mode 100644 src/Processors/examples/processors_test_merge_sorting_transform.cpp diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index c45d75bb3f2..71133451889 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -109,7 +109,6 @@ set (ORC_CXX_HAS_CSTDINT 1) set (ORC_CXX_HAS_THREAD_LOCAL 1) include(orc_check.cmake) -configure_file("${ORC_INCLUDE_DIR}/orc/orc-config.hh.in" "${ORC_BUILD_INCLUDE_DIR}/orc/orc-config.hh") configure_file("${ORC_SOURCE_SRC_DIR}/Adaptor.hh.in" "${ORC_BUILD_INCLUDE_DIR}/Adaptor.hh") @@ -198,7 +197,9 @@ target_link_libraries(_orc PRIVATE ch_contrib::snappy ch_contrib::zlib ch_contrib::zstd) -target_include_directories(_orc SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR}) +target_include_directories(_orc SYSTEM BEFORE PUBLIC + ${ORC_INCLUDE_DIR} + "${ClickHouse_SOURCE_DIR}/contrib/arrow-cmake/cpp/src/orc/c++/include") target_include_directories(_orc SYSTEM BEFORE PUBLIC ${ORC_BUILD_INCLUDE_DIR}) target_include_directories(_orc SYSTEM PRIVATE ${ORC_SOURCE_SRC_DIR} @@ -212,8 +213,6 @@ target_include_directories(_orc SYSTEM PRIVATE set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/arrow") -configure_file("${LIBRARY_DIR}/util/config.h.cmake" "${CMAKE_CURRENT_BINARY_DIR}/cpp/src/arrow/util/config.h") - # arrow/cpp/src/arrow/CMakeLists.txt (ARROW_SRCS + ARROW_COMPUTE + ARROW_IPC) set(ARROW_SRCS "${LIBRARY_DIR}/array/array_base.cc" @@ -450,7 +449,7 @@ target_link_libraries(_arrow PUBLIC _orc) add_dependencies(_arrow protoc) target_include_directories(_arrow SYSTEM BEFORE PUBLIC ${ARROW_SRC_DIR}) -target_include_directories(_arrow SYSTEM BEFORE PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/cpp/src") +target_include_directories(_arrow SYSTEM BEFORE PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/arrow-cmake/cpp/src") target_include_directories(_arrow SYSTEM PRIVATE ${ARROW_SRC_DIR}) target_include_directories(_arrow SYSTEM PRIVATE ${HDFS_INCLUDE_DIR}) diff --git a/contrib/arrow-cmake/cpp/src/arrow/util/config.h b/contrib/arrow-cmake/cpp/src/arrow/util/config.h new file mode 100644 index 00000000000..cacff7b16cb --- /dev/null +++ b/contrib/arrow-cmake/cpp/src/arrow/util/config.h @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#define ARROW_VERSION_MAJOR 11 +#define ARROW_VERSION_MINOR 0 +#define ARROW_VERSION_PATCH 0 +#define ARROW_VERSION ((ARROW_VERSION_MAJOR * 1000) + ARROW_VERSION_MINOR) * 1000 + ARROW_VERSION_PATCH + +#define ARROW_VERSION_STRING "11.0.0" + +#define ARROW_SO_VERSION "1100" +#define ARROW_FULL_SO_VERSION "1100.0.0" + +#define ARROW_CXX_COMPILER_ID "Clang" +#define ARROW_CXX_COMPILER_VERSION "ClickHouse" +#define ARROW_CXX_COMPILER_FLAGS "" + +#define ARROW_BUILD_TYPE "" + +#define ARROW_GIT_ID "" +#define ARROW_GIT_DESCRIPTION "" + +#define ARROW_PACKAGE_KIND "" + +/* #undef ARROW_COMPUTE */ +/* #undef ARROW_CSV */ +/* #undef ARROW_CUDA */ +/* #undef ARROW_DATASET */ +/* #undef ARROW_FILESYSTEM */ +/* #undef ARROW_FLIGHT */ +/* #undef ARROW_FLIGHT_SQL */ +/* #undef ARROW_IPC */ +/* #undef ARROW_JEMALLOC */ +/* #undef ARROW_JEMALLOC_VENDORED */ +/* #undef ARROW_JSON */ +/* #undef ARROW_ORC */ +/* #undef ARROW_PARQUET */ +/* #undef ARROW_SUBSTRAIT */ + +/* #undef ARROW_GCS */ +/* #undef ARROW_S3 */ +/* #undef ARROW_USE_NATIVE_INT128 */ +/* #undef ARROW_WITH_MUSL */ +/* #undef ARROW_WITH_OPENTELEMETRY */ +/* #undef ARROW_WITH_UCX */ + +/* #undef GRPCPP_PP_INCLUDE */ diff --git a/contrib/arrow-cmake/cpp/src/orc/c++/include/orc/orc-config.hh b/contrib/arrow-cmake/cpp/src/orc/c++/include/orc/orc-config.hh new file mode 100644 index 00000000000..1b0f71ddd40 --- /dev/null +++ b/contrib/arrow-cmake/cpp/src/orc/c++/include/orc/orc-config.hh @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#ifndef ORC_CONFIG_HH +#define ORC_CONFIG_HH + +#define ORC_VERSION "" + +#define ORC_CXX_HAS_CSTDINT + +#ifdef ORC_CXX_HAS_CSTDINT + #include +#else + #include +#endif + +// Following MACROS should be keeped for backward compatibility. +#define ORC_NOEXCEPT noexcept +#define ORC_NULLPTR nullptr +#define ORC_OVERRIDE override +#define ORC_UNIQUE_PTR std::unique_ptr + +#endif diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index 0c8734aee3c..5d43a0d7d08 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -2,9 +2,3 @@ if (TARGET ch_contrib::hivemetastore) clickhouse_add_executable (comma_separated_streams comma_separated_streams.cpp) target_link_libraries (comma_separated_streams PRIVATE dbms) endif() - -if (USE_ORC) - clickhouse_add_executable (native_orc native_orc.cpp) - target_link_libraries (native_orc PRIVATE dbms) - target_include_directories (native_orc PRIVATE ${ClickHouse_SOURCE_DIR}/contrib/orc/c++/include) -endif () diff --git a/src/Processors/examples/native_orc.cpp b/src/Processors/examples/native_orc.cpp deleted file mode 100644 index 201e87b1f56..00000000000 --- a/src/Processors/examples/native_orc.cpp +++ /dev/null @@ -1,36 +0,0 @@ -#include -#include -#include -#include - -using namespace DB; - -int main() -{ - /// Read schema from orc file - String path = "/path/to/orc/file"; - // String path = "/data1/clickhouse_official/data/user_files/bigolive_audience_stats_orc.orc"; - { - ReadBufferFromFile in(path); - NativeORCSchemaReader schema_reader(in, {}); - auto schema = schema_reader.readSchema(); - std::cout << "schema:" << schema.toString() << std::endl; - } - - /// Read schema from string with orc data - { - ReadBufferFromFile in(path); - - String content; - WriteBufferFromString out(content); - - copyData(in, out); - - content.resize(out.count()); - ReadBufferFromString in2(content); - NativeORCSchemaReader schema_reader(in2, {}); - auto schema = schema_reader.readSchema(); - std::cout << "schema:" << schema.toString() << std::endl; - } - return 0; -} diff --git a/src/Processors/examples/processors_test_aggregation.cpp b/src/Processors/examples/processors_test_aggregation.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Processors/examples/processors_test_merge_sorting_transform.cpp b/src/Processors/examples/processors_test_merge_sorting_transform.cpp deleted file mode 100644 index e69de29bb2d..00000000000 From 45f5d2829970902557f0e9df15a3ce1b406c1174 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 1 Nov 2023 23:23:38 +0100 Subject: [PATCH 251/813] Deduplicate concurrent requests in system.replicas --- src/Storages/System/StorageSystemReplicas.cpp | 171 ++++++++++++++++-- src/Storages/System/StorageSystemReplicas.h | 7 +- 2 files changed, 158 insertions(+), 20 deletions(-) diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 1b87382c222..8f7614273a2 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -1,3 +1,6 @@ +#include +#include +#include #include #include #include @@ -13,6 +16,7 @@ #include #include #include +#include "Storages/MergeTree/ReplicatedTableStatus.h" namespace CurrentMetrics @@ -24,9 +28,128 @@ namespace CurrentMetrics namespace DB { +/// Allows to "deduplicate" getStatus() requests for the same table: if a request for a table is already in progress +/// then the new request will return the same future as the previous one. +class StatusRequestsPool +{ + ThreadPool & thread_pool; + + std::mutex mutex; + std::unordered_map> current_requests TSA_GUARDED_BY(mutex); + std::deque>, bool>> requests_to_schedule TSA_GUARDED_BY(mutex); + UInt64 request_id TSA_GUARDED_BY(mutex) = 0; + + Poco::Logger * log; + +public: + explicit StatusRequestsPool(ThreadPool & thread_pool_) + : thread_pool(thread_pool_) + , log(&Poco::Logger::get("StatusRequestsPool")) + {} + + ~StatusRequestsPool() + { + thread_pool.wait(); + } + + /// Make a new request or "attach" to an existing one. + std::pair> addRequest(StoragePtr storage, bool with_zk_fields) + { + std::shared_ptr> promise; + std::shared_future future; + UInt64 this_request_id = 0; + + { + std::lock_guard lock(mutex); + auto existing_request = current_requests.find(storage); + if (existing_request != current_requests.end()) + { + LOG_TEST(log, "Attaching to existing request for table {}", storage->getStorageID().getNameForLogs()); + return {0, existing_request->second}; + } + + promise = std::make_shared>(); + future = promise->get_future().share(); + + current_requests.emplace(storage, future); + + LOG_TEST(log, "Making new request for table {}", storage->getStorageID().getNameForLogs()); + + requests_to_schedule.emplace_back(request_id, storage, promise, with_zk_fields); + this_request_id = request_id; + ++request_id; + } + + return {this_request_id, future}; + } + + /// Schedule requests (if any) that are needed for the current query. This is determined by the maximum request id + /// returned by addRequest. + void scheduleRequests(UInt64 max_request_id) + { + while (true) + { + std::tuple>, bool> req; + { + std::lock_guard lock(mutex); + if (requests_to_schedule.empty()) + break; + + req = requests_to_schedule.front(); + if (std::get<0>(req) > max_request_id) + break; + + requests_to_schedule.pop_front(); + } + + auto & [_, storage, promise, with_zk_fields] = req; + + /// TODO: handle failure when trying to schedule a request: need to either retry or set an error in the promise. + thread_pool.scheduleOrThrowOnError([this, storage, with_zk_fields, promise] () mutable + { + ReplicatedTableStatus status; + if (auto * replicated_table = dynamic_cast(storage.get())) + { + replicated_table->getStatus(status, with_zk_fields); + } + promise->set_value(std::move(status)); + + /// Remove the completed request + { + std::lock_guard l(mutex); + current_requests.erase(storage); + } + }); + } + } +}; + + +class StorageSystemReplicasImpl +{ +public: + explicit StorageSystemReplicasImpl(size_t max_threads) + : thread_pool(CurrentMetrics::SystemReplicasThreads, CurrentMetrics::SystemReplicasThreadsActive, max_threads) + , requests_without_zk_fields(thread_pool) + , requests_with_zk_fields(thread_pool) + {} + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context); + +private: + ThreadPool thread_pool; + StatusRequestsPool requests_without_zk_fields; + StatusRequestsPool requests_with_zk_fields; +}; + StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) : IStorage(table_id_) + , impl(std::make_unique(128)) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription({ @@ -69,6 +192,8 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) setInMemoryMetadata(storage_metadata); } +StorageSystemReplicas::~StorageSystemReplicas() = default; + Pipe StorageSystemReplicas::read( const Names & column_names, @@ -78,6 +203,15 @@ Pipe StorageSystemReplicas::read( QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, const size_t /*num_streams*/) +{ + return impl->read(column_names, storage_snapshot, query_info, context); +} + +Pipe StorageSystemReplicasImpl::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context) { storage_snapshot->check(column_names); @@ -164,30 +298,29 @@ Pipe StorageSystemReplicas::read( MutableColumns res_columns = storage_snapshot->metadata->getSampleBlock().cloneEmptyColumns(); size_t tables_size = col_database->size(); - std::vector statuses(tables_size); - size_t thread_pool_size = std::min(tables_size, static_cast(getNumberOfPhysicalCPUCores())); - auto settings = context->getSettingsRef(); - if (settings.max_threads != 0) - thread_pool_size = std::min(thread_pool_size, static_cast(settings.max_threads)); + /// Use separate queues for requests with and without ZooKeeper fields. + StatusRequestsPool & get_status_requests = with_zk_fields ? requests_with_zk_fields : requests_without_zk_fields; - ThreadPool thread_pool(CurrentMetrics::SystemReplicasThreads, CurrentMetrics::SystemReplicasThreadsActive, thread_pool_size); + std::vector> futures; + futures.reserve(tables_size); + UInt64 max_request_id = 0; + for (size_t i = 0; i < tables_size; ++i) + { + auto & storage = replicated_tables[(*col_database)[i].safeGet()] + [(*col_table)[i].safeGet()]; + + auto [request_id, future] = get_status_requests.addRequest(storage, with_zk_fields); + futures.emplace_back(future); + max_request_id = std::max(max_request_id, request_id); + } + /// Schedule requests up to the maximum request needed for the current query. + /// If there are more requests, they will be scheduled by the query that needs them. + get_status_requests.scheduleRequests(max_request_id); for (size_t i = 0; i < tables_size; ++i) { - thread_pool.scheduleOrThrowOnError([&, my_i = i] - { - dynamic_cast( - *replicated_tables - [(*col_database)[my_i].safeGet()] - [(*col_table)[my_i].safeGet()]).getStatus(statuses[my_i], with_zk_fields); - }); - } - - thread_pool.wait(); - - for (const auto & status: statuses) - { + const auto & status = futures[i].get(); size_t col_num = 3; res_columns[col_num++]->insert(status.is_leader); res_columns[col_num++]->insert(status.can_become_leader); diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index e9c29dec0fd..f6bdfc937ea 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -7,7 +8,7 @@ namespace DB { class Context; - +class StorageSystemReplicasImpl; /** Implements `replicas` system table, which provides information about the status of the replicated tables. */ @@ -15,6 +16,7 @@ class StorageSystemReplicas final : public IStorage { public: explicit StorageSystemReplicas(const StorageID & table_id_); + ~StorageSystemReplicas() override; std::string getName() const override { return "SystemReplicas"; } @@ -28,6 +30,9 @@ public: size_t num_streams) override; bool isSystemStorage() const override { return true; } + +private: + std::unique_ptr impl; }; } From c27913f09a58ac0ddf4f6dd9e651e33d9f87ae48 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 2 Nov 2023 20:44:12 +0100 Subject: [PATCH 252/813] Separate thread pools for request with and w/o zk --- src/Storages/System/StorageSystemReplicas.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 8f7614273a2..19fc9d7e619 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -32,7 +32,7 @@ namespace DB /// then the new request will return the same future as the previous one. class StatusRequestsPool { - ThreadPool & thread_pool; + ThreadPool thread_pool; std::mutex mutex; std::unordered_map> current_requests TSA_GUARDED_BY(mutex); @@ -42,8 +42,8 @@ class StatusRequestsPool Poco::Logger * log; public: - explicit StatusRequestsPool(ThreadPool & thread_pool_) - : thread_pool(thread_pool_) + explicit StatusRequestsPool(size_t max_threads) + : thread_pool(CurrentMetrics::SystemReplicasThreads, CurrentMetrics::SystemReplicasThreadsActive, max_threads) , log(&Poco::Logger::get("StatusRequestsPool")) {} @@ -129,9 +129,8 @@ class StorageSystemReplicasImpl { public: explicit StorageSystemReplicasImpl(size_t max_threads) - : thread_pool(CurrentMetrics::SystemReplicasThreads, CurrentMetrics::SystemReplicasThreadsActive, max_threads) - , requests_without_zk_fields(thread_pool) - , requests_with_zk_fields(thread_pool) + : requests_without_zk_fields(max_threads) + , requests_with_zk_fields(max_threads) {} Pipe read( @@ -141,7 +140,6 @@ public: ContextPtr context); private: - ThreadPool thread_pool; StatusRequestsPool requests_without_zk_fields; StatusRequestsPool requests_with_zk_fields; }; From 1113b3041b1f9a7d5af6a3b73a44ff9b6dc33c59 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 6 Nov 2023 13:12:51 +0100 Subject: [PATCH 253/813] Return exception from the job --- src/Storages/System/StorageSystemReplicas.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 19fc9d7e619..dfabaaaa78b 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -107,12 +107,20 @@ public: /// TODO: handle failure when trying to schedule a request: need to either retry or set an error in the promise. thread_pool.scheduleOrThrowOnError([this, storage, with_zk_fields, promise] () mutable { - ReplicatedTableStatus status; - if (auto * replicated_table = dynamic_cast(storage.get())) + try { - replicated_table->getStatus(status, with_zk_fields); + ReplicatedTableStatus status; + if (auto * replicated_table = dynamic_cast(storage.get())) + { + replicated_table->getStatus(status, with_zk_fields); + } + promise->set_value(std::move(status)); + } + catch (...) + { + tryLogCurrentException(log, "Error getting status for table " + storage->getStorageID().getNameForLogs()); + promise->set_exception(std::current_exception()); } - promise->set_value(std::move(status)); /// Remove the completed request { From 6e861a617616268ffb1c81758ab366cd72869060 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 6 Nov 2023 22:05:07 +0100 Subject: [PATCH 254/813] Handle query cancellation --- src/Storages/System/StorageSystemReplicas.cpp | 77 ++++++++++++++----- 1 file changed, 58 insertions(+), 19 deletions(-) diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index dfabaaaa78b..0e371c0e736 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -9,14 +9,14 @@ #include #include #include +#include +#include #include #include #include #include #include #include -#include -#include "Storages/MergeTree/ReplicatedTableStatus.h" namespace CurrentMetrics @@ -32,11 +32,22 @@ namespace DB /// then the new request will return the same future as the previous one. class StatusRequestsPool { +public: + struct RequestInfo + { + UInt64 request_id = 0; + std::shared_future future; + }; + +private: ThreadPool thread_pool; std::mutex mutex; - std::unordered_map> current_requests TSA_GUARDED_BY(mutex); + /// All requests from the queries that are currently being executed. + std::unordered_map current_requests TSA_GUARDED_BY(mutex); + /// Requests that were added by currently executing queries but have not been scheduled yet. std::deque>, bool>> requests_to_schedule TSA_GUARDED_BY(mutex); + /// Used to assign unique incremental ids to requests. UInt64 request_id TSA_GUARDED_BY(mutex) = 0; Poco::Logger * log; @@ -53,7 +64,7 @@ public: } /// Make a new request or "attach" to an existing one. - std::pair> addRequest(StoragePtr storage, bool with_zk_fields) + RequestInfo addRequest(StoragePtr storage, bool with_zk_fields) { std::shared_ptr> promise; std::shared_future future; @@ -65,19 +76,20 @@ public: if (existing_request != current_requests.end()) { LOG_TEST(log, "Attaching to existing request for table {}", storage->getStorageID().getNameForLogs()); - return {0, existing_request->second}; + return existing_request->second; } + this_request_id = request_id; + ++request_id; + promise = std::make_shared>(); future = promise->get_future().share(); - current_requests.emplace(storage, future); + current_requests[storage] = { .request_id = this_request_id, .future = future }; LOG_TEST(log, "Making new request for table {}", storage->getStorageID().getNameForLogs()); - requests_to_schedule.emplace_back(request_id, storage, promise, with_zk_fields); - this_request_id = request_id; - ++request_id; + requests_to_schedule.emplace_back(this_request_id, storage, promise, with_zk_fields); } return {this_request_id, future}; @@ -85,10 +97,14 @@ public: /// Schedule requests (if any) that are needed for the current query. This is determined by the maximum request id /// returned by addRequest. - void scheduleRequests(UInt64 max_request_id) + void scheduleRequests(UInt64 max_request_id, QueryStatusPtr query_status) { while (true) { + if (query_status) + query_status->checkTimeLimit(); + + /// Try to pick up a request to schedule std::tuple>, bool> req; { std::lock_guard lock(mutex); @@ -96,6 +112,8 @@ public: break; req = requests_to_schedule.front(); + + /// Check if all requests for the current query have been scheduled if (std::get<0>(req) > max_request_id) break; @@ -104,8 +122,7 @@ public: auto & [_, storage, promise, with_zk_fields] = req; - /// TODO: handle failure when trying to schedule a request: need to either retry or set an error in the promise. - thread_pool.scheduleOrThrowOnError([this, storage, with_zk_fields, promise] () mutable + auto get_status_task = [this, storage, with_zk_fields, promise] () mutable { try { @@ -122,14 +139,28 @@ public: promise->set_exception(std::current_exception()); } - /// Remove the completed request - { - std::lock_guard l(mutex); - current_requests.erase(storage); - } - }); + completeRequest(storage); + }; + + try + { + thread_pool.scheduleOrThrowOnError(std::move(get_status_task)); + } + catch (...) + { + tryLogCurrentException(log, "Error scheduling get status task for table " + storage->getStorageID().getNameForLogs()); + promise->set_exception(std::current_exception()); + completeRequest(storage); + } } } + +private: + void completeRequest(StoragePtr storage) + { + std::lock_guard lock(mutex); + current_requests.erase(storage); + } }; @@ -308,11 +339,16 @@ Pipe StorageSystemReplicasImpl::read( /// Use separate queues for requests with and without ZooKeeper fields. StatusRequestsPool & get_status_requests = with_zk_fields ? requests_with_zk_fields : requests_without_zk_fields; + QueryStatusPtr query_status = context ? context->getProcessListElement() : nullptr; + std::vector> futures; futures.reserve(tables_size); UInt64 max_request_id = 0; for (size_t i = 0; i < tables_size; ++i) { + if (query_status) + query_status->checkTimeLimit(); + auto & storage = replicated_tables[(*col_database)[i].safeGet()] [(*col_table)[i].safeGet()]; @@ -322,10 +358,13 @@ Pipe StorageSystemReplicasImpl::read( } /// Schedule requests up to the maximum request needed for the current query. /// If there are more requests, they will be scheduled by the query that needs them. - get_status_requests.scheduleRequests(max_request_id); + get_status_requests.scheduleRequests(max_request_id, query_status); for (size_t i = 0; i < tables_size; ++i) { + if (query_status) + query_status->checkTimeLimit(); + const auto & status = futures[i].get(); size_t col_num = 3; res_columns[col_num++]->insert(status.is_leader); From 54cc193d01b90663d94764e72d1603786068e045 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 7 Nov 2023 21:41:39 +0100 Subject: [PATCH 255/813] Test with many concurrent queries to system.replicas --- ...many_requests_to_system_replicas.reference | 5 ++ .../02908_many_requests_to_system_replicas.sh | 48 +++++++++++++++++++ 2 files changed, 53 insertions(+) create mode 100644 tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference create mode 100755 tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference new file mode 100644 index 00000000000..d7850e59dec --- /dev/null +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference @@ -0,0 +1,5 @@ +Creating 300 tables +Making making 500 requests to system.replicas +Query system.replicas while waiting for other concurrent requests to finish +0 +900 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 new file mode 100755 index 00000000000..70dc5f4d8c4 --- /dev/null +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash +# Tags: long, zookeeper, no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +NUM_TABLES=300 +CONCURRENCY=500 + +echo "Creating $NUM_TABLES tables" + +function init_table() +{ + i=$1 + curl $CLICKHOUSE_URL --silent --fail --data "CREATE TABLE test_02908_r1_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r1') ORDER BY tuple()" + curl $CLICKHOUSE_URL --silent --fail --data "CREATE TABLE test_02908_r2_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r2') ORDER BY tuple()" + curl $CLICKHOUSE_URL --silent --fail --data "CREATE TABLE test_02908_r3_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r3') ORDER BY tuple()" + + curl $CLICKHOUSE_URL --silent --fail --data "INSERT INTO test_02908_r1_$i SELECT rand64() FROM numbers(5);" +} + +export init_table; + +for i in `seq 1 $NUM_TABLES`; +do + init_table $i & +done + +wait; + + +echo "Making making $CONCURRENCY requests to system.replicas" + +for i in `seq 1 $CONCURRENCY`; +do + curl $CLICKHOUSE_URL --silent --fail --data "SELECT * FROM system.replicas WHERE database=currentDatabase() FORMAT Null;" & +done + +echo "Query system.replicas while waiting for other concurrent requests to finish" +# lost_part_count column is read from ZooKeeper +curl $CLICKHOUSE_URL --silent --fail --data "SELECT sum(lost_part_count) FROM system.replicas WHERE database=currentDatabase();"; +# is_leader column is filled without ZooKeeper +curl $CLICKHOUSE_URL --silent --fail --data "SELECT sum(is_leader) FROM system.replicas WHERE database=currentDatabase();"; + +wait; From b49f50bf47f3c55d10a39fa437b9ada2f82143b3 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 13 Nov 2023 14:56:14 +0100 Subject: [PATCH 256/813] Set DB:Exception on unscheduled requests --- src/Storages/System/StorageSystemReplicas.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 0e371c0e736..ffefd41327d 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -28,6 +28,11 @@ namespace CurrentMetrics namespace DB { +namespace ErrorCodes +{ + extern const int QUERY_WAS_CANCELLED; +} + /// Allows to "deduplicate" getStatus() requests for the same table: if a request for a table is already in progress /// then the new request will return the same future as the previous one. class StatusRequestsPool @@ -61,6 +66,10 @@ public: ~StatusRequestsPool() { thread_pool.wait(); + /// Cancel unscheduled requests + for (auto & request : requests_to_schedule) + std::get<2>(request)->set_exception(std::make_exception_ptr( + DB::Exception(ErrorCodes::QUERY_WAS_CANCELLED, "StatusRequestsPool is destroyed"))); } /// Make a new request or "attach" to an existing one. From 64d5f636a26b28e2240a89648de94c6ec8449a6f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 13 Nov 2023 18:07:25 +0100 Subject: [PATCH 257/813] Do not fetch submodules during preparation performance archive --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 150ce1ab385..ba9b104b40b 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -177,7 +177,7 @@ then tar c -C /build/ --exclude='.git/modules/**' .git | tar x -C "$PERF_OUTPUT"/ch # Create branch pr and origin/master to have them for the following performance comparison git -C "$PERF_OUTPUT"/ch branch pr - git -C "$PERF_OUTPUT"/ch fetch --no-tags --depth 50 origin master:origin/master + git -C "$PERF_OUTPUT"/ch fetch --no-tags --no-recurse-submodules --depth 50 origin master:origin/master # Clean remote, to not have it stale git -C "$PERF_OUTPUT"/ch remote | xargs -n1 git -C "$PERF_OUTPUT"/ch remote remove # And clean all tags From 4fdca75daba49fca36c73685347ecd7889fd3933 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 13 Nov 2023 17:08:40 +0000 Subject: [PATCH 258/813] Change cluster name --- .../0_stateless/02911_row_policy_on_cluster.sql | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02911_row_policy_on_cluster.sql b/tests/queries/0_stateless/02911_row_policy_on_cluster.sql index 4a132e889a9..6e94e301733 100644 --- a/tests/queries/0_stateless/02911_row_policy_on_cluster.sql +++ b/tests/queries/0_stateless/02911_row_policy_on_cluster.sql @@ -1,10 +1,10 @@ -- Tags: no-parallel, zookeeper -DROP ROW POLICY IF EXISTS 02911_rowpolicy ON default.* ON CLUSTER default; -DROP USER IF EXISTS 02911_user ON CLUSTER default; +DROP ROW POLICY IF EXISTS 02911_rowpolicy ON default.* ON CLUSTER test_shard_localhost; +DROP USER IF EXISTS 02911_user ON CLUSTER test_shard_localhost; -CREATE USER 02911_user ON CLUSTER default; -CREATE ROW POLICY 02911_rowpolicy ON CLUSTER default ON default.* USING 1 TO 02911_user; +CREATE USER 02911_user ON CLUSTER test_shard_localhost; +CREATE ROW POLICY 02911_rowpolicy ON CLUSTER test_shard_localhost ON default.* USING 1 TO 02911_user; -DROP ROW POLICY 02911_rowpolicy ON default.* ON CLUSTER default; -DROP USER 02911_user ON CLUSTER default; +DROP ROW POLICY 02911_rowpolicy ON default.* ON CLUSTER test_shard_localhost; +DROP USER 02911_user ON CLUSTER test_shard_localhost; From 38d4d669981f8760b0aa1ddb1d2172777db81e3f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 13 Nov 2023 18:43:42 +0100 Subject: [PATCH 259/813] Fix missing argument for style_check.py in master workflow --- .github/workflows/master.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 260fc2fc7d5..e662a5b6f98 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -96,7 +96,7 @@ jobs: runner_type: style-checker run_command: | cd "$REPO_COPY/tests/ci" - python3 style_check.py + python3 style_check.py --no-push CompatibilityCheckX86: needs: [BuilderDebRelease] uses: ./.github/workflows/reusable_test.yml From d92e3329234005483a5f6f77baf7c03afa2b96bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 13 Nov 2023 19:33:56 +0100 Subject: [PATCH 260/813] Fix build --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 9c8952aea96..9e548c5a6d0 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -554,7 +554,7 @@ if (ENABLE_RUST) endif() endif() -if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND NOT SANITIZE AND OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64)) set(CHECK_LARGE_OBJECT_SIZES_DEFAULT ON) else () set(CHECK_LARGE_OBJECT_SIZES_DEFAULT OFF) From ed95698ff6137962f1bd40d557d50bf4260ddd17 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 13 Nov 2023 19:38:11 +0100 Subject: [PATCH 261/813] Decrease unnecessary verbosity --- docker/packager/binary/build.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index ba9b104b40b..6b6374d08c9 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -181,7 +181,8 @@ then # Clean remote, to not have it stale git -C "$PERF_OUTPUT"/ch remote | xargs -n1 git -C "$PERF_OUTPUT"/ch remote remove # And clean all tags - git -C "$PERF_OUTPUT"/ch tag | xargs git -C "$PERF_OUTPUT"/ch tag -d + echo "Deleting $(git -C "$PERF_OUTPUT"/ch tag | wc -l) tags" + git -C "$PERF_OUTPUT"/ch tag | xargs git -C "$PERF_OUTPUT"/ch tag -d >/dev/null git -C "$PERF_OUTPUT"/ch reset --soft pr git -C "$PERF_OUTPUT"/ch log -5 ( From 1fe986bff29f1d2fae4d59fcd79d0b9a504db19f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Nov 2023 18:59:09 +0000 Subject: [PATCH 262/813] Fix duplicate usage of table function input(). --- ...input_table_function_in_subquery.reference | 0 .../02915_input_table_function_in_subquery.sh | 40 +++++++++++++++++++ 2 files changed, 40 insertions(+) create mode 100644 tests/queries/0_stateless/02915_input_table_function_in_subquery.reference create mode 100755 tests/queries/0_stateless/02915_input_table_function_in_subquery.sh diff --git a/tests/queries/0_stateless/02915_input_table_function_in_subquery.reference b/tests/queries/0_stateless/02915_input_table_function_in_subquery.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh b/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh new file mode 100755 index 00000000000..80e38338751 --- /dev/null +++ b/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash +# Tags: no-random-merge-tree-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " +CREATE TABLE IF NOT EXISTS ts_data_double_raw +( + device_id UInt32 NOT NULL CODEC(ZSTD), + data_item_id UInt32 NOT NULL CODEC(ZSTD), + data_time DateTime64(3, 'UTC') NOT NULL CODEC(Delta, ZSTD), + data_value Float64 NOT NULL CODEC(Delta, ZSTD), + is_deleted Bool CODEC(ZSTD), + ingestion_time DateTime64(3, 'UTC') NOT NULL CODEC(Delta, ZSTD) +) +ENGINE = ReplacingMergeTree +PARTITION BY toYYYYMM(data_time) +ORDER BY (device_id, data_item_id, data_time) +SETTINGS index_granularity = 8192; + + +CREATE VIEW ts_data_double AS +SELECT + device_id, + data_item_id, + data_time, + argMax(data_value, ingestion_time) data_value, + max(ingestion_time) version, + argMax(is_deleted, ingestion_time) is_deleted +FROM ts_data_double_raw +GROUP BY device_id, data_item_id, data_time +HAVING is_deleted = 0; + +INSERT INTO ts_data_double_raw VALUES (100, 1, fromUnixTimestamp64Milli(1697547086760), 3.6, false, fromUnixTimestamp64Milli(1)), (100, 1, fromUnixTimestamp64Milli(1697547086761), 4.6, false, fromUnixTimestamp64Milli(1)); +INSERT INTO ts_data_double_raw VALUES (100, 1, fromUnixTimestamp64Milli(1697547086760), 3.6, true, fromUnixTimestamp64Milli(5)), (100, 1, fromUnixTimestamp64Milli(1697547086761), 4.6, false, fromUnixTimestamp64Milli(4)); +" + +$CLICKHOUSE_CLIENT -q "select 1697547086760 format RowBinary" | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20ts_data_double_raw%20%28device_id%2C%20data_item_id%2C%20data_time%2C%20data_value%2C%20is_deleted%2C%20ingestion_time%29%0ASELECT%0A%20%20%20device_id%2C%0A%20%20%20data_item_id%2C%0A%20%20%20data_time%2C%0A%20%20%20data_value%2C%0A%20%20%201%2C%20%20--%20mark%20as%20deleted%0A%20%20%20fromUnixTimestamp64Milli%281697547088995%2C%20%27UTC%27%29%20--%20all%20inserted%20records%20have%20new%20ingestion%20time%0AFROM%20ts_data_double%0AWHERE%20%28device_id%20%3D%20100%29%20AND%20%28data_item_id%20%3D%201%29%0A%20%20%20%20AND%20%28data_time%20%3E%3D%20fromUnixTimestamp64Milli%280%2C%20%27UTC%27%29%29%0A%20%20%20%20AND%20%28data_time%20%3C%3D%20fromUnixTimestamp64Milli%281697547086764%2C%20%27UTC%27%29%29%0A%20%20%20%20AND%20version%20%3C%20fromUnixTimestamp64Milli%281697547088995%2C%20%27UTC%27%29%0A%20%20%20%20AND%20%28toUnixTimestamp64Milli%28data_time%29%20IN%20%28SELECT%20timestamp%20FROM%20input%28%27timestamp%20UInt64%27%29%29%29%20SETTINGS%20insert_quorum%3D1%0A%20FORMAT%20RowBinary" --data-binary @- From 4004248c13d6e4b191e0b00ece7b4d30281f40e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Nov 2023 19:01:36 +0000 Subject: [PATCH 263/813] Fix duplicate usage of table function input(). --- .../PredicateExpressionsOptimizer.cpp | 15 +++++ src/Storages/StorageInput.cpp | 55 ++++++++++++++++--- src/Storages/StorageInput.h | 6 +- 3 files changed, 67 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index e64ff34b11f..885c99aeb90 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -53,6 +53,18 @@ bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query) return false; } +static bool hasInputTableFunction(const ASTPtr & expr) +{ + if (const auto * func = typeid_cast(expr.get()); func && func->name == "input") + return true; + + for (const auto & child : expr->children) + if (hasInputTableFunction(child)) + return true; + + return false; +} + std::vector PredicateExpressionsOptimizer::extractTablesPredicates(const ASTPtr & where, const ASTPtr & prewhere) { std::vector tables_predicates(tables_with_columns.size()); @@ -72,6 +84,9 @@ std::vector PredicateExpressionsOptimizer::extractTablesPredicates(const A return {}; /// Not optimized when predicate contains stateful function or indeterministic function or window functions } + if (hasInputTableFunction(predicate_expression)) + return {}; /// Not optimized when predicate contains input table function + if (!expression_info.is_array_join) { if (expression_info.unique_reference_tables_pos.size() == 1) diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 0ad3afb2e8a..2314d3fb581 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -5,8 +5,10 @@ #include #include +#include +#include #include - +#include namespace DB { @@ -47,11 +49,33 @@ public: void StorageInput::setPipe(Pipe pipe_) { pipe = std::move(pipe_); + was_pipe_initialized = true; } +class ReadFromInput : public ISourceStep +{ +public: + std::string getName() const override { return "ReadFromInput"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; -Pipe StorageInput::read( - const Names & /*column_names*/, + ReadFromInput( + Block sample_block, + //StorageSnapshotPtr storage_snapshot_, + StorageInput & storage_) + : ISourceStep(DataStream{.header = std::move(sample_block)}) + //, storage_snapshot(std::move(storage_snapshot_)) + , storage(storage_) + { + } + +private: + //StorageSnapshotPtr storage_snapshot; + StorageInput & storage; +}; + +void StorageInput::read( + QueryPlan & query_plan, + const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & /*query_info*/, ContextPtr context, @@ -59,20 +83,35 @@ Pipe StorageInput::read( size_t /*max_block_size*/, size_t /*num_streams*/) { - Pipes pipes; + storage_snapshot->check(column_names); + Block sample_block = storage_snapshot->metadata->getSampleBlock(); + auto query_context = context->getQueryContext(); /// It is TCP request if we have callbacks for input(). - if (query_context->getInputBlocksReaderCallback()) + if (!was_pipe_initialized && query_context->getInputBlocksReaderCallback()) { /// Send structure to the client. query_context->initializeInput(shared_from_this()); - return Pipe(std::make_shared(query_context, storage_snapshot->metadata->getSampleBlock())); } - if (pipe.empty()) + if (!was_pipe_initialized) throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "Input stream is not initialized, input() must be used only in INSERT SELECT query"); - return std::move(pipe); + auto reading = std::make_unique( + std::move(sample_block), + //storage_snapshot, + *this); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromInput::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + if (storage.was_pipe_used) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to read from input() twice."); + + pipeline.init(std::move(storage.pipe)); + storage.was_pipe_used = true; } } diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index da4669aaf37..9ac082a9add 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -10,6 +10,7 @@ namespace DB class StorageInput final : public IStorage { + friend class ReadFromInput; public: StorageInput(const StorageID & table_id, const ColumnsDescription & columns_); @@ -18,7 +19,8 @@ public: /// A table will read from this stream. void setPipe(Pipe pipe_); - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -29,5 +31,7 @@ public: private: Pipe pipe; + bool was_pipe_initialized = false; + bool was_pipe_used = false; }; } From d3517140d9c471411fba8d7d98a6f7da7bd48172 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Mon, 13 Nov 2023 19:03:27 +0000 Subject: [PATCH 264/813] Revert "Revert "Add /keeper/availability-zone node to allow server load balancing within AZ."" This reverts commit e5aad4a6fd308e49c16d36a4bb04462cc2574bb8. --- src/Common/ErrorCodes.cpp | 1 + src/Coordination/KeeperConstants.h | 1 + src/Coordination/KeeperContext.cpp | 10 +- src/Coordination/KeeperContext.h | 3 +- src/Coordination/KeeperDispatcher.cpp | 12 +- src/Coordination/KeeperStorage.cpp | 3 +- src/IO/S3/Credentials.cpp | 155 ++++++++++++++---- src/IO/S3/Credentials.h | 23 ++- tests/integration/helpers/keeper_config1.xml | 1 + tests/integration/helpers/keeper_config2.xml | 1 + .../test_keeper_availability_zone/__init__.py | 0 .../configs/keeper_config.xml | 2 + .../test_keeper_availability_zone/test.py | 31 ++++ .../test_keeper_four_word_command/test.py | 8 +- 14 files changed, 212 insertions(+), 39 deletions(-) create mode 100644 tests/integration/test_keeper_availability_zone/__init__.py create mode 100644 tests/integration/test_keeper_availability_zone/configs/keeper_config.xml create mode 100644 tests/integration/test_keeper_availability_zone/test.py diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 28f8e6c6021..8e0d7fabf6b 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -586,6 +586,7 @@ M(704, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ M(705, TABLE_NOT_EMPTY) \ M(706, LIBSSH_ERROR) \ + M(707, GCP_ERROR) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 5a52fbf272b..08a7c85585a 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -17,5 +17,6 @@ const String keeper_system_path = "/keeper"; const String keeper_api_version_path = keeper_system_path + "/api_version"; const String keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; const String keeper_config_path = keeper_system_path + "/config"; +const String keeper_availability_zone_path = keeper_system_path + "/availability_zone"; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 25bfb6c6384..1cee2a8e446 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -32,9 +32,17 @@ KeeperContext::KeeperContext(bool standalone_keeper_) system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_) +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, const std::string & environment_az) { dispatcher = dispatcher_; + + /// We only use the environment availability zone when configuration option is missing. + auto keeper_az = config.getString("keeper_server.availability_zone", environment_az); + if (!keeper_az.empty()) + system_nodes_with_data[keeper_availability_zone_path] = keeper_az; + LOG_INFO(&Poco::Logger::get("KeeperContext"), + "Initialize the KeeperContext with availability zone: '{}', environment availability zone '{}'. ", keeper_az, environment_az); + digest_enabled = config.getBool("keeper_server.digest_enabled", false); ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 2485f3d6c55..71f00ce5a93 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -3,7 +3,6 @@ #include #include #include - #include #include @@ -24,7 +23,7 @@ public: SHUTDOWN }; - void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_); + void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, const std::string & environment_az); Phase getServerState() const; void setServerState(Phase server_state_); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index ca454c18084..6d43445474d 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -370,7 +371,16 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf configuration_and_settings = KeeperConfigurationAndSettings::loadFromConfig(config, standalone_keeper); keeper_context = std::make_shared(standalone_keeper); - keeper_context->initialize(config, this); + String availability_zone; + try + { + availability_zone = DB::S3::getRunningAvailabilityZone(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + keeper_context->initialize(config, this, availability_zone); requests_queue = std::make_unique(configuration_and_settings->coordination_settings->max_request_queue_size); request_thread = ThreadFromGlobalPool([this] { requestThread(); }); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index a4bb65a2a72..0d1d07ec7c5 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1081,7 +1081,8 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); if (request.path == Coordination::keeper_api_feature_flags_path - || request.path == Coordination::keeper_config_path) + || request.path == Coordination::keeper_config_path + || request.path == Coordination::keeper_availability_zone_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 1eea167e1b9..eb87568c8ab 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,4 +1,9 @@ +#include +#include #include +#include +#include +#include "Common/Exception.h" #if USE_AWS_S3 @@ -11,6 +16,7 @@ # include # include +# include # include # include @@ -22,6 +28,16 @@ # include # include +#include +#include + + +#include +#include +#include +#include +#include + namespace DB { @@ -29,6 +45,8 @@ namespace DB namespace ErrorCodes { extern const int AWS_ERROR; + extern const int GCP_ERROR; + extern const int UNSUPPORTED_METHOD; } namespace S3 @@ -151,30 +169,6 @@ Aws::String AWSEC2MetadataClient::getDefaultCredentialsSecurely() const return GetResourceWithAWSWebServiceResult(credentials_request).GetPayload(); } -Aws::String AWSEC2MetadataClient::getCurrentAvailabilityZone() const -{ - String user_agent_string = awsComputeUserAgentString(); - auto [new_token, response_code] = getEC2MetadataToken(user_agent_string); - if (response_code != Aws::Http::HttpResponseCode::OK || new_token.empty()) - throw DB::Exception(ErrorCodes::AWS_ERROR, - "Failed to make token request. HTTP response code: {}", response_code); - - token = std::move(new_token); - const String url = endpoint + EC2_AVAILABILITY_ZONE_RESOURCE; - std::shared_ptr profile_request( - Aws::Http::CreateHttpRequest(url, Aws::Http::HttpMethod::HTTP_GET, Aws::Utils::Stream::DefaultResponseStreamFactoryMethod)); - - profile_request->SetHeaderValue(EC2_IMDS_TOKEN_HEADER, token); - profile_request->SetUserAgent(user_agent_string); - - const auto result = GetResourceWithAWSWebServiceResult(profile_request); - if (result.GetResponseCode() != Aws::Http::HttpResponseCode::OK) - throw DB::Exception(ErrorCodes::AWS_ERROR, - "Failed to get availability zone. HTTP response code: {}", result.GetResponseCode()); - - return Aws::Utils::StringUtils::Trim(result.GetPayload().c_str()); -} - std::pair AWSEC2MetadataClient::getEC2MetadataToken(const std::string & user_agent_string) const { std::lock_guard locker(token_mutex); @@ -199,10 +193,10 @@ Aws::String AWSEC2MetadataClient::getCurrentRegion() const return Aws::Region::AWS_GLOBAL; } -std::shared_ptr InitEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration) +static Aws::String getAWSMetadataEndpoint() { - Aws::String ec2_metadata_service_endpoint = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT"); auto * logger = &Poco::Logger::get("AWSEC2InstanceProfileConfigLoader"); + Aws::String ec2_metadata_service_endpoint = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT"); if (ec2_metadata_service_endpoint.empty()) { Aws::String ec2_metadata_service_endpoint_mode = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT_MODE"); @@ -233,8 +227,95 @@ std::shared_ptr InitEC2MetadataClient(const Aws::Client::C } } } - LOG_INFO(logger, "Using IMDS endpoint: {}", ec2_metadata_service_endpoint); - return std::make_shared(client_configuration, ec2_metadata_service_endpoint.c_str()); + return ec2_metadata_service_endpoint; +} + +std::shared_ptr InitEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration) +{ + auto endpoint = getAWSMetadataEndpoint(); + return std::make_shared(client_configuration, endpoint.c_str()); +} + +String AWSEC2MetadataClient::getAvailabilityZoneOrException() +{ + Poco::URI uri(getAWSMetadataEndpoint() + EC2_AVAILABILITY_ZONE_RESOURCE); + Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort()); + + Poco::Net::HTTPResponse response; + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri.getPath()); + session.sendRequest(request); + + std::istream & rs = session.receiveResponse(response); + if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) + throw DB::Exception(ErrorCodes::AWS_ERROR, "Failed to get AWS availability zone. HTTP response code: {}", response.getStatus()); + String response_data; + Poco::StreamCopier::copyToString(rs, response_data); + return response_data; +} + +String getGCPAvailabilityZoneOrException() +{ + Poco::URI uri(String(GCP_METADATA_SERVICE_ENDPOINT) + "/computeMetadata/v1/instance/zone"); + Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort()); + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri.getPath()); + Poco::Net::HTTPResponse response; + request.set("Metadata-Flavor", "Google"); + session.sendRequest(request); + std::istream & rs = session.receiveResponse(response); + if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) + throw DB::Exception(ErrorCodes::GCP_ERROR, "Failed to get GCP availability zone. HTTP response code: {}", response.getStatus()); + String response_data; + Poco::StreamCopier::copyToString(rs, response_data); + Strings zone_info; + boost::split(zone_info, response_data, boost::is_any_of("/")); + /// We expect GCP returns a string as "projects/123456789/zones/us-central1a". + if (zone_info.size() != 4) + throw DB::Exception(ErrorCodes::GCP_ERROR, "Invalid format of GCP zone information, expect projects//zones/, got {}", response_data); + return zone_info[3]; +} + +String getRunningAvailabilityZoneImpl() +{ + LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); + try + { + auto aws_az = AWSEC2MetadataClient::getAvailabilityZoneOrException(); + return aws_az; + } + catch (const DB::Exception & aws_ex) + { + try + { + auto gcp_zone = getGCPAvailabilityZoneOrException(); + return gcp_zone; + } + catch (const DB::Exception & gcp_ex) + { + throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Failed to find the availability zone, tried AWS and GCP. AWS Error: {}\nGCP Error: {}", aws_ex.displayText(), gcp_ex.displayText()); + } + } +} + +std::variant getRunningAvailabilityZoneImplOrException() +{ + try + { + return getRunningAvailabilityZoneImpl(); + } + catch (...) + { + return std::current_exception(); + } +} + +String getRunningAvailabilityZone() +{ + static auto az_or_exception = getRunningAvailabilityZoneImplOrException(); + if (const auto * az = std::get_if(&az_or_exception)) + return *az; + else + std::rethrow_exception(std::get(az_or_exception)); } AWSEC2InstanceProfileConfigLoader::AWSEC2InstanceProfileConfigLoader(const std::shared_ptr & client_, bool use_secure_pull_) @@ -703,7 +784,6 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( aws_client_configuration.requestTimeoutMs = 1000; aws_client_configuration.retryStrategy = std::make_shared(1, 1000); - auto ec2_metadata_client = InitEC2MetadataClient(aws_client_configuration); auto config_loader = std::make_shared(ec2_metadata_client, !credentials_configuration.use_insecure_imds_request); @@ -721,4 +801,21 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( } +#else + +namespace DB +{ + +namespace S3 +{ + +String getRunningAvailabilityZone() +{ + throw Poco::Exception("Does not support availability zone detection for non-cloud environment"); +} + +} + +} + #endif diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 0243e8e4986..a978679348f 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -1,5 +1,8 @@ #pragma once +#include +#include +#include #include "config.h" #if USE_AWS_S3 @@ -18,6 +21,12 @@ namespace DB::S3 inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; +/// In GCP metadata service can be accessed via DNS regardless of IPv4 or IPv6. +static constexpr char GCP_METADATA_SERVICE_ENDPOINT[] = "http://metadata.google.internal"; + +/// getRunningAvailabilityZone returns the availability zone of the underlying compute resources where the current process runs. +String getRunningAvailabilityZone(); + class AWSEC2MetadataClient : public Aws::Internal::AWSHttpResourceClient { static constexpr char EC2_SECURITY_CREDENTIALS_RESOURCE[] = "/latest/meta-data/iam/security-credentials"; @@ -50,10 +59,11 @@ public: virtual Aws::String getCurrentRegion() const; - virtual Aws::String getCurrentAvailabilityZone() const; + friend String getRunningAvailabilityZoneImpl(); private: std::pair getEC2MetadataToken(const std::string & user_agent_string) const; + static String getAvailabilityZoneOrException(); const Aws::String endpoint; mutable std::recursive_mutex token_mutex; @@ -177,4 +187,15 @@ public: } +#else + +namespace DB +{ + +namespace S3 +{ +String getRunningAvailabilityZone(); +} + +} #endif diff --git a/tests/integration/helpers/keeper_config1.xml b/tests/integration/helpers/keeper_config1.xml index 7702aecba9c..29232989084 100644 --- a/tests/integration/helpers/keeper_config1.xml +++ b/tests/integration/helpers/keeper_config1.xml @@ -11,6 +11,7 @@ 2181 + az-zoo1 1 diff --git a/tests/integration/helpers/keeper_config2.xml b/tests/integration/helpers/keeper_config2.xml index 2a1a1c1003c..2601efbb313 100644 --- a/tests/integration/helpers/keeper_config2.xml +++ b/tests/integration/helpers/keeper_config2.xml @@ -12,6 +12,7 @@ 2181 2 + az-zoo2 10000 diff --git a/tests/integration/test_keeper_availability_zone/__init__.py b/tests/integration/test_keeper_availability_zone/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_availability_zone/configs/keeper_config.xml b/tests/integration/test_keeper_availability_zone/configs/keeper_config.xml new file mode 100644 index 00000000000..3cbf717bb67 --- /dev/null +++ b/tests/integration/test_keeper_availability_zone/configs/keeper_config.xml @@ -0,0 +1,2 @@ + + diff --git a/tests/integration/test_keeper_availability_zone/test.py b/tests/integration/test_keeper_availability_zone/test.py new file mode 100644 index 00000000000..1836f0e679b --- /dev/null +++ b/tests/integration/test_keeper_availability_zone/test.py @@ -0,0 +1,31 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.keeper_utils import KeeperClient + + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/keeper_config.xml"], + with_zookeeper=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_get_availability_zone(): + with KeeperClient.from_cluster(cluster, "zoo1") as client1: + assert client1.get("/keeper/availability_zone") == "az-zoo1" + + with KeeperClient.from_cluster(cluster, "zoo2") as client2: + assert client2.get("/keeper/availability_zone") == "az-zoo2" diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 71501133ae7..25c4bc55327 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -183,8 +183,8 @@ def test_cmd_mntr(started_cluster): # contains: # 10 nodes created by test # 3 nodes created by clickhouse "/clickhouse/task_queue/ddl" - # 1 root node, 3 keeper system nodes - assert int(result["zk_znode_count"]) == 14 + # 1 root node, 4 keeper system nodes + assert int(result["zk_znode_count"]) == 15 assert int(result["zk_watch_count"]) == 2 assert int(result["zk_ephemerals_count"]) == 2 assert int(result["zk_approximate_data_size"]) > 0 @@ -333,7 +333,7 @@ def test_cmd_srvr(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"], 16) > 10 assert result["Mode"] == "leader" - assert result["Node count"] == "14" + assert result["Node count"] == "15" finally: destroy_zk_client(zk) @@ -373,7 +373,7 @@ def test_cmd_stat(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"], 16) >= 10 assert result["Mode"] == "leader" - assert result["Node count"] == "14" + assert result["Node count"] == "15" # filter connection statistics cons = [n for n in data.split("\n") if "=" in n] From 0622379ec7079174bc0f1f34c27eff9ea5cb0721 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 13 Nov 2023 21:11:07 +0100 Subject: [PATCH 265/813] Revert "Add a setting max_execution_time_leaf to limit the execution time on shard for distributed query" --- .../operations/settings/query-complexity.md | 22 +------------------ src/Core/Settings.h | 16 ++++++-------- .../ClusterProxy/SelectStreamFactory.cpp | 13 +++++------ .../ClusterProxy/executeQuery.cpp | 8 ------- .../02786_max_execution_time_leaf.reference | 0 .../02786_max_execution_time_leaf.sql | 4 ---- 6 files changed, 13 insertions(+), 50 deletions(-) delete mode 100644 tests/queries/0_stateless/02786_max_execution_time_leaf.reference delete mode 100644 tests/queries/0_stateless/02786_max_execution_time_leaf.sql diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 9e36aa26946..2211b0c2de2 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -172,27 +172,7 @@ If you set `timeout_before_checking_execution_speed `to 0, ClickHouse will use c ## timeout_overflow_mode {#timeout-overflow-mode} -What to do if the query is run longer than `max_execution_time`: `throw` or `break`. By default, `throw`. - -# max_execution_time_leaf - -Similar semantic to `max_execution_time` but only apply on leaf node for distributed or remote queries. - -For example, if we want to limit execution time on leaf node to `10s` but no limit on the initial node, instead of having `max_execution_time` in the nested subquery settings: - -``` sql -SELECT count() FROM cluster(cluster, view(SELECT * FROM t SETTINGS max_execution_time = 10)); -``` - -We can use `max_execution_time_leaf` as the query settings: - -``` sql -SELECT count() FROM cluster(cluster, view(SELECT * FROM t)) SETTINGS max_execution_time_leaf = 10; -``` - -# timeout_overflow_mode_leaf - -What to do when the query in leaf node run longer than `max_execution_time_leaf`: `throw` or `break`. By default, `throw`. +What to do if the query is run longer than ‘max_execution_time’: ‘throw’ or ‘break’. By default, throw. ## min_execution_speed {#min-execution-speed} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0c9c1d21852..0a26e4ef5f3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -364,16 +364,16 @@ class IColumn; M(UInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \ M(OverflowMode, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ - M(UInt64, max_rows_to_read_leaf, 0, "Limit on read rows on the leaf nodes for distributed queries. Limit is applied for local reads only, excluding the final merge stage on the root node. Note, the setting is unstable with prefer_localhost_replica=1.", 0) \ - M(UInt64, max_bytes_to_read_leaf, 0, "Limit on read bytes (after decompression) on the leaf nodes for distributed queries. Limit is applied for local reads only, excluding the final merge stage on the root node. Note, the setting is unstable with prefer_localhost_replica=1.", 0) \ + M(UInt64, max_rows_to_read_leaf, 0, "Limit on read rows on the leaf nodes for distributed queries. Limit is applied for local reads only excluding the final merge stage on the root node. Note, the setting is unstable with prefer_localhost_replica=1.", 0) \ + M(UInt64, max_bytes_to_read_leaf, 0, "Limit on read bytes (after decompression) on the leaf nodes for distributed queries. Limit is applied for local reads only excluding the final merge stage on the root node. Note, the setting is unstable with prefer_localhost_replica=1.", 0) \ M(OverflowMode, read_overflow_mode_leaf, OverflowMode::THROW, "What to do when the leaf limit is exceeded.", 0) \ \ - M(UInt64, max_rows_to_group_by, 0, "If aggregation during GROUP BY is generating more than the specified number of rows (unique GROUP BY keys), the behavior will be determined by the 'group_by_overflow_mode' which by default is - throw an exception, but can be also switched to an approximate GROUP BY mode.", 0) \ + M(UInt64, max_rows_to_group_by, 0, "If aggregation during GROUP BY is generating more than specified number of rows (unique GROUP BY keys), the behavior will be determined by the 'group_by_overflow_mode' which by default is - throw an exception, but can be also switched to an approximate GROUP BY mode.", 0) \ M(OverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(UInt64, max_bytes_before_external_group_by, 0, "If memory usage during GROUP BY operation is exceeding this threshold in bytes, activate the 'external aggregation' mode (spill data to disk). Recommended value is half of available system memory.", 0) \ \ - M(UInt64, max_rows_to_sort, 0, "If more than the specified amount of records have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ - M(UInt64, max_bytes_to_sort, 0, "If more than the specified amount of (uncompressed) bytes have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ + M(UInt64, max_rows_to_sort, 0, "If more than specified amount of records have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ + M(UInt64, max_bytes_to_sort, 0, "If more than specified amount of (uncompressed) bytes have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ M(OverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(UInt64, max_bytes_before_external_sort, 0, "If memory usage during ORDER BY operation is exceeding this threshold in bytes, activate the 'external sorting' mode (spill data to disk). Recommended value is half of available system memory.", 0) \ M(UInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.", 0) \ @@ -384,10 +384,8 @@ class IColumn; M(OverflowMode, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ /* TODO: Check also when merging and finalizing aggregate functions. */ \ - M(Seconds, max_execution_time, 0, "If query runtime exceeds the specified number of seconds, the behavior will be determined by the 'timeout_overflow_mode', which by default is - throw an exception. Note that the timeout is checked and query can stop only in designated places during data processing. It currently cannot stop during merging of aggregation states or during query analysis, and the actual run time will be higher than the value of this setting.", 0) \ + M(Seconds, max_execution_time, 0, "If query run time exceeded the specified number of seconds, the behavior will be determined by the 'timeout_overflow_mode' which by default is - throw an exception. Note that the timeout is checked and query can stop only in designated places during data processing. It currently cannot stop during merging of aggregation states or during query analysis, and the actual run time will be higher than the value of this setting.", 0) \ M(OverflowMode, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ - M(Seconds, max_execution_time_leaf, 0, "Similar semantic to max_execution_time but only apply on leaf node for distributed queries, the time out behavior will be determined by 'timeout_overflow_mode_leaf' which by default is - throw an exception", 0) \ - M(OverflowMode, timeout_overflow_mode_leaf, OverflowMode::THROW, "What to do when the leaf limit is exceeded.", 0) \ \ M(UInt64, min_execution_speed, 0, "Minimum number of execution rows per second.", 0) \ M(UInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \ @@ -401,7 +399,7 @@ class IColumn; \ M(UInt64, max_sessions_for_user, 0, "Maximum number of simultaneous sessions for a user.", 0) \ \ - M(UInt64, max_subquery_depth, 100, "If a query has more than the specified number of nested subqueries, throw an exception. This allows you to have a sanity check to protect the users of your cluster from going insane with their queries.", 0) \ + M(UInt64, max_subquery_depth, 100, "If a query has more than specified number of nested subqueries, throw an exception. This allows you to have a sanity check to protect the users of your cluster from going insane with their queries.", 0) \ M(UInt64, max_analyze_depth, 5000, "Maximum number of analyses performed by interpreter.", 0) \ M(UInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \ M(UInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \ diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 4edc9d4d4e5..3935028f27c 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -1,21 +1,20 @@ +#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 @@ -23,7 +22,6 @@ #include #include - namespace ProfileEvents { extern const Event DistributedConnectionMissingTable; @@ -123,7 +121,6 @@ void SelectStreamFactory::createForShard( if (it != objects_by_shard.end()) replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, query_ast); - auto emplace_local_stream = [&]() { local_plans.emplace_back(createLocalPlan( diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 420bb447027..41235d107cd 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -141,14 +141,6 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, new_settings.allow_experimental_parallel_reading_from_replicas = false; } - if (settings.max_execution_time_leaf.value > 0) - { - /// Replace 'max_execution_time' of this sub-query with 'max_execution_time_leaf' and 'timeout_overflow_mode' - /// with 'timeout_overflow_mode_leaf' - new_settings.max_execution_time = settings.max_execution_time_leaf; - new_settings.timeout_overflow_mode = settings.timeout_overflow_mode_leaf; - } - auto new_context = Context::createCopy(context); new_context->setSettings(new_settings); return new_context; diff --git a/tests/queries/0_stateless/02786_max_execution_time_leaf.reference b/tests/queries/0_stateless/02786_max_execution_time_leaf.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02786_max_execution_time_leaf.sql b/tests/queries/0_stateless/02786_max_execution_time_leaf.sql deleted file mode 100644 index 1d02e82569c..00000000000 --- a/tests/queries/0_stateless/02786_max_execution_time_leaf.sql +++ /dev/null @@ -1,4 +0,0 @@ --- Tags: no-fasttest -SELECT count() FROM cluster('test_cluster_two_shards', view( SELECT * FROM numbers(100000000000) )) SETTINGS max_execution_time_leaf = 1; -- { serverError 159 } --- Can return partial result -SELECT count() FROM cluster('test_cluster_two_shards', view( SELECT * FROM numbers(100000000000) )) FORMAT Null SETTINGS max_execution_time_leaf = 1, timeout_overflow_mode_leaf = 'break'; From de42b7f9e0020e71e94a4b09062b110e9d014690 Mon Sep 17 00:00:00 2001 From: justindeguzman Date: Mon, 13 Nov 2023 12:57:05 -0800 Subject: [PATCH 266/813] [Docs] Add details to S3 and GCS table functions --- docs/en/sql-reference/table-functions/gcs.md | 2 ++ docs/en/sql-reference/table-functions/s3.md | 2 ++ docs/en/sql-reference/table-functions/s3Cluster.md | 2 +- 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/gcs.md b/docs/en/sql-reference/table-functions/gcs.md index c49ae6a8501..1cd1bfd3aae 100644 --- a/docs/en/sql-reference/table-functions/gcs.md +++ b/docs/en/sql-reference/table-functions/gcs.md @@ -9,6 +9,8 @@ keywords: [gcs, bucket] Provides a table-like interface to `SELECT` and `INSERT` data from [Google Cloud Storage](https://cloud.google.com/storage/). Requires the [`Storage Object User` IAM role](https://cloud.google.com/storage/docs/access-control/iam-roles). +If you have multiple replicas in your cluster, you can use the [s3Cluster function](../../sql-reference/table-functions/s3Cluster.md) (which works with GCS) instead to parallelize inserts. + **Syntax** ``` sql diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 8649295e815..9263e10540f 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -9,6 +9,8 @@ keywords: [s3, gcs, bucket] Provides a table-like interface to select/insert files in [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/). This table function is similar to the [hdfs function](../../sql-reference/table-functions/hdfs.md), but provides S3-specific features. +If you have multiple replicas in your cluster, you can use the [s3Cluster function](../../sql-reference/table-functions/s3Cluster.md) instead to parallelize inserts. + **Syntax** ``` sql diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index 675aef54d34..799eb31446a 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -5,7 +5,7 @@ sidebar_label: s3Cluster title: "s3Cluster Table Function" --- -Allows processing files from [Amazon S3](https://aws.amazon.com/s3/) in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster, discloses asterisks in S3 file path, and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. +Allows processing files from [Amazon S3](https://aws.amazon.com/s3/) and Google Cloud Storage [Google Cloud Storage](https://cloud.google.com/storage/) in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster, discloses asterisks in S3 file path, and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. **Syntax** From bdb624f20f208e571a8b7b85882eca3c8b5df480 Mon Sep 17 00:00:00 2001 From: justindeguzman Date: Mon, 13 Nov 2023 13:03:25 -0800 Subject: [PATCH 267/813] [Docs] Add details on S3 table function memory usage/streaming --- docs/en/sql-reference/table-functions/gcs.md | 2 ++ docs/en/sql-reference/table-functions/s3.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/sql-reference/table-functions/gcs.md b/docs/en/sql-reference/table-functions/gcs.md index 1cd1bfd3aae..5ffc20189da 100644 --- a/docs/en/sql-reference/table-functions/gcs.md +++ b/docs/en/sql-reference/table-functions/gcs.md @@ -9,6 +9,8 @@ keywords: [gcs, bucket] Provides a table-like interface to `SELECT` and `INSERT` data from [Google Cloud Storage](https://cloud.google.com/storage/). Requires the [`Storage Object User` IAM role](https://cloud.google.com/storage/docs/access-control/iam-roles). +This is an alias of the [s3 table function](../../sql-reference/table-functions/s3.md). + If you have multiple replicas in your cluster, you can use the [s3Cluster function](../../sql-reference/table-functions/s3Cluster.md) (which works with GCS) instead to parallelize inserts. **Syntax** diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 9263e10540f..c80488df05e 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -11,6 +11,8 @@ Provides a table-like interface to select/insert files in [Amazon S3](https://aw If you have multiple replicas in your cluster, you can use the [s3Cluster function](../../sql-reference/table-functions/s3Cluster.md) instead to parallelize inserts. +When using the `s3 table function` with [`INSERT INTO...SELECT`](../../sql-reference/statements/insert-into#inserting-the-results-of-select), data is read and inserted in a streaming fashion. Only a few blocks of data reside in memory while the blocks are continuously read from S3 and pushed into the destination table. + **Syntax** ``` sql From 207c4c50cb3ec790aa168113adec9212674f922b Mon Sep 17 00:00:00 2001 From: Ethan Shea <142333519+ethshea@users.noreply.github.com> Date: Mon, 13 Nov 2023 16:12:05 -0500 Subject: [PATCH 268/813] Improve legibility of window functions list Before, markdown was rendering these as one big paragraph. --- docs/en/sql-reference/window-functions/index.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index a8f494a5afc..7a18335fbdb 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -86,14 +86,14 @@ WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column] These functions can be used only as a window function. -`row_number()` - Number the current row within its partition starting from 1. -`first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. -`last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. -`nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. -`rank()` - Rank the current row within its partition with gaps. -`dense_rank()` - Rank the current row within its partition without gaps. -`lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. -`leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. +- `row_number()` - Number the current row within its partition starting from 1. +- `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. +- `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. +- `nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. +- `rank()` - Rank the current row within its partition with gaps. +- `dense_rank()` - Rank the current row within its partition without gaps. +- `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. +- `leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. ```text PARTITION From 37dc535d780ed2f515b3a904a14b44555d232435 Mon Sep 17 00:00:00 2001 From: Ethan Shea <142333519+ethshea@users.noreply.github.com> Date: Mon, 13 Nov 2023 16:16:07 -0500 Subject: [PATCH 269/813] Remove tabs --- docs/en/sql-reference/window-functions/index.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 7a18335fbdb..6340c369bff 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -86,12 +86,12 @@ WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column] These functions can be used only as a window function. -- `row_number()` - Number the current row within its partition starting from 1. -- `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. +- `row_number()` - Number the current row within its partition starting from 1. +- `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. - `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. - `nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. -- `rank()` - Rank the current row within its partition with gaps. -- `dense_rank()` - Rank the current row within its partition without gaps. +- `rank()` - Rank the current row within its partition with gaps. +- `dense_rank()` - Rank the current row within its partition without gaps. - `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. - `leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. From a09437c81630e522345cc805be010d200166e61e Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 13 Nov 2023 22:23:38 +0000 Subject: [PATCH 270/813] Fix possible 'uncaught exception of type std::__1::system_error: mutex lock failed: Invalid argument' in clickhouse-local --- src/Client/LocalConnection.cpp | 2 ++ .../02916_local_insert_into_function.reference | 0 .../0_stateless/02916_local_insert_into_function.sh | 8 ++++++++ 3 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02916_local_insert_into_function.reference create mode 100755 tests/queries/0_stateless/02916_local_insert_into_function.sh diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index d01c40a9c34..849308155b0 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -251,10 +251,12 @@ void LocalConnection::finishQuery() else if (state->pushing_async_executor) { state->pushing_async_executor->finish(); + state->pushing_async_executor.reset(); } else if (state->pushing_executor) { state->pushing_executor->finish(); + state->pushing_executor.reset(); } state->io.onFinish(); diff --git a/tests/queries/0_stateless/02916_local_insert_into_function.reference b/tests/queries/0_stateless/02916_local_insert_into_function.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02916_local_insert_into_function.sh b/tests/queries/0_stateless/02916_local_insert_into_function.sh new file mode 100755 index 00000000000..2eca2c2ce0a --- /dev/null +++ b/tests/queries/0_stateless/02916_local_insert_into_function.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "insert into function file('/dev/null', CSV, 'c1 UInt32') values (42)" + From 554d9071898f13082576145e6c7409c1964ef2e6 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Mon, 13 Nov 2023 23:42:51 +0000 Subject: [PATCH 271/813] Fix the keeper_server availability zone configuration. Signed-off-by: Jianfei Hu --- src/Coordination/KeeperContext.cpp | 29 ++++++++++++++----- src/Coordination/KeeperContext.h | 2 +- src/Coordination/KeeperDispatcher.cpp | 12 +------- src/IO/S3/Credentials.cpp | 5 ++++ tests/integration/helpers/keeper_config1.xml | 4 ++- tests/integration/helpers/keeper_config2.xml | 4 ++- .../test_keeper_availability_zone/test.py | 5 ++++ 7 files changed, 40 insertions(+), 21 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 1cee2a8e446..9745a53d1ab 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -32,16 +33,30 @@ KeeperContext::KeeperContext(bool standalone_keeper_) system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, const std::string & environment_az) +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_) { dispatcher = dispatcher_; - /// We only use the environment availability zone when configuration option is missing. - auto keeper_az = config.getString("keeper_server.availability_zone", environment_az); - if (!keeper_az.empty()) - system_nodes_with_data[keeper_availability_zone_path] = keeper_az; - LOG_INFO(&Poco::Logger::get("KeeperContext"), - "Initialize the KeeperContext with availability zone: '{}', environment availability zone '{}'. ", keeper_az, environment_az); + if (config.hasProperty("keeper_server.availability_zone")) + { + auto keeper_az = config.getString("keeper_server.availability_zone.value"); + if (config.getBool("keeper_server.availability_zone.enable_auto_detection_on_cloud", false)) + { + try + { + keeper_az = DB::S3::getRunningAvailabilityZone(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + if (!keeper_az.empty()) + { + system_nodes_with_data[keeper_availability_zone_path] = keeper_az; + LOG_INFO(&Poco::Logger::get("KeeperContext"), "Initialize the KeeperContext with availability zone: '{}'.'. ", keeper_az); + } + } digest_enabled = config.getBool("keeper_server.digest_enabled", false); ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 71f00ce5a93..1af34b19ccf 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -23,7 +23,7 @@ public: SHUTDOWN }; - void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, const std::string & environment_az); + void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_); Phase getServerState() const; void setServerState(Phase server_state_); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 6d43445474d..ca454c18084 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include @@ -371,16 +370,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf configuration_and_settings = KeeperConfigurationAndSettings::loadFromConfig(config, standalone_keeper); keeper_context = std::make_shared(standalone_keeper); - String availability_zone; - try - { - availability_zone = DB::S3::getRunningAvailabilityZone(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - keeper_context->initialize(config, this, availability_zone); + keeper_context->initialize(config, this); requests_queue = std::make_unique(configuration_and_settings->coordination_settings->max_request_queue_size); request_thread = ThreadFromGlobalPool([this] { requestThread(); }); diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index eb87568c8ab..4ba14572589 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -65,6 +65,7 @@ bool areCredentialsEmptyOrExpired(const Aws::Auth::AWSCredentials & credentials, } const char SSO_CREDENTIALS_PROVIDER_LOG_TAG[] = "SSOCredentialsProvider"; +const int AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS = 5; } @@ -244,6 +245,7 @@ String AWSEC2MetadataClient::getAvailabilityZoneOrException() Poco::Net::HTTPResponse response; Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri.getPath()); session.sendRequest(request); + session.setTimeout(Poco::Timespan(AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS, 0)); std::istream & rs = session.receiveResponse(response); if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) @@ -257,9 +259,12 @@ String getGCPAvailabilityZoneOrException() { Poco::URI uri(String(GCP_METADATA_SERVICE_ENDPOINT) + "/computeMetadata/v1/instance/zone"); Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort()); + session.setTimeout(Poco::Timespan(AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS, 0)); + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri.getPath()); Poco::Net::HTTPResponse response; request.set("Metadata-Flavor", "Google"); + session.sendRequest(request); std::istream & rs = session.receiveResponse(response); if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) diff --git a/tests/integration/helpers/keeper_config1.xml b/tests/integration/helpers/keeper_config1.xml index 29232989084..12c6c0b78b6 100644 --- a/tests/integration/helpers/keeper_config1.xml +++ b/tests/integration/helpers/keeper_config1.xml @@ -11,7 +11,9 @@ 2181 - az-zoo1 + + az-zoo1 + 1 diff --git a/tests/integration/helpers/keeper_config2.xml b/tests/integration/helpers/keeper_config2.xml index 2601efbb313..0c58aaceb1c 100644 --- a/tests/integration/helpers/keeper_config2.xml +++ b/tests/integration/helpers/keeper_config2.xml @@ -12,7 +12,9 @@ 2181 2 - az-zoo2 + + az-zoo2 + 10000 diff --git a/tests/integration/test_keeper_availability_zone/test.py b/tests/integration/test_keeper_availability_zone/test.py index 1836f0e679b..b78e776f3c6 100644 --- a/tests/integration/test_keeper_availability_zone/test.py +++ b/tests/integration/test_keeper_availability_zone/test.py @@ -29,3 +29,8 @@ def test_get_availability_zone(): with KeeperClient.from_cluster(cluster, "zoo2") as client2: assert client2.get("/keeper/availability_zone") == "az-zoo2" + + # keeper3 is not configured with availability_zone value. + with KeeperClient.from_cluster(cluster, "zoo3") as client3: + with pytest.raises(Exception): + client3.get("/keeper/availability_zone") \ No newline at end of file From 8c9f12fe17ecdf3b4449a8a2bd699cf0cea4852d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Nov 2023 08:05:33 +0300 Subject: [PATCH 272/813] Update wide_integer_impl.h --- base/base/wide_integer_impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 8d1da7de642..c1fd7b69b7f 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -104,7 +104,7 @@ public: static constexpr wide::integer min() noexcept { - if (is_same_v) + if constexpr (is_same_v) { using T = wide::integer; T res{}; From 32dd1b26b305a323144f77f3d6081746f4c28578 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Nov 2023 06:26:05 +0100 Subject: [PATCH 273/813] Make autogenerated file as light as possible --- src/Daemon/BaseDaemon.cpp | 2 +- src/Daemon/GitHash.cpp.in | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 8833156386f..7d19b935c2d 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -967,7 +967,7 @@ static void blockSignals(const std::vector & signals) throw Poco::Exception("Cannot block signal."); } -extern String getGitHash(); +extern const char * getGitHash(); void BaseDaemon::initializeTerminationAndSignalProcessing() { diff --git a/src/Daemon/GitHash.cpp.in b/src/Daemon/GitHash.cpp.in index 7e33682d670..65937af8848 100644 --- a/src/Daemon/GitHash.cpp.in +++ b/src/Daemon/GitHash.cpp.in @@ -1,8 +1,6 @@ /// This file was autogenerated by CMake -#include - -String getGitHash() +const char * getGitHash() { return "@GIT_HASH@"; } From cb74f52c5cec99d674068b7304d150c2a5bd1e23 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Nov 2023 06:27:39 +0100 Subject: [PATCH 274/813] Make autogenerated file as light as possible --- src/Daemon/BaseDaemon.cpp | 4 ++-- src/Daemon/GitHash.cpp.in | 5 +---- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 7d19b935c2d..d66bdf3583f 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -967,7 +967,7 @@ static void blockSignals(const std::vector & signals) throw Poco::Exception("Cannot block signal."); } -extern const char * getGitHash(); +extern const char * GIT_HASH; void BaseDaemon::initializeTerminationAndSignalProcessing() { @@ -1007,7 +1007,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() build_id = ""; #endif - git_hash = getGitHash(); + git_hash = GIT_HASH; #if defined(OS_LINUX) std::string executable_path = getExecutablePath(); diff --git a/src/Daemon/GitHash.cpp.in b/src/Daemon/GitHash.cpp.in index 65937af8848..c5990531449 100644 --- a/src/Daemon/GitHash.cpp.in +++ b/src/Daemon/GitHash.cpp.in @@ -1,6 +1,3 @@ /// This file was autogenerated by CMake -const char * getGitHash() -{ - return "@GIT_HASH@"; -} +const char * GIT_HASH = "@GIT_HASH@"; From 80bbe7dcc6a3d6f381325ae93a1a59f41904fe91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Nov 2023 06:55:06 +0100 Subject: [PATCH 275/813] Less CMake checks --- base/glibc-compatibility/CMakeLists.txt | 8 -- cmake/cpu_features.cmake | 180 ++++-------------------- contrib/aws-cmake/AwsFeatureTests.cmake | 115 +-------------- contrib/aws-cmake/AwsSIMD.cmake | 51 +------ contrib/libunwind-cmake/CMakeLists.txt | 23 +-- 5 files changed, 38 insertions(+), 339 deletions(-) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 0539f0c231d..65677ed2cf3 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -5,9 +5,6 @@ if (GLIBC_COMPATIBILITY) endif() enable_language(ASM) - include(CheckIncludeFile) - - check_include_file("sys/random.h" HAVE_SYS_RANDOM_H) add_headers_and_sources(glibc_compatibility .) add_headers_and_sources(glibc_compatibility musl) @@ -21,11 +18,6 @@ if (GLIBC_COMPATIBILITY) message (FATAL_ERROR "glibc_compatibility can only be used on x86_64 or aarch64.") endif () - list(REMOVE_ITEM glibc_compatibility_sources musl/getentropy.c) - if(HAVE_SYS_RANDOM_H) - list(APPEND glibc_compatibility_sources musl/getentropy.c) - endif() - # Need to omit frame pointers to match the performance of glibc set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fomit-frame-pointer") diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 9fc3960c166..765e36403ad 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -1,10 +1,5 @@ # https://software.intel.com/sites/landingpage/IntrinsicsGuide/ -include (CheckCXXSourceCompiles) -include (CMakePushCheckState) - -cmake_push_check_state () - # The variables HAVE_* determine if compiler has support for the flag to use the corresponding instruction set. # The options ENABLE_* determine if we will tell compiler to actually use the corresponding instruction set if compiler can do it. @@ -137,178 +132,53 @@ elseif (ARCH_AMD64) endif() # ClickHouse can be cross-compiled (e.g. on an ARM host for x86) but it is also possible to build ClickHouse on x86 w/o AVX for x86 w/ - # AVX. We only check that the compiler can emit certain SIMD instructions, we don't care if the host system is able to run the binary. - # Therefore, use check_cxx_source_compiles (= does the code compile+link?) instead of check_cxx_source_runs (= does the code - # compile+link+run). + # AVX. We only assume that the compiler can emit certain SIMD instructions, we don't care if the host system is able to run the binary. + + SET (HAVE_SSSE3 1) + SET (HAVE_SSE41 1) + SET (HAVE_SSE42 1) + SET (HAVE_PCLMULQDQ 1) + SET (HAVE_POPCNT 1) + SET (HAVE_AVX 1) + SET (HAVE_AVX2 1) + SET (HAVE_AVX512 1) + SET (HAVE_AVX512_VBMI 1) + SET (HAVE_BMI 1) + SET (HAVE_BMI2 1) - set (TEST_FLAG "-mssse3") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - __m64 a = _mm_abs_pi8(__m64()); - (void)a; - return 0; - } - " HAVE_SSSE3) if (HAVE_SSSE3 AND ENABLE_SSSE3) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mssse3") endif () - - set (TEST_FLAG "-msse4.1") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _mm_insert_epi8(__m128i(), 0, 0); - (void)a; - return 0; - } - " HAVE_SSE41) if (HAVE_SSE41 AND ENABLE_SSE41) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -msse4.1") endif () - - set (TEST_FLAG "-msse4.2") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _mm_crc32_u64(0, 0); - (void)a; - return 0; - } - " HAVE_SSE42) if (HAVE_SSE42 AND ENABLE_SSE42) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -msse4.2") endif () - - set (TEST_FLAG "-mpclmul") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _mm_clmulepi64_si128(__m128i(), __m128i(), 0); - (void)a; - return 0; - } - " HAVE_PCLMULQDQ) if (HAVE_PCLMULQDQ AND ENABLE_PCLMULQDQ) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mpclmul") endif () - - set (TEST_FLAG "-mpopcnt") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - int main() { - auto a = __builtin_popcountll(0); - (void)a; - return 0; - } - " HAVE_POPCNT) if (HAVE_POPCNT AND ENABLE_POPCNT) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mpopcnt") endif () - - set (TEST_FLAG "-mavx") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _mm256_insert_epi8(__m256i(), 0, 0); - (void)a; - return 0; - } - " HAVE_AVX) if (HAVE_AVX AND ENABLE_AVX) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx") endif () - - set (TEST_FLAG "-mavx2") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _mm256_add_epi16(__m256i(), __m256i()); - (void)a; - return 0; - } - " HAVE_AVX2) if (HAVE_AVX2 AND ENABLE_AVX2) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx2") endif () - - set (TEST_FLAG "-mavx512f -mavx512bw -mavx512vl") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _mm512_setzero_epi32(); - (void)a; - auto b = _mm512_add_epi16(__m512i(), __m512i()); - (void)b; - auto c = _mm_cmp_epi8_mask(__m128i(), __m128i(), 0); - (void)c; - return 0; - } - " HAVE_AVX512) if (HAVE_AVX512 AND ENABLE_AVX512) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx512f -mavx512bw -mavx512vl") endif () - - set (TEST_FLAG "-mavx512vbmi") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _mm512_permutexvar_epi8(__m512i(), __m512i()); - (void)a; - return 0; - } - " HAVE_AVX512_VBMI) if (HAVE_AVX512 AND ENABLE_AVX512 AND HAVE_AVX512_VBMI AND ENABLE_AVX512_VBMI) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx512vbmi") endif () - - set (TEST_FLAG "-mbmi") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _blsr_u32(0); - (void)a; - return 0; - } - " HAVE_BMI) if (HAVE_BMI AND ENABLE_BMI) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mbmi") endif () - - set (TEST_FLAG "-mbmi2") - set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") - check_cxx_source_compiles(" - #include - int main() { - auto a = _pdep_u64(0, 0); - (void)a; - return 0; - } - " HAVE_BMI2) if (HAVE_BMI2 AND HAVE_AVX2 AND ENABLE_AVX2 AND ENABLE_BMI2) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mbmi2") endif () - - # Limit avx2/avx512 flag for specific source build - set (X86_INTRINSICS_FLAGS "") - if (ENABLE_AVX2_FOR_SPEC_OP) - if (HAVE_BMI) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") - endif () - if (HAVE_AVX AND HAVE_AVX2) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx -mavx2") - endif () - endif () - if (ENABLE_AVX512_FOR_SPEC_OP) set (X86_INTRINSICS_FLAGS "") if (HAVE_BMI) @@ -321,5 +191,3 @@ elseif (ARCH_AMD64) else () # RISC-V + exotic platforms endif () - -cmake_pop_check_state () diff --git a/contrib/aws-cmake/AwsFeatureTests.cmake b/contrib/aws-cmake/AwsFeatureTests.cmake index 54727e08d6b..e58b6634f42 100644 --- a/contrib/aws-cmake/AwsFeatureTests.cmake +++ b/contrib/aws-cmake/AwsFeatureTests.cmake @@ -1,114 +1,13 @@ # Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. # SPDX-License-Identifier: Apache-2.0. -include(CheckCSourceRuns) - option(USE_CPU_EXTENSIONS "Whenever possible, use functions optimized for CPUs with specific extensions (ex: SSE, AVX)." ON) -# In the current (11/2/21) state of mingw64, the packaged gcc is not capable of emitting properly aligned avx2 instructions under certain circumstances. -# This leads to crashes for windows builds using mingw64 when invoking the avx2-enabled versions of certain functions. Until we can find a better -# work-around, disable avx2 (and all other extensions) in mingw builds. -# -# https://gcc.gnu.org/bugzilla/show_bug.cgi?id=54412 -# -if (MINGW) - message(STATUS "MINGW detected! Disabling avx2 and other CPU extensions") - set(USE_CPU_EXTENSIONS OFF) -endif() +if (ARCH_AMD64) + set (AWS_ARCH_INTEL 1) +elseif (ARCH_AARCH64) + set (AWS_ARCH_ARM64 1) +endif () -if(NOT CMAKE_CROSSCOMPILING) - check_c_source_runs(" - #include - bool foo(int a, int b, int *c) { - return __builtin_mul_overflow(a, b, c); - } - - int main() { - int out; - if (foo(1, 2, &out)) { - return 0; - } - - return 0; - }" AWS_HAVE_GCC_OVERFLOW_MATH_EXTENSIONS) - - if (USE_CPU_EXTENSIONS) - check_c_source_runs(" - int main() { - int foo = 42; - _mulx_u32(1, 2, &foo); - return foo != 2; - }" AWS_HAVE_MSVC_MULX) - endif() - -endif() - -check_c_source_compiles(" - #include - #if WINAPI_FAMILY_PARTITION(WINAPI_PARTITION_DESKTOP) - int main() { - return 0; - } - #else - it's not windows desktop - #endif -" AWS_HAVE_WINAPI_DESKTOP) - -check_c_source_compiles(" - int main() { -#if !(defined(__x86_64__) || defined(__i386__) || defined(_M_X64) || defined(_M_IX86)) -# error \"not intel\" -#endif - return 0; - } -" AWS_ARCH_INTEL) - -check_c_source_compiles(" - int main() { -#if !(defined(__aarch64__) || defined(_M_ARM64)) -# error \"not arm64\" -#endif - return 0; - } -" AWS_ARCH_ARM64) - -check_c_source_compiles(" - int main() { -#if !(defined(__arm__) || defined(_M_ARM)) -# error \"not arm\" -#endif - return 0; - } -" AWS_ARCH_ARM32) - -check_c_source_compiles(" -int main() { - int foo = 42, bar = 24; - __asm__ __volatile__(\"\":\"=r\"(foo):\"r\"(bar):\"memory\"); -}" AWS_HAVE_GCC_INLINE_ASM) - -check_c_source_compiles(" -#include -int main() { -#ifdef __linux__ - getauxval(AT_HWCAP); - getauxval(AT_HWCAP2); -#endif - return 0; -}" AWS_HAVE_AUXV) - -string(REGEX MATCH "^(aarch64|arm)" ARM_CPU "${CMAKE_SYSTEM_PROCESSOR}") -if(NOT LEGACY_COMPILER_SUPPORT OR ARM_CPU) - check_c_source_compiles(" - #include - int main() { - backtrace(NULL, 0); - return 0; - }" AWS_HAVE_EXECINFO) -endif() - -check_c_source_compiles(" -#include -int main() { - return 1; -}" AWS_HAVE_LINUX_IF_LINK_H) +set (AWS_HAVE_GCC_INLINE_ASM 1) +set (AWS_HAVE_AUXV 1) diff --git a/contrib/aws-cmake/AwsSIMD.cmake b/contrib/aws-cmake/AwsSIMD.cmake index bd6f4064e78..a2f50f27d4e 100644 --- a/contrib/aws-cmake/AwsSIMD.cmake +++ b/contrib/aws-cmake/AwsSIMD.cmake @@ -1,54 +1,13 @@ # Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. # SPDX-License-Identifier: Apache-2.0. -include(CheckCCompilerFlag) -include(CheckIncludeFile) - if (USE_CPU_EXTENSIONS) - if (MSVC) - check_c_compiler_flag("/arch:AVX2" HAVE_M_AVX2_FLAG) - if (HAVE_M_AVX2_FLAG) - set(AVX2_CFLAGS "/arch:AVX2") - endif() - else() - check_c_compiler_flag(-mavx2 HAVE_M_AVX2_FLAG) - if (HAVE_M_AVX2_FLAG) - set(AVX2_CFLAGS "-mavx -mavx2") - endif() + if (HAVE_AVX2) + set (AVX2_CFLAGS "-mavx -mavx2") + set (HAVE_AVX2_INTRINSICS 1) + set (HAVE_MM256_EXTRACT_EPI64 1) endif() - - - cmake_push_check_state() - set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} ${AVX2_CFLAGS}") - - check_c_source_compiles(" - #include - #include - #include - - int main() { - __m256i vec; - memset(&vec, 0, sizeof(vec)); - - _mm256_shuffle_epi8(vec, vec); - _mm256_set_epi32(1,2,3,4,5,6,7,8); - _mm256_permutevar8x32_epi32(vec, vec); - - return 0; - }" HAVE_AVX2_INTRINSICS) - - check_c_source_compiles(" - #include - #include - - int main() { - __m256i vec; - memset(&vec, 0, sizeof(vec)); - return (int)_mm256_extract_epi64(vec, 2); - }" HAVE_MM256_EXTRACT_EPI64) - - cmake_pop_check_state() -endif() # USE_CPU_EXTENSIONS +endif() macro(simd_add_definition_if target definition) if(${definition}) diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 733f99d07f5..0d872bae5d1 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -1,6 +1,3 @@ -include(CheckCCompilerFlag) -include(CheckCXXCompilerFlag) - set(LIBUNWIND_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libunwind") set(LIBUNWIND_CXX_SOURCES @@ -48,27 +45,11 @@ target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_NO_HEAP=1 -D_DEBUG -D_LIB # and disable sanitizers (otherwise infinite loop may happen) target_compile_options(unwind PRIVATE -O3 -fno-exceptions -funwind-tables -fno-sanitize=all $<$:-nostdinc++ -fno-rtti>) -check_c_compiler_flag(-Wunused-but-set-variable HAVE_WARNING_UNUSED_BUT_SET_VARIABLE) -if (HAVE_WARNING_UNUSED_BUT_SET_VARIABLE) - target_compile_options(unwind PRIVATE -Wno-unused-but-set-variable) -endif () - -check_cxx_compiler_flag(-Wmissing-attributes HAVE_WARNING_MISSING_ATTRIBUTES) -if (HAVE_WARNING_MISSING_ATTRIBUTES) - target_compile_options(unwind PRIVATE -Wno-missing-attributes) -endif () - -check_cxx_compiler_flag(-Wmaybe-uninitialized HAVE_WARNING_MAYBE_UNINITIALIZED) -if (HAVE_WARNING_MAYBE_UNINITIALIZED) - target_compile_options(unwind PRIVATE -Wno-maybe-uninitialized) -endif () +target_compile_options(unwind PRIVATE -Wno-unused-but-set-variable) # The library is using register variables that are bound to specific registers # Example: DwarfInstructions.hpp: register unsigned long long x16 __asm("x16") = cfa; -check_cxx_compiler_flag(-Wregister HAVE_WARNING_REGISTER) -if (HAVE_WARNING_REGISTER) - target_compile_options(unwind PRIVATE "$<$:-Wno-register>") -endif () +target_compile_options(unwind PRIVATE "$<$:-Wno-register>") install( TARGETS unwind From 4e3e8e32fde948d69ad78fb7f7cf9fe293f55b91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Nov 2023 07:24:41 +0100 Subject: [PATCH 276/813] Remove more garbage --- contrib/aws-cmake/AwsThreadAffinity.cmake | 45 +------------ contrib/aws-cmake/AwsThreadName.cmake | 52 +-------------- contrib/libcxx-cmake/CMakeLists.txt | 2 - contrib/rocksdb-cmake/CMakeLists.txt | 80 ++--------------------- contrib/xz-cmake/CMakeLists.txt | 2 - utils/check-style/check-style | 3 + 6 files changed, 13 insertions(+), 171 deletions(-) diff --git a/contrib/aws-cmake/AwsThreadAffinity.cmake b/contrib/aws-cmake/AwsThreadAffinity.cmake index 9e53481272c..7f30fb71b43 100644 --- a/contrib/aws-cmake/AwsThreadAffinity.cmake +++ b/contrib/aws-cmake/AwsThreadAffinity.cmake @@ -1,50 +1,9 @@ # Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. # SPDX-License-Identifier: Apache-2.0. -include(CheckSymbolExists) - # Check if the platform supports setting thread affinity # (important for hitting full NIC entitlement on NUMA architectures) function(aws_set_thread_affinity_method target) - - # Non-POSIX, Android, and Apple platforms do not support thread affinity. - if (NOT UNIX OR ANDROID OR APPLE) - target_compile_definitions(${target} PRIVATE - -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_NONE) - return() - endif() - - cmake_push_check_state() - list(APPEND CMAKE_REQUIRED_DEFINITIONS -D_GNU_SOURCE) - list(APPEND CMAKE_REQUIRED_LIBRARIES pthread) - - set(headers "pthread.h") - # BSDs put nonportable pthread declarations in a separate header. - if(CMAKE_SYSTEM_NAME MATCHES BSD) - set(headers "${headers};pthread_np.h") - endif() - - # Using pthread attrs is the preferred method, but is glibc-specific. - check_symbol_exists(pthread_attr_setaffinity_np "${headers}" USE_PTHREAD_ATTR_SETAFFINITY) - if (USE_PTHREAD_ATTR_SETAFFINITY) - target_compile_definitions(${target} PRIVATE - -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_PTHREAD_ATTR) - return() - endif() - - # This method is still nonportable, but is supported by musl and BSDs. - check_symbol_exists(pthread_setaffinity_np "${headers}" USE_PTHREAD_SETAFFINITY) - if (USE_PTHREAD_SETAFFINITY) - target_compile_definitions(${target} PRIVATE - -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_PTHREAD) - return() - endif() - - # If we got here, we expected thread affinity support but didn't find it. - # We still build with degraded NUMA performance, but show a warning. - message(WARNING "No supported method for setting thread affinity") - target_compile_definitions(${target} PRIVATE - -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_NONE) - - cmake_pop_check_state() + # This code has been cut, because I don't care about it. + target_compile_definitions(${target} PRIVATE -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_NONE) endfunction() diff --git a/contrib/aws-cmake/AwsThreadName.cmake b/contrib/aws-cmake/AwsThreadName.cmake index a67416b4f83..e17759435ed 100644 --- a/contrib/aws-cmake/AwsThreadName.cmake +++ b/contrib/aws-cmake/AwsThreadName.cmake @@ -1,61 +1,13 @@ # Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. # SPDX-License-Identifier: Apache-2.0. -include(CheckSymbolExists) - # Check how the platform supports setting thread name function(aws_set_thread_name_method target) - - if (WINDOWS) - # On Windows we do a runtime check, instead of compile-time check - return() - elseif (APPLE) + if (APPLE) # All Apple platforms we support have the same function, so no need for compile-time check. return() endif() - cmake_push_check_state() - list(APPEND CMAKE_REQUIRED_DEFINITIONS -D_GNU_SOURCE) - list(APPEND CMAKE_REQUIRED_LIBRARIES pthread) - - # The start of the test program - set(c_source_start " - #define _GNU_SOURCE - #include - - #if defined(__FreeBSD__) || defined(__NETBSD__) - #include - #endif - - int main() { - pthread_t thread_id; - ") - - # The end of the test program - set(c_source_end "}") - # pthread_setname_np() usually takes 2 args - check_c_source_compiles(" - ${c_source_start} - pthread_setname_np(thread_id, \"asdf\"); - ${c_source_end}" - PTHREAD_SETNAME_TAKES_2ARGS) - if (PTHREAD_SETNAME_TAKES_2ARGS) - target_compile_definitions(${target} PRIVATE -DAWS_PTHREAD_SETNAME_TAKES_2ARGS) - return() - endif() - - # But on NetBSD it takes 3! - check_c_source_compiles(" - ${c_source_start} - pthread_setname_np(thread_id, \"asdf\", NULL); - ${c_source_end} - " PTHREAD_SETNAME_TAKES_3ARGS) - if (PTHREAD_SETNAME_TAKES_3ARGS) - target_compile_definitions(${target} PRIVATE -DAWS_PTHREAD_SETNAME_TAKES_3ARGS) - return() - endif() - - # And on many older/weirder platforms it's just not supported - cmake_pop_check_state() + target_compile_definitions(${target} PRIVATE -DAWS_PTHREAD_SETNAME_TAKES_2ARGS) endfunction() diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index b7e59e2c9a3..c77d5d8319e 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -1,5 +1,3 @@ -include(CheckCXXCompilerFlag) - set(LIBCXX_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm-project/libcxx") set(SRCS diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 466adf6aff0..2b6c48f0b38 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -76,7 +76,6 @@ else() endif() endif() -include(CheckCCompilerFlag) if(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") if(POWER9) set(HAS_POWER9 1) @@ -88,21 +87,12 @@ if(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") endif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") if(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64|arm64|ARM64") - CHECK_C_COMPILER_FLAG("-march=armv8-a+crc+crypto" HAS_ARMV8_CRC) - if(HAS_ARMV8_CRC) - message(STATUS " HAS_ARMV8_CRC yes") - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") - endif(HAS_ARMV8_CRC) + set(HAS_ARMV8_CRC 1) + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") endif(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64|arm64|ARM64") -include(CheckCXXSourceCompiles) -if(NOT MSVC) - set(CMAKE_REQUIRED_FLAGS "-msse4.2 -mpclmul") -endif() - -unset(CMAKE_REQUIRED_FLAGS) if(HAVE_SSE42) add_definitions(-DHAVE_SSE42) add_definitions(-DHAVE_PCLMUL) @@ -121,75 +111,18 @@ elseif(CMAKE_SYSTEM_NAME MATCHES "Linux") add_definitions(-DOS_LINUX) elseif(CMAKE_SYSTEM_NAME MATCHES "SunOS") add_definitions(-DOS_SOLARIS) -elseif(CMAKE_SYSTEM_NAME MATCHES "kFreeBSD") - add_definitions(-DOS_GNU_KFREEBSD) elseif(CMAKE_SYSTEM_NAME MATCHES "FreeBSD") add_definitions(-DOS_FREEBSD) -elseif(CMAKE_SYSTEM_NAME MATCHES "NetBSD") - add_definitions(-DOS_NETBSD) -elseif(CMAKE_SYSTEM_NAME MATCHES "OpenBSD") - add_definitions(-DOS_OPENBSD) -elseif(CMAKE_SYSTEM_NAME MATCHES "DragonFly") - add_definitions(-DOS_DRAGONFLYBSD) elseif(CMAKE_SYSTEM_NAME MATCHES "Android") add_definitions(-DOS_ANDROID) -elseif(CMAKE_SYSTEM_NAME MATCHES "Windows") - add_definitions(-DWIN32 -DOS_WIN -D_MBCS -DWIN64 -DNOMINMAX) - if(MINGW) - add_definitions(-D_WIN32_WINNT=_WIN32_WINNT_VISTA) - endif() endif() -if(NOT WIN32) - add_definitions(-DROCKSDB_PLATFORM_POSIX -DROCKSDB_LIB_IO_POSIX) -endif() +add_definitions(-DROCKSDB_PLATFORM_POSIX -DROCKSDB_LIB_IO_POSIX) -option(WITH_FALLOCATE "build with fallocate" ON) -if(WITH_FALLOCATE) - CHECK_C_SOURCE_COMPILES(" -#include -#include -int main() { - int fd = open(\"/dev/null\", 0); - fallocate(fd, FALLOC_FL_KEEP_SIZE, 0, 1024); -} -" HAVE_FALLOCATE) - if(HAVE_FALLOCATE) - add_definitions(-DROCKSDB_FALLOCATE_PRESENT) - endif() -endif() - -CHECK_C_SOURCE_COMPILES(" -#include -int main() { - int fd = open(\"/dev/null\", 0); - sync_file_range(fd, 0, 1024, SYNC_FILE_RANGE_WRITE); -} -" HAVE_SYNC_FILE_RANGE_WRITE) -if(HAVE_SYNC_FILE_RANGE_WRITE) - add_definitions(-DROCKSDB_RANGESYNC_PRESENT) -endif() - -CHECK_C_SOURCE_COMPILES(" -#include -int main() { - (void) PTHREAD_MUTEX_ADAPTIVE_NP; -} -" HAVE_PTHREAD_MUTEX_ADAPTIVE_NP) -if(HAVE_PTHREAD_MUTEX_ADAPTIVE_NP) +if (OS_LINUX OR OS_FREEBSD) add_definitions(-DROCKSDB_PTHREAD_ADAPTIVE_MUTEX) endif() -include(CheckCXXSymbolExists) -if (OS_FREEBSD) - check_cxx_symbol_exists(malloc_usable_size "${ROCKSDB_SOURCE_DIR}/malloc_np.h" HAVE_MALLOC_USABLE_SIZE) -else() - check_cxx_symbol_exists(malloc_usable_size "${ROCKSDB_SOURCE_DIR}/malloc.h" HAVE_MALLOC_USABLE_SIZE) -endif() -if(HAVE_MALLOC_USABLE_SIZE) - add_definitions(-DROCKSDB_MALLOC_USABLE_SIZE) -endif() - if (OS_LINUX) add_definitions(-DROCKSDB_SCHED_GETCPU_PRESENT) add_definitions(-DROCKSDB_AUXV_SYSAUXV_PRESENT) @@ -204,7 +137,6 @@ include_directories("${ROCKSDB_SOURCE_DIR}/include") if(WITH_FOLLY_DISTRIBUTED_MUTEX) include_directories("${ROCKSDB_SOURCE_DIR}/third-party/folly") endif() -find_package(Threads REQUIRED) # Main library source code @@ -497,7 +429,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc rocksdb_build_version.cc) -if(HAVE_SSE42 AND NOT MSVC) +if(HAVE_SSE42) set_source_files_properties( "${ROCKSDB_SOURCE_DIR}/util/crc32c.cc" PROPERTIES COMPILE_FLAGS "-msse4.2 -mpclmul") diff --git a/contrib/xz-cmake/CMakeLists.txt b/contrib/xz-cmake/CMakeLists.txt index c3a8203c83e..c73433d9863 100644 --- a/contrib/xz-cmake/CMakeLists.txt +++ b/contrib/xz-cmake/CMakeLists.txt @@ -98,8 +98,6 @@ if (ARCH_S390X) add_compile_definitions(WORDS_BIGENDIAN) endif () -find_package(Threads REQUIRED) - add_library(_liblzma ${SRC_DIR}/src/common/mythread.h diff --git a/utils/check-style/check-style b/utils/check-style/check-style index f87d2e292b5..bd6f111354e 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -422,3 +422,6 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep # Cyrillic characters hiding inside Latin. find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '[a-zA-Z][а-яА-ЯёЁ]|[а-яА-ЯёЁ][a-zA-Z]' && echo "^ Cyrillic characters found in unexpected place." + +# Don't allow dynamic compiler check with CMake, because we are using hermetic, reproducible, cross-compiled, static (TLDR, good) builds. +ls -1d $ROOT_PATH/contrib/*-cmake | xargs -I@ find @ -name 'CMakeLists.txt' -or -name '*.cmake' | xargs grep --with-filename -i -P 'check_c_compiler_flag|check_cxx_compiler_flag|check_c_source_compiles|check_cxx_source_compiles|check_include_file|check_symbol_exists|cmake_push_check_state|cmake_pop_check_state|find_package|CMAKE_REQUIRED_FLAGS|CheckIncludeFile|CheckCCompilerFlag|CheckCXXCompilerFlag|CheckCSourceCompiles|CheckCXXSourceCompiles|CheckCSymbolExists|CheckCXXSymbolExists' | grep -v Rust && echo "^ It's not allowed to have dynamic compiler checks with CMake." From 9bf62dd6220739e183208ee0152606bae7595efe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Nov 2023 07:43:11 +0100 Subject: [PATCH 277/813] Remove orphan header files --- src/Common/HashTable/FixedClearableHashMap.h | 75 ---------------- src/Common/SharedBlockRowRef.h | 89 ------------------- src/Common/SimpleActionBlocker.h | 79 ---------------- src/Common/SmallObjectPool.h | 52 ----------- src/Databases/DictionaryAttachInfo.h | 18 ---- .../ExternalLoaderDatabaseConfigRepository.h | 32 ------- .../Executors/IReadProgressCallback.h | 18 ---- src/Processors/Executors/traverse.h | 30 ------- src/Processors/QueueBuffer.h | 41 --------- utils/check-style/check-style | 4 + 10 files changed, 4 insertions(+), 434 deletions(-) delete mode 100644 src/Common/HashTable/FixedClearableHashMap.h delete mode 100644 src/Common/SharedBlockRowRef.h delete mode 100644 src/Common/SimpleActionBlocker.h delete mode 100644 src/Common/SmallObjectPool.h delete mode 100644 src/Databases/DictionaryAttachInfo.h delete mode 100644 src/Interpreters/ExternalLoaderDatabaseConfigRepository.h delete mode 100644 src/Processors/Executors/IReadProgressCallback.h delete mode 100644 src/Processors/Executors/traverse.h delete mode 100644 src/Processors/QueueBuffer.h diff --git a/src/Common/HashTable/FixedClearableHashMap.h b/src/Common/HashTable/FixedClearableHashMap.h deleted file mode 100644 index 6be7fde88b5..00000000000 --- a/src/Common/HashTable/FixedClearableHashMap.h +++ /dev/null @@ -1,75 +0,0 @@ -#pragma once - -#include -#include - - -template -struct FixedClearableHashMapCell -{ - using Mapped = TMapped; - using State = ClearableHashSetState; - - using value_type = PairNoInit; - using mapped_type = Mapped; - - UInt32 version; - Mapped mapped; - - FixedClearableHashMapCell() {} /// NOLINT - FixedClearableHashMapCell(const Key &, const State & state) : version(state.version) {} - FixedClearableHashMapCell(const value_type & value_, const State & state) : version(state.version), mapped(value_.second) {} - - const VoidKey getKey() const { return {}; } /// NOLINT - Mapped & getMapped() { return mapped; } - const Mapped & getMapped() const { return mapped; } - - bool isZero(const State & state) const { return version != state.version; } - void setZero() { version = 0; } - - struct CellExt - { - CellExt() {} /// NOLINT - CellExt(Key && key_, FixedClearableHashMapCell * ptr_) : key(key_), ptr(ptr_) {} - void update(Key && key_, FixedClearableHashMapCell * ptr_) - { - key = key_; - ptr = ptr_; - } - Key key; - FixedClearableHashMapCell * ptr; - const Key & getKey() const { return key; } - Mapped & getMapped() { return ptr->mapped; } - const Mapped & getMapped() const { return *ptr->mapped; } - const value_type getValue() const { return {key, *ptr->mapped}; } /// NOLINT - }; -}; - - -template -class FixedClearableHashMap : public FixedHashMap, Allocator> -{ -public: - using Base = FixedHashMap, Allocator>; - using Self = FixedClearableHashMap; - using LookupResult = typename Base::LookupResult; - - using Base::Base; - - Mapped & operator[](const Key & x) - { - LookupResult it; - bool inserted; - this->emplace(x, it, inserted); - if (inserted) - new (&it->getMapped()) Mapped(); - - return it->getMapped(); - } - - void clear() - { - ++this->version; - this->m_size = 0; - } -}; diff --git a/src/Common/SharedBlockRowRef.h b/src/Common/SharedBlockRowRef.h deleted file mode 100644 index 77dd0f1cc13..00000000000 --- a/src/Common/SharedBlockRowRef.h +++ /dev/null @@ -1,89 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -/// Allows you refer to the row in the block and hold the block ownership, -/// and thus avoid creating a temporary row object. -/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`; -/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece; -/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr; -/// The reference counter is not atomic, since it is used from one thread. -namespace detail -{ - struct SharedBlock : Block - { - int refcount = 0; - - ColumnRawPtrs all_columns; - ColumnRawPtrs sort_columns; - - explicit SharedBlock(Block && block) : Block(std::move(block)) {} - }; -} - -inline void intrusive_ptr_add_ref(detail::SharedBlock * ptr) -{ - ++ptr->refcount; -} - -inline void intrusive_ptr_release(detail::SharedBlock * ptr) -{ - if (0 == --ptr->refcount) - delete ptr; -} - -using SharedBlockPtr = boost::intrusive_ptr; - -struct SharedBlockRowRef -{ - ColumnRawPtrs * columns = nullptr; - size_t row_num = 0; - SharedBlockPtr shared_block; - - void swap(SharedBlockRowRef & other) - { - std::swap(columns, other.columns); - std::swap(row_num, other.row_num); - std::swap(shared_block, other.shared_block); - } - - /// The number and types of columns must match. - bool operator==(const SharedBlockRowRef & other) const - { - size_t size = columns->size(); - for (size_t i = 0; i < size; ++i) - if (0 != (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1)) - return false; - return true; - } - - bool operator!=(const SharedBlockRowRef & other) const - { - return !(*this == other); - } - - void reset() - { - SharedBlockRowRef empty; - swap(empty); - } - - bool empty() const { return columns == nullptr; } - size_t size() const { return empty() ? 0 : columns->size(); } - - void set(SharedBlockPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_) - { - shared_block = shared_block_; - columns = columns_; - row_num = row_num_; - } -}; - -} diff --git a/src/Common/SimpleActionBlocker.h b/src/Common/SimpleActionBlocker.h deleted file mode 100644 index 4a96db0e09d..00000000000 --- a/src/Common/SimpleActionBlocker.h +++ /dev/null @@ -1,79 +0,0 @@ -#pragma once -#include - - -namespace DB -{ - -class SimpleActionLock; - - -/// Similar to ActionBlocker, but without weak_ptr magic -class SimpleActionBlocker -{ - using Counter = std::atomic; - Counter counter = 0; - -public: - - SimpleActionBlocker() = default; - - bool isCancelled() const { return counter > 0; } - - /// Temporarily blocks corresponding actions (while the returned object is alive) - friend class SimpleActionLock; - inline SimpleActionLock cancel(); - - /// Cancel the actions forever. - void cancelForever() { ++counter; } -}; - - -/// Blocks related action while a SimpleActionLock instance exists -class SimpleActionLock -{ - SimpleActionBlocker * block = nullptr; - -public: - - SimpleActionLock() = default; - - explicit SimpleActionLock(SimpleActionBlocker & block_) : block(&block_) - { - ++block->counter; - } - - SimpleActionLock(const SimpleActionLock &) = delete; - - SimpleActionLock(SimpleActionLock && rhs) noexcept - { - *this = std::move(rhs); - } - - SimpleActionLock & operator=(const SimpleActionLock &) = delete; - - SimpleActionLock & operator=(SimpleActionLock && rhs) noexcept - { - if (block) - --block->counter; - - block = rhs.block; - rhs.block = nullptr; - - return *this; - } - - ~SimpleActionLock() - { - if (block) - --block->counter; - } -}; - - -SimpleActionLock SimpleActionBlocker::cancel() -{ - return SimpleActionLock(*this); -} - -} diff --git a/src/Common/SmallObjectPool.h b/src/Common/SmallObjectPool.h deleted file mode 100644 index 4d56a92a419..00000000000 --- a/src/Common/SmallObjectPool.h +++ /dev/null @@ -1,52 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Can allocate memory objects of fixed size with deletion support. - * For small `object_size`s allocated no less than pointer size. - */ -class SmallObjectPool -{ -private: - const size_t object_size; - Arena pool; - char * free_list = nullptr; - -public: - explicit SmallObjectPool(size_t object_size_) - : object_size{std::max(object_size_, sizeof(char *))} - { - } - - char * alloc() - { - if (free_list) - { - char * res = free_list; - free_list = unalignedLoad(free_list); - return res; - } - - return pool.alloc(object_size); - } - - void free(char * ptr) - { - unalignedStore(ptr, free_list); - free_list = ptr; - } - - /// The size of the allocated pool in bytes - size_t size() const - { - return pool.size(); - } - -}; - -} diff --git a/src/Databases/DictionaryAttachInfo.h b/src/Databases/DictionaryAttachInfo.h deleted file mode 100644 index b2214d26f3c..00000000000 --- a/src/Databases/DictionaryAttachInfo.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -struct DictionaryAttachInfo -{ - ASTPtr create_query; - Poco::AutoPtr config; - time_t modification_time; -}; - -} diff --git a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h deleted file mode 100644 index b8dd6e278ad..00000000000 --- a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h +++ /dev/null @@ -1,32 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/// Repository from database, which stores dictionary definitions on disk. -/// Tracks update time and existence of .sql files through IDatabase. -class ExternalLoaderDatabaseConfigRepository : public IExternalLoaderConfigRepository, WithContext -{ -public: - ExternalLoaderDatabaseConfigRepository(IDatabase & database_, ContextPtr global_context_); - - std::string getName() const override { return database_name; } - - std::set getAllLoadablesDefinitionNames() override; - - bool exists(const std::string & loadable_definition_name) override; - - Poco::Timestamp getUpdateTime(const std::string & loadable_definition_name) override; - - LoadablesConfigurationPtr load(const std::string & loadable_definition_name) override; - -private: - const String database_name; - IDatabase & database; -}; - -} diff --git a/src/Processors/Executors/IReadProgressCallback.h b/src/Processors/Executors/IReadProgressCallback.h deleted file mode 100644 index 75a75eeb61d..00000000000 --- a/src/Processors/Executors/IReadProgressCallback.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -/// An interface for read progress callback. -class IReadProgressCallback -{ -public: - virtual ~IReadProgressCallback() = default; - virtual bool onProgress(uint64_t read_rows, uint64_t read_bytes) = 0; -}; - -using ReadProgressCallbackPtr = std::unique_ptr; - - -} diff --git a/src/Processors/Executors/traverse.h b/src/Processors/Executors/traverse.h deleted file mode 100644 index 2fd89adcb43..00000000000 --- a/src/Processors/Executors/traverse.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/// Look for first Ready or Async processor by depth-first search in needed input ports and full output ports. -/// NOTE: Pipeline must not have cycles. -//template -//void traverse(IProcessor & processor, Visit && visit) -//{ -// IProcessor::Status status = visit(processor); -// -// if (status == IProcessor::Status::Ready || status == IProcessor::Status::Async) -// return; -// -// if (status == IProcessor::Status::NeedData) -// for (auto & input : processor.getInputs()) -// if (input.isNeeded() && !input.hasData()) -// traverse(input.getOutputPort().getProcessor(), std::forward(visit)); -// -// if (status == IProcessor::Status::PortFull) -// for (auto & output : processor.getOutputs()) -// if (output.hasData()) -// traverse(output.getInputPort().getProcessor(), std::forward(visit)); -//} - -} diff --git a/src/Processors/QueueBuffer.h b/src/Processors/QueueBuffer.h deleted file mode 100644 index 0736d6fbf43..00000000000 --- a/src/Processors/QueueBuffer.h +++ /dev/null @@ -1,41 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Reads all data into queue. - * After all data has been read - output it in the same order. - */ -class QueueBuffer final : public IAccumulatingTransform -{ -private: - std::queue chunks; -public: - String getName() const override { return "QueueBuffer"; } - - explicit QueueBuffer(Block header) - : IAccumulatingTransform(header, header) - { - } - - void consume(Chunk block) override - { - chunks.push(std::move(block)); - } - - Chunk generate() override - { - if (chunks.empty()) - return {}; - - auto res = std::move(chunks.front()); - chunks.pop(); - return res; - } -}; - -} diff --git a/utils/check-style/check-style b/utils/check-style/check-style index f87d2e292b5..5076c737cd9 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -422,3 +422,7 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep # Cyrillic characters hiding inside Latin. find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '[a-zA-Z][а-яА-ЯёЁ]|[а-яА-ЯёЁ][a-zA-Z]' && echo "^ Cyrillic characters found in unexpected place." + +# Orphaned header files. +join -v1 <(find $ROOT_PATH/{src,programs,utils} -name '*.h' -printf '%f\n' | sort | uniq) <(find $ROOT_PATH/{src,programs,utils} -name '*.cpp' -or -name '*.c' -or -name '*.h' -or -name '*.S' | xargs grep --no-filename -o -P '[\w-]+\.h' | sort | uniq) | + grep . && echo '^ Found orphan header files.' From 57b160b6b9c7cdcf07efd70f947748b6a813f2f7 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 14 Nov 2023 07:12:12 +0000 Subject: [PATCH 278/813] compare type name in column description Signed-off-by: Duc Canh Le --- src/Storages/ColumnsDescription.cpp | 1 + ...cated_with_simple_aggregate_column.reference | 0 ..._replicated_with_simple_aggregate_column.sql | 17 +++++++++++++++++ 3 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.reference create mode 100644 tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.sql diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index dafd341a9f8..961cb8b4704 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -59,6 +59,7 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const return name == other.name && type->equals(*other.type) + && type->getName() == other.type->getName() && default_desc == other.default_desc && comment == other.comment && ast_to_str(codec) == ast_to_str(other.codec) diff --git a/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.reference b/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.sql b/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.sql new file mode 100644 index 00000000000..90da4725ace --- /dev/null +++ b/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.sql @@ -0,0 +1,17 @@ +CREATE TABLE t_r1 +( + `id` UInt64, + `val` SimpleAggregateFunction(max, Nullable(String)) +) +ENGINE = ReplicatedAggregatingMergeTree('/tables/{database}/t', 'r1') +ORDER BY id +SETTINGS index_granularity = 8192; + +CREATE TABLE t_r2 +( + `id` UInt64, + `val` SimpleAggregateFunction(anyLast, Nullable(String)) +) +ENGINE = ReplicatedAggregatingMergeTree('/tables/{database}/t', 'r2') +ORDER BY id +SETTINGS index_granularity = 8192; -- { serverError INCOMPATIBLE_COLUMNS } \ No newline at end of file From 3152cd25dc5ad7286648b0e44e483596a03fb961 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 14 Nov 2023 07:41:45 +0000 Subject: [PATCH 279/813] only check for exact column definitions in replicated merge tree Signed-off-by: Duc Canh Le --- src/Storages/ColumnsDescription.cpp | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 961cb8b4704..dafd341a9f8 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -59,7 +59,6 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const return name == other.name && type->equals(*other.type) - && type->getName() == other.type->getName() && default_desc == other.default_desc && comment == other.comment && ast_to_str(codec) == ast_to_str(other.codec) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 03ebe38e6cf..837c0c6bbbf 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1227,8 +1227,12 @@ bool StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr Coordination::Stat columns_stat; auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat)); - const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); - if (columns_from_zk == old_columns) + const auto & old_columns = metadata_snapshot->getColumns(); + + /// Replicated tables on different replicas must have exactly same column definitions + /// We cannot just compare column descriptions here because data types like SimpleAggregateFunction + /// may have different aggregate function in 1st argument but still compatible if 2nd argument is same. + if (columns_from_zk.toString() == old_columns.toString()) return true; if (!strict_check && metadata_stat.version != 0) From 5d81080e3108510ba0cd20883bc83fa9ed2ca7e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Nov 2023 09:01:50 +0100 Subject: [PATCH 280/813] Enable profiling for linker --- CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 9e548c5a6d0..060f3407bb2 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -273,6 +273,11 @@ option (ENABLE_BUILD_PROFILING "Enable profiling of build time" OFF) if (ENABLE_BUILD_PROFILING) if (COMPILER_CLANG) set (COMPILER_FLAGS "${COMPILER_FLAGS} -ftime-trace") + + if (LINKER_NAME MATCHES "lld") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--time-trace") + set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -Wl,--time-trace") + endif () else () message (${RECONFIGURE_MESSAGE_LEVEL} "Build profiling is only available with CLang") endif () From 5849e06595d96372d0e74c469550332b836c49ba Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 14 Nov 2023 09:07:17 +0000 Subject: [PATCH 281/813] Fix --- src/Coordination/KeeperServer.cpp | 2 +- src/Coordination/KeeperStateMachine.cpp | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index f229c4499c7..56c873bbbb9 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -616,6 +616,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ { const auto preprocess_logs = [&] { + keeper_context->local_logs_preprocessed = true; auto log_store = state_manager->load_log_store(); if (last_log_idx_on_disk > 0 && last_log_idx_on_disk > state_machine->last_commit_index()) { @@ -641,7 +642,6 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ { LOG_INFO(log, "All local log entries preprocessed"); } - keeper_context->local_logs_preprocessed = true; }; switch (type) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index cc0a07d2339..dcdd724f2bd 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -511,6 +511,10 @@ void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptrlocal_logs_preprocessed) + return; + auto request_for_session = parseRequest(data, true); // If we received a log from an older node, use the log_idx as the zxid // log_idx will always be larger or equal to the zxid so we can safely do this From 3f5ccabba60de0181319822292664452393a059c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 14 Nov 2023 08:32:37 +0000 Subject: [PATCH 282/813] Don't append different log types in same file --- src/Coordination/Changelog.cpp | 12 ++++++++++-- src/Coordination/Changelog.h | 1 + 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 3c2004a1b75..11f73da63d9 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -14,6 +14,7 @@ #include #include #include +#include "IO/CompressionMethod.h" #include @@ -476,6 +477,9 @@ struct ChangelogReadResult /// last offset we were able to read from log off_t last_position; + + /// Whether the changelog file was written using compression + bool compressed_log; bool error; }; @@ -484,7 +488,7 @@ class ChangelogReader public: explicit ChangelogReader(DiskPtr disk_, const std::string & filepath_) : disk(disk_), filepath(filepath_) { - auto compression_method = chooseCompressionMethod(filepath, ""); + compression_method = chooseCompressionMethod(filepath, ""); auto read_buffer_from_file = disk->readFile(filepath); read_buf = wrapReadBufferWithCompressionMethod(std::move(read_buffer_from_file), compression_method); } @@ -493,6 +497,7 @@ public: ChangelogReadResult readChangelog(IndexToLogEntry & logs, uint64_t start_log_index, Poco::Logger * log) { ChangelogReadResult result{}; + result.compressed_log = compression_method != CompressionMethod::None; try { while (!read_buf->eof()) @@ -583,6 +588,7 @@ public: private: DiskPtr disk; std::string filepath; + CompressionMethod compression_method; std::unique_ptr read_buf; }; @@ -590,6 +596,7 @@ Changelog::Changelog( Poco::Logger * log_, LogFileSettings log_file_settings, FlushSettings flush_settings_, KeeperContextPtr keeper_context_) : changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) + , compress_logs(log_file_settings.compress_logs) , log(log_) , write_operations(std::numeric_limits::max()) , append_completion_queue(std::numeric_limits::max()) @@ -830,7 +837,8 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin existing_changelogs.erase(last_log_read_result->log_start_index); std::erase_if(logs, [last_log_read_result](const auto & item) { return item.first >= last_log_read_result->log_start_index; }); } - else + /// don't mix compressed and uncompressed writes + else if (compress_logs == last_log_read_result->compressed_log) { initWriter(description); } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index dddcb9aa218..68d8c810823 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -182,6 +182,7 @@ private: const String changelogs_detached_dir; const uint64_t rotate_interval; + const bool compress_logs; Poco::Logger * log; std::mutex writer_mutex; From 1ab7e5693de914f6147e701788d023315c2d6de8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 14 Nov 2023 09:43:09 +0000 Subject: [PATCH 283/813] Add unit test --- src/Coordination/CoordinationSettings.h | 2 +- src/Coordination/tests/gtest_coordination.cpp | 96 +++++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 3cbfa3e449d..a58f2b04797 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -45,7 +45,7 @@ struct Settings; M(UInt64, max_requests_quick_batch_size, 100, "Max size of batch of requests to try to get before proceeding with RAFT. Keeper will not wait for requests but take only requests that are already in queue" , 0) \ M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ - M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \ + M(Bool, compress_logs, false, "Write compressed coordination logs in ZSTD format", 0) \ M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \ M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) \ M(UInt64, max_log_file_size, 50 * 1024 * 1024, "Max size of the Raft log file. If possible, each created log file will preallocate this amount of bytes on disk. Set to 0 to disable the limit", 0) \ diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 418b5225fa4..3658df12f69 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1102,6 +1102,102 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); } +TEST_F(CoordinationTest, ChangelogTestMixedLogTypes) +{ + ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); + + std::vector changelog_files; + + const auto verify_changelog_files = [&] + { + for (const auto & log_file : changelog_files) + EXPECT_TRUE(fs::exists(log_file)) << "File " << log_file << " not found"; + }; + + size_t last_term = 0; + size_t log_size = 0; + + const auto append_log = [&](auto & changelog, const std::string & data, uint64_t term) + { + last_term = term; + ++log_size; + auto entry = getLogEntry(data, last_term); + changelog.append(entry); + }; + + const auto verify_log_content = [&](const auto & changelog) + { + EXPECT_EQ(changelog.size(), log_size); + EXPECT_EQ(changelog.last_entry()->get_term(), last_term); + }; + + { + SCOPED_TRACE("Initial uncompressed log"); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = false, .rotate_interval = 20}, + DB::FlushSettings(), + keeper_context); + changelog.init(1, 0); + + for (size_t i = 0; i < 35; ++i) + append_log(changelog, std::to_string(i) + "_hello_world", (i+ 44) * 10); + + changelog.end_of_append_batch(0, 0); + + waitDurableLogs(changelog); + changelog_files.push_back("./logs/changelog_1_20.bin"); + changelog_files.push_back("./logs/changelog_21_40.bin"); + verify_changelog_files(); + + verify_log_content(changelog); + } + + { + SCOPED_TRACE("Compressed log"); + DB::KeeperLogStore changelog_compressed( + DB::LogFileSettings{.force_sync = true, .compress_logs = true, .rotate_interval = 20}, + DB::FlushSettings(), + keeper_context); + changelog_compressed.init(1, 0); + + verify_changelog_files(); + verify_log_content(changelog_compressed); + + append_log(changelog_compressed, "hello_world", 7777); + changelog_compressed.end_of_append_batch(0, 0); + + waitDurableLogs(changelog_compressed); + + verify_log_content(changelog_compressed); + + changelog_files.push_back("./logs/changelog_36_55.bin.zstd"); + verify_changelog_files(); + } + + { + SCOPED_TRACE("Final uncompressed log"); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = false, .rotate_interval = 20}, + DB::FlushSettings(), + keeper_context); + changelog.init(1, 0); + + verify_changelog_files(); + verify_log_content(changelog); + + append_log(changelog, "hello_world", 7778); + changelog.end_of_append_batch(0, 0); + + waitDurableLogs(changelog); + + verify_log_content(changelog); + + changelog_files.push_back("./logs/changelog_37_56.bin"); + verify_changelog_files(); + } +} + TEST_P(CoordinationTest, ChangelogTestLostFiles) { auto params = GetParam(); From 0ae79f6d183279ad3aa2a7b795fe77db74344cef Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Nov 2023 21:49:36 +0100 Subject: [PATCH 284/813] Fix crash --- src/Compression/CompressionCodecGCD.cpp | 31 +++++++++++++++++-- .../02896_gcd-codec-crash.reference | 0 .../0_stateless/02896_gcd-codec-crash.sql | 7 +++++ .../1_stateful/00178_gcd_codec.reference | 0 tests/queries/1_stateful/00178_gcd_codec.sql | 13 ++++++++ 5 files changed, 48 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02896_gcd-codec-crash.reference create mode 100644 tests/queries/0_stateless/02896_gcd-codec-crash.sql create mode 100644 tests/queries/1_stateful/00178_gcd_codec.reference create mode 100644 tests/queries/1_stateful/00178_gcd_codec.sql diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index d51e75067a1..cc3079f647f 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -1,10 +1,11 @@ #include +#include #include #include +#include +#include #include #include -#include "Common/Exception.h" -#include "DataTypes/IDataType.h" #include #include @@ -92,6 +93,16 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest) unalignedStore(dest, gcd_divider); dest += sizeof(T); + /// There are two cases: + /// 1) GCD is 0. It may happen if there are some zeros in the source data. + /// We cannot apply this codec anymore. So let's copy the source data to the destination. + /// 2) GCD is 1. It means that we can do nothing (except copying source data) and the result will be the same. + if unlikely(gcd_divider == 0) + { + memcpy(dest, source, source_size); + return; + } + if constexpr (sizeof(T) <= 8) { /// libdivide supports only UInt32 and UInt64. @@ -132,10 +143,24 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, const T gcd_multiplier = unalignedLoad(source); source += sizeof(T); + /// Again two cases: + /// 1) GCD is 0. This is "special" flag which signals what remaining data + /// left unchanged due to unapplicability of the codec. + /// 2) GCD is 1. Even if we proceed futher with the loop the resulting data will likely be the same. + if unlikely(gcd_multiplier == 0 || gcd_multiplier == 1) + { + /// Subtraction is safe, because we checked that source_size >= sizeof(T) + if unlikely(source_size - sizeof(T) != output_size) + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot GCD decompress"); + + memcpy(dest, source, source_size); + return; + } + while (source < source_end) { if (dest + sizeof(T) > dest_end) [[unlikely]] - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot GCD decompress"); unalignedStore(dest, unalignedLoad(source) * gcd_multiplier); source += sizeof(T); diff --git a/tests/queries/0_stateless/02896_gcd-codec-crash.reference b/tests/queries/0_stateless/02896_gcd-codec-crash.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02896_gcd-codec-crash.sql b/tests/queries/0_stateless/02896_gcd-codec-crash.sql new file mode 100644 index 00000000000..9af6906cb0f --- /dev/null +++ b/tests/queries/0_stateless/02896_gcd-codec-crash.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (a Nullable(Int64) CODEC (GCD,LZ4)) ENGINE=MergeTree Order by (); +INSERT INTO test SELECT 0 FROM numbers(1e2); +SELECT * FROM test FORMAT Null; + +DROP TABLE IF EXISTS test; \ No newline at end of file diff --git a/tests/queries/1_stateful/00178_gcd_codec.reference b/tests/queries/1_stateful/00178_gcd_codec.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/1_stateful/00178_gcd_codec.sql b/tests/queries/1_stateful/00178_gcd_codec.sql new file mode 100644 index 00000000000..443075b1b24 --- /dev/null +++ b/tests/queries/1_stateful/00178_gcd_codec.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS hits_gcd; + +CREATE TABLE hits_gcd (`WatchID` UInt64 CODEC (GCD,LZ4), `JavaEnable` UInt8 CODEC (GCD,LZ4), `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32 CODEC (GCD,LZ4), `ClientIP` UInt32 CODEC (GCD,LZ4), `ClientIP6` FixedString(16), `RegionID` UInt32 CODEC (GCD,LZ4), `UserID` UInt64 CODEC (GCD,LZ4), `CounterClass` Int8, `OS` UInt8 CODEC (GCD,LZ4), `UserAgent` UInt8 CODEC (GCD,LZ4), `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8 CODEC (GCD,LZ4), `IsRobot` UInt8 CODEC (GCD,LZ4), `RefererCategories` Array(UInt16) CODEC (GCD,LZ4), `URLCategories` Array(UInt16) CODEC (GCD,LZ4), `URLRegions` Array(UInt32) CODEC (GCD,LZ4), `RefererRegions` Array(UInt32) CODEC (GCD,LZ4), `ResolutionWidth` UInt16 CODEC (GCD,LZ4), `ResolutionHeight` UInt16 CODEC (GCD,LZ4), `ResolutionDepth` UInt8 CODEC (GCD,LZ4), `FlashMajor` UInt8 CODEC (GCD,LZ4), `FlashMinor` UInt8 CODEC (GCD,LZ4), `FlashMinor2` String, `NetMajor` UInt8 CODEC (GCD,LZ4), `NetMinor` UInt8 CODEC (GCD,LZ4), `UserAgentMajor` UInt16 CODEC (GCD,LZ4), `UserAgentMinor` FixedString(2), `CookieEnable` UInt8 CODEC (GCD,LZ4), `JavascriptEnable` UInt8 CODEC (GCD,LZ4), `IsMobile` UInt8 CODEC (GCD,LZ4), `MobilePhone` UInt8 CODEC (GCD,LZ4), `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32 CODEC (GCD,LZ4), `TraficSourceID` Int8, `SearchEngineID` UInt16 CODEC (GCD,LZ4), `SearchPhrase` String, `AdvEngineID` UInt8 CODEC (GCD,LZ4), `IsArtifical` UInt8 CODEC (GCD,LZ4), `WindowClientWidth` UInt16 CODEC (GCD,LZ4), `WindowClientHeight` UInt16 CODEC (GCD,LZ4), `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8 CODEC (GCD,LZ4), `SilverlightVersion2` UInt8 CODEC (GCD,LZ4), `SilverlightVersion3` UInt32 CODEC (GCD,LZ4), `SilverlightVersion4` UInt16 CODEC (GCD,LZ4), `PageCharset` String, `CodeVersion` UInt32 CODEC (GCD,LZ4), `IsLink` UInt8 CODEC (GCD,LZ4), `IsDownload` UInt8 CODEC (GCD,LZ4), `IsNotBounce` UInt8 CODEC (GCD,LZ4), `FUniqID` UInt64 CODEC (GCD,LZ4), `HID` UInt32 CODEC (GCD,LZ4), `IsOldCounter` UInt8 CODEC (GCD,LZ4), `IsEvent` UInt8 CODEC (GCD,LZ4), `IsParameter` UInt8 CODEC (GCD,LZ4), `DontCountHits` UInt8 CODEC (GCD,LZ4), `WithHash` UInt8 CODEC (GCD,LZ4), `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8 CODEC (GCD,LZ4), `Sex` UInt8 CODEC (GCD,LZ4), `Income` UInt8 CODEC (GCD,LZ4), `Interests` UInt16 CODEC (GCD,LZ4), `Robotness` UInt8 CODEC (GCD,LZ4), `GeneralInterests` Array(UInt16) CODEC (GCD,LZ4), `RemoteIP` UInt32 CODEC (GCD,LZ4), `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16 CODEC (GCD,LZ4), `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8 CODEC (GCD,LZ4), `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16 CODEC (GCD,LZ4), `GoalsReached` Array(UInt32) CODEC (GCD,LZ4), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8 CODEC (GCD,LZ4), `RefererHash` UInt64 CODEC (GCD,LZ4), `URLHash` UInt64 CODEC (GCD,LZ4), `CLID` UInt32 CODEC (GCD,LZ4), `YCLID` UInt64 CODEC (GCD,LZ4), `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32 CODEC (GCD,LZ4), `RequestTry` UInt8) + ENGINE = MergeTree() + PARTITION BY toYYYYMM(EventDate) + ORDER BY (CounterID, EventDate, intHash32(UserID)) + SAMPLE BY intHash32(UserID); + + +INSERT INTO hits_gcd SELECT * FROM hits; +SELECT * FROM hits_gcd FORMAT Null; + +DROP TABLE IF EXISTS hits_gcd; \ No newline at end of file From a7f8171d3ec581148f133e7b98d61785267f798b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Nov 2023 21:51:34 +0100 Subject: [PATCH 285/813] Update CompressionCodecGCD.cpp --- src/Compression/CompressionCodecGCD.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index cc3079f647f..b25f251a844 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include From 6811d80338a5d411429dc76d88cf25bd5db84cda Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Nov 2023 21:55:13 +0100 Subject: [PATCH 286/813] Better --- .../02896_gcd-codec-crash.reference | 200 ++++++++++++++++++ .../0_stateless/02896_gcd-codec-crash.sql | 10 +- 2 files changed, 208 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02896_gcd-codec-crash.reference b/tests/queries/0_stateless/02896_gcd-codec-crash.reference index e69de29bb2d..56f0d46cddc 100644 --- a/tests/queries/0_stateless/02896_gcd-codec-crash.reference +++ b/tests/queries/0_stateless/02896_gcd-codec-crash.reference @@ -0,0 +1,200 @@ +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02896_gcd-codec-crash.sql b/tests/queries/0_stateless/02896_gcd-codec-crash.sql index 9af6906cb0f..764d7168b07 100644 --- a/tests/queries/0_stateless/02896_gcd-codec-crash.sql +++ b/tests/queries/0_stateless/02896_gcd-codec-crash.sql @@ -1,7 +1,13 @@ DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS test2; CREATE TABLE test (a Nullable(Int64) CODEC (GCD,LZ4)) ENGINE=MergeTree Order by (); INSERT INTO test SELECT 0 FROM numbers(1e2); -SELECT * FROM test FORMAT Null; +SELECT * FROM test; -DROP TABLE IF EXISTS test; \ No newline at end of file +CREATE TABLE test2 (a Nullable(Int64) CODEC (GCD,LZ4)) ENGINE=MergeTree Order by (); +INSERT INTO test2 SELECT 1 FROM numbers(1e2); +SELECT * FROM test2; + +DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS test2; From 71f4f7519ba5f91925c2372d4e631dd6acc8625b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Nov 2023 21:56:18 +0100 Subject: [PATCH 287/813] Update CompressionCodecGCD.cpp --- src/Compression/CompressionCodecGCD.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index b25f251a844..6a3b889011a 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -143,7 +143,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, source += sizeof(T); /// Again two cases: - /// 1) GCD is 0. This is "special" flag which signals what remaining data + /// 1) GCD is 0. This is "special" flag which signals that remaining data /// left unchanged due to unapplicability of the codec. /// 2) GCD is 1. Even if we proceed futher with the loop the resulting data will likely be the same. if unlikely(gcd_multiplier == 0 || gcd_multiplier == 1) From a80f8082db60ea3026cff386cd72d5068adee50b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Nov 2023 21:58:43 +0100 Subject: [PATCH 288/813] Update CompressionCodecGCD.cpp --- src/Compression/CompressionCodecGCD.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index 6a3b889011a..cdda77a4150 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -96,7 +96,7 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest) /// 1) GCD is 0. It may happen if there are some zeros in the source data. /// We cannot apply this codec anymore. So let's copy the source data to the destination. /// 2) GCD is 1. It means that we can do nothing (except copying source data) and the result will be the same. - if unlikely(gcd_divider == 0) + if unlikely(gcd_divider == 0 || gcd_divider == 1) { memcpy(dest, source, source_size); return; From 40df5b8a74f36f47eb904dfd7669b76cc47a97c6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 13 Nov 2023 22:08:44 +0100 Subject: [PATCH 289/813] Update CompressionCodecGCD.cpp --- src/Compression/CompressionCodecGCD.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index cdda77a4150..a89603af4bc 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -145,7 +145,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, /// Again two cases: /// 1) GCD is 0. This is "special" flag which signals that remaining data /// left unchanged due to unapplicability of the codec. - /// 2) GCD is 1. Even if we proceed futher with the loop the resulting data will likely be the same. + /// 2) GCD is 1. Even if we proceed further with the loop the resulting data will likely be the same. if unlikely(gcd_multiplier == 0 || gcd_multiplier == 1) { /// Subtraction is safe, because we checked that source_size >= sizeof(T) From afb5c3e81047c6d00a14efc7858288e820b5ce50 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Nov 2023 21:36:30 +0000 Subject: [PATCH 290/813] Some fixups (mostly for consistency) - Made the exception texts consistent accross codecs - Slightly less noisy comments in C++ code - Remove unlikely() because it did not make a difference considering the total function costs - Consolidated tests into existing test, delete leftover 02869_gcd_codec_test_incorrect_type test (it was consolidated into 02872_gcd_codec long time ago) --- src/Compression/CompressionCodecDelta.cpp | 12 +- .../CompressionCodecDoubleDelta.cpp | 14 +- src/Compression/CompressionCodecFPC.cpp | 8 +- src/Compression/CompressionCodecGCD.cpp | 55 +++-- src/Compression/CompressionCodecGorilla.cpp | 10 +- src/Compression/CompressionCodecLZ4.cpp | 4 +- src/Compression/CompressionCodecT64.cpp | 14 +- src/Compression/CompressionCodecZSTD.cpp | 4 +- ...69_gcd_codec_test_incorrect_type.reference | 0 .../02869_gcd_codec_test_incorrect_type.sql | 2 - .../0_stateless/02872_gcd_codec.reference | 200 ++++++++++++++++++ tests/queries/0_stateless/02872_gcd_codec.sql | 24 ++- .../02896_gcd-codec-crash.reference | 200 ------------------ .../0_stateless/02896_gcd-codec-crash.sql | 13 -- 14 files changed, 278 insertions(+), 282 deletions(-) delete mode 100644 tests/queries/0_stateless/02869_gcd_codec_test_incorrect_type.reference delete mode 100644 tests/queries/0_stateless/02869_gcd_codec_test_incorrect_type.sql delete mode 100644 tests/queries/0_stateless/02896_gcd-codec-crash.reference delete mode 100644 tests/queries/0_stateless/02896_gcd-codec-crash.sql diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 113535489ac..924cdd9a60f 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -66,7 +66,7 @@ template void compressDataForType(const char * source, UInt32 source_size, char * dest) { if (source_size % sizeof(T) != 0) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot delta compress, data size {} is not aligned to {}", source_size, sizeof(T)); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with Delta codec, data size {} is not aligned to {}", source_size, sizeof(T)); T prev_src = 0; const char * const source_end = source + source_size; @@ -87,7 +87,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, const char * const output_end = dest + output_size; if (source_size % sizeof(T) != 0) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot delta decompress, data size {} is not aligned to {}", source_size, sizeof(T)); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress Delta-encoded data, data size {} is not aligned to {}", source_size, sizeof(T)); T accumulator{}; const char * const source_end = source + source_size; @@ -95,7 +95,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, { accumulator += unalignedLoadLittleEndian(source); if (dest + sizeof(accumulator) > output_end) [[unlikely]] - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress delta-encoded data"); unalignedStoreLittleEndian(dest, accumulator); source += sizeof(T); @@ -133,7 +133,7 @@ UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { if (source_size < 2) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress delta-encoded data. File has wrong header"); if (uncompressed_size == 0) return; @@ -141,13 +141,13 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_ UInt8 bytes_size = source[0]; if (!(bytes_size == 1 || bytes_size == 2 || bytes_size == 4 || bytes_size == 8)) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress delta-encoded data. File has wrong header"); UInt8 bytes_to_skip = uncompressed_size % bytes_size; UInt32 output_size = uncompressed_size - bytes_to_skip; if (static_cast(2 + bytes_to_skip) > source_size) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress delta-encoded data. File has wrong header"); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 60e81bbb640..99089ed6770 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -287,7 +287,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest) using SignedDeltaType = typename std::make_signed_t; if (source_size % sizeof(ValueType) != 0) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress, data size {} is not aligned to {}", + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with DoubleDelta codec, data size {} is not aligned to {}", source_size, sizeof(ValueType)); const char * source_end = source + source_size; const char * dest_start = dest; @@ -381,7 +381,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, prev_value = unalignedLoadLittleEndian(source); if (dest + sizeof(prev_value) > output_end) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress double-delta-encoded data"); unalignedStoreLittleEndian(dest, prev_value); source += sizeof(prev_value); @@ -394,7 +394,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, prev_delta = unalignedLoadLittleEndian(source); prev_value = prev_value + static_cast(prev_delta); if (dest + sizeof(prev_value) > output_end) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress double-delta-encoded data"); unalignedStoreLittleEndian(dest, prev_value); source += sizeof(prev_delta); @@ -427,7 +427,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, const UnsignedDeltaType delta = double_delta + prev_delta; const ValueType curr_value = prev_value + delta; if (dest + sizeof(curr_value) > output_end) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress double-delta-encoded data"); unalignedStoreLittleEndian(dest, curr_value); dest += sizeof(curr_value); @@ -511,18 +511,18 @@ UInt32 CompressionCodecDoubleDelta::doCompressData(const char * source, UInt32 s void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { if (source_size < 2) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress double-delta encoded data. File has wrong header"); UInt8 bytes_size = source[0]; if (bytes_size == 0) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress double-delta encoded data. File has wrong header"); UInt8 bytes_to_skip = uncompressed_size % bytes_size; UInt32 output_size = uncompressed_size - bytes_to_skip; if (static_cast(2 + bytes_to_skip) > source_size) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress double-delta encoded data. File has wrong header"); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/src/Compression/CompressionCodecFPC.cpp b/src/Compression/CompressionCodecFPC.cpp index 506093bbe49..81a27e30cd8 100644 --- a/src/Compression/CompressionCodecFPC.cpp +++ b/src/Compression/CompressionCodecFPC.cpp @@ -475,19 +475,19 @@ UInt32 CompressionCodecFPC::doCompressData(const char * source, UInt32 source_si default: break; } - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress. File has incorrect float width"); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with FPC codec. File has incorrect float width"); } void CompressionCodecFPC::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { if (source_size < HEADER_SIZE) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress FPC-encoded data. File has wrong header"); auto compressed_data = std::as_bytes(std::span(source, source_size)); auto compressed_float_width = std::to_integer(compressed_data[0]); auto compressed_level = std::to_integer(compressed_data[1]); if (compressed_level == 0 || compressed_level > MAX_COMPRESSION_LEVEL) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has incorrect level"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress FPC-encoded data. File has incorrect level"); auto destination = std::as_writable_bytes(std::span(dest, uncompressed_size)); auto src = compressed_data.subspan(HEADER_SIZE); @@ -500,7 +500,7 @@ void CompressionCodecFPC::doDecompressData(const char * source, UInt32 source_si FPCOperation(destination, compressed_level).decode(src, uncompressed_size); break; default: - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has incorrect float width"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress FPC-encoded data. File has incorrect float width"); } } diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index a89603af4bc..70fff01ebf7 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -1,10 +1,10 @@ #include -#include -#include -#include #include -#include +#include +#include +#include #include +#include #include #include @@ -74,29 +74,27 @@ template void compressDataForType(const char * source, UInt32 source_size, char * dest) { if (source_size % sizeof(T) != 0) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot GCD compress, data size {} is not aligned to {}", source_size, sizeof(T)); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with GCD codec, data size {} is not aligned to {}", source_size, sizeof(T)); const char * const source_end = source + source_size; - T gcd_divider = 0; + T gcd = 0; const auto * cur_source = source; - while (gcd_divider != T(1) && cur_source < source_end) + while (gcd != T(1) && cur_source < source_end) { if (cur_source == source) - gcd_divider = unalignedLoad(cur_source); + gcd = unalignedLoad(cur_source); else - gcd_divider = boost::integer::gcd(gcd_divider, unalignedLoad(cur_source)); + gcd = boost::integer::gcd(gcd, unalignedLoad(cur_source)); cur_source += sizeof(T); } - unalignedStore(dest, gcd_divider); + unalignedStore(dest, gcd); dest += sizeof(T); - /// There are two cases: - /// 1) GCD is 0. It may happen if there are some zeros in the source data. - /// We cannot apply this codec anymore. So let's copy the source data to the destination. - /// 2) GCD is 1. It means that we can do nothing (except copying source data) and the result will be the same. - if unlikely(gcd_divider == 0 || gcd_divider == 1) + /// GCD compression is pointless if GCD = 1 or GCD = 0 (happens with 0 values in data). + /// In these cases only copy the source to dest, i.e. don't compress. + if (gcd == 0 || gcd == 1) { memcpy(dest, source, source_size); return; @@ -106,7 +104,7 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest) { /// libdivide supports only UInt32 and UInt64. using LibdivideT = std::conditional_t; - libdivide::divider divider(static_cast(gcd_divider)); + libdivide::divider divider(static_cast(gcd)); cur_source = source; while (cur_source < source_end) { @@ -120,7 +118,7 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest) cur_source = source; while (cur_source < source_end) { - unalignedStore(dest, unalignedLoad(cur_source) / gcd_divider); + unalignedStore(dest, unalignedLoad(cur_source) / gcd); cur_source += sizeof(T); dest += sizeof(T); } @@ -131,10 +129,10 @@ template void decompressDataForType(const char * source, UInt32 source_size, char * dest, UInt32 output_size) { if (source_size % sizeof(T) != 0) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot GCD decompress, data size {} is not aligned to {}", source_size, sizeof(T)); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data, data size {} is not aligned to {}", source_size, sizeof(T)); if (source_size < sizeof(T)) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot GCD decompress, data size {} is less than {}", source_size, sizeof(T)); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data, data size {} is less than {}", source_size, sizeof(T)); const char * const source_end = source + source_size; const char * const dest_end = dest + output_size; @@ -142,15 +140,12 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, const T gcd_multiplier = unalignedLoad(source); source += sizeof(T); - /// Again two cases: - /// 1) GCD is 0. This is "special" flag which signals that remaining data - /// left unchanged due to unapplicability of the codec. - /// 2) GCD is 1. Even if we proceed further with the loop the resulting data will likely be the same. - if unlikely(gcd_multiplier == 0 || gcd_multiplier == 1) + /// Handle special cases GCD = 1 and GCD = 0. + if (gcd_multiplier == 0 || gcd_multiplier == 1) { /// Subtraction is safe, because we checked that source_size >= sizeof(T) - if unlikely(source_size - sizeof(T) != output_size) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot GCD decompress"); + if (source_size - sizeof(T) != output_size) + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data"); memcpy(dest, source, source_size); return; @@ -159,7 +154,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, while (source < source_end) { if (dest + sizeof(T) > dest_end) [[unlikely]] - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot GCD decompress"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data"); unalignedStore(dest, unalignedLoad(source) * gcd_multiplier); source += sizeof(T); @@ -203,7 +198,7 @@ UInt32 CompressionCodecGCD::doCompressData(const char * source, UInt32 source_si void CompressionCodecGCD::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { if (source_size < 2) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data. File has wrong header"); if (uncompressed_size == 0) return; @@ -211,13 +206,13 @@ void CompressionCodecGCD::doDecompressData(const char * source, UInt32 source_si UInt8 bytes_size = source[0]; if (!(bytes_size == 1 || bytes_size == 2 || bytes_size == 4 || bytes_size == 8 || bytes_size == 16 || bytes_size == 32)) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data. File has wrong header"); UInt8 bytes_to_skip = uncompressed_size % bytes_size; UInt32 output_size = uncompressed_size - bytes_to_skip; if (static_cast(2 + bytes_to_skip) > source_size) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data. File has wrong header"); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 7e36a424f03..a41a3d1fe8e 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -197,7 +197,7 @@ template UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) { if (source_size % sizeof(T) != 0) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress, data size {} is not aligned to {}", source_size, sizeof(T)); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with Gorilla codec, data size {} is not aligned to {}", source_size, sizeof(T)); const char * const source_end = source + source_size; const char * const dest_start = dest; @@ -317,7 +317,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) && curr_xored_info.data_bits == 0 && curr_xored_info.trailing_zero_bits == 0) [[unlikely]] { - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress gorilla-encoded data: corrupted input data."); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress Gorilla-encoded data: corrupted input data."); } xored_data = static_cast(reader.readBits(curr_xored_info.data_bits)); @@ -410,17 +410,17 @@ UInt32 CompressionCodecGorilla::doCompressData(const char * source, UInt32 sourc void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { if (source_size < 2) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress Gorilla-encoded data. File has wrong header"); UInt8 bytes_size = source[0]; if (bytes_size == 0) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress Gorilla-encoded data. File has wrong header"); UInt8 bytes_to_skip = uncompressed_size % bytes_size; if (static_cast(2 + bytes_to_skip) > source_size) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress. File has wrong header"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress Gorilla-encoded data. File has wrong header"); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/src/Compression/CompressionCodecLZ4.cpp b/src/Compression/CompressionCodecLZ4.cpp index ae739ab4e6b..597c1ba3acb 100644 --- a/src/Compression/CompressionCodecLZ4.cpp +++ b/src/Compression/CompressionCodecLZ4.cpp @@ -96,7 +96,7 @@ void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_si bool success = LZ4::decompress(source, dest, source_size, uncompressed_size, lz4_stat); if (!success) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress LZ4-encoded data"); } void registerCodecLZ4(CompressionCodecFactory & factory) @@ -112,7 +112,7 @@ UInt32 CompressionCodecLZ4HC::doCompressData(const char * source, UInt32 source_ auto success = LZ4_compress_HC(source, dest, source_size, LZ4_COMPRESSBOUND(source_size), level); if (!success) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot LZ4_compress_HC"); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with LZ4 codec"); return success; } diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index ec7922a030d..bf9a9414bc1 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -483,7 +483,7 @@ UInt32 compressData(const char * src, UInt32 bytes_size, char * dst) static constexpr const UInt32 header_size = 2 * sizeof(UInt64); if (bytes_size % sizeof(T)) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress, data size {} is not multiplier of {}", + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with T64 codec, data size {} is not multiplier of {}", bytes_size, sizeof(T)); UInt32 src_size = bytes_size / sizeof(T); @@ -538,11 +538,11 @@ void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 unco static constexpr const UInt32 header_size = 2 * sizeof(UInt64); if (bytes_size < header_size) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress, data size ({}) is less than the size of T64 header", + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress T64-encoded data, data size ({}) is less than the size of T64 header", bytes_size); if (uncompressed_size % sizeof(T)) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress, unexpected uncompressed size ({})" + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress T64-encoded data, unexpected uncompressed size ({})" " isn't a multiple of the data type size ({})", uncompressed_size, sizeof(T)); @@ -571,7 +571,7 @@ void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 unco UInt32 dst_shift = sizeof(T) * matrix_size; if (!bytes_size || bytes_size % src_shift) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress, data size ({}) is not a multiplier of {}", + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress T64-encoded data, data size ({}) is not a multiplier of {}", bytes_size, src_shift); UInt32 num_full = bytes_size / src_shift; @@ -666,13 +666,13 @@ UInt32 CompressionCodecT64::doCompressData(const char * src, UInt32 src_size, ch break; } - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with T64"); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with T64 codec"); } void CompressionCodecT64::doDecompressData(const char * src, UInt32 src_size, char * dst, UInt32 uncompressed_size) const { if (!src_size) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress with T64"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress T64-encoded data"); UInt8 cookie = unalignedLoad(src); src += 1; @@ -703,7 +703,7 @@ void CompressionCodecT64::doDecompressData(const char * src, UInt32 src_size, ch break; } - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress with T64"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress T64-encoded data"); } uint8_t CompressionCodecT64::getMethodByte() const diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index 891f1acaa9a..ec37ec6a7b5 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -82,7 +82,7 @@ UInt32 CompressionCodecZSTD::doCompressData(const char * source, UInt32 source_s ZSTD_freeCCtx(cctx); if (ZSTD_isError(compressed_size)) - throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress block with ZSTD: {}", std::string(ZSTD_getErrorName(compressed_size))); + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress with ZSTD codec: {}", std::string(ZSTD_getErrorName(compressed_size))); return static_cast(compressed_size); } @@ -93,7 +93,7 @@ void CompressionCodecZSTD::doDecompressData(const char * source, UInt32 source_s size_t res = ZSTD_decompress(dest, uncompressed_size, source, source_size); if (ZSTD_isError(res)) - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot ZSTD_decompress: {}", std::string(ZSTD_getErrorName(res))); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress ZSTD-encoded data: {}", std::string(ZSTD_getErrorName(res))); } CompressionCodecZSTD::CompressionCodecZSTD(int level_, int window_log_) : level(level_), enable_long_range(true), window_log(window_log_) diff --git a/tests/queries/0_stateless/02869_gcd_codec_test_incorrect_type.reference b/tests/queries/0_stateless/02869_gcd_codec_test_incorrect_type.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02869_gcd_codec_test_incorrect_type.sql b/tests/queries/0_stateless/02869_gcd_codec_test_incorrect_type.sql deleted file mode 100644 index 61a964a288f..00000000000 --- a/tests/queries/0_stateless/02869_gcd_codec_test_incorrect_type.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS table_gcd_codec; -CREATE TABLE table_gcd_codec (str String CODEC(GCD, LZ4)) ENGINE = Memory; -- { serverError 36 } diff --git a/tests/queries/0_stateless/02872_gcd_codec.reference b/tests/queries/0_stateless/02872_gcd_codec.reference index 1dd1b67e047..a8224b85b3c 100644 --- a/tests/queries/0_stateless/02872_gcd_codec.reference +++ b/tests/queries/0_stateless/02872_gcd_codec.reference @@ -1002,3 +1002,203 @@ 0 0 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02872_gcd_codec.sql b/tests/queries/0_stateless/02872_gcd_codec.sql index 245a1211052..3020bbc44ac 100644 --- a/tests/queries/0_stateless/02872_gcd_codec.sql +++ b/tests/queries/0_stateless/02872_gcd_codec.sql @@ -103,8 +103,24 @@ SELECT * FROM table_gcd_codec_datetime; SELECT * FROM table_gcd_codec_datetime64; --- A column with all 0 values can be compressed/decompressed +-- A column with all zero values can be compressed/decompressed -CREATE TEMPORARY TABLE table_gcd_codec_only_zero_values (n UInt8 CODEC(GCD, LZ4)) ENGINE = Memory; -INSERT INTO table_gcd_codec_only_zero_values VALUES (0), (0), (0); -SELECT * FROM table_gcd_codec_only_zero_values; +CREATE TEMPORARY TABLE table_gcd_codec_only_zeros (n UInt8 CODEC(GCD, LZ4)) ENGINE = Memory; +INSERT INTO table_gcd_codec_only_zeros VALUES (0), (0), (0); +SELECT * FROM table_gcd_codec_only_zeros; + +-- Tests for Bug #56672: + +DROP TABLE IF EXISTS table_gcd_codec_one_hundred_zeros; +DROP TABLE IF EXISTS table_gcd_codec_one_hundred_ones; + +CREATE TABLE table_gcd_codec_one_hundred_zeros (a Nullable(Int64) CODEC (GCD,LZ4)) ENGINE=MergeTree ORDER BY (); +INSERT INTO table_gcd_codec_one_hundred_zeros SELECT 0 FROM numbers(100); +SELECT * FROM table_gcd_codec_one_hundred_zeros; + +CREATE TABLE table_gcd_codec_one_hundred_ones (a Nullable(Int64) CODEC (GCD,LZ4)) ENGINE=MergeTree Order by (); +INSERT INTO table_gcd_codec_one_hundred_ones SELECT 1 FROM numbers(100); +SELECT * FROM table_gcd_codec_one_hundred_ones; + +DROP TABLE table_gcd_codec_one_hundred_zeros; +DROP TABLE table_gcd_codec_one_hundred_ones; diff --git a/tests/queries/0_stateless/02896_gcd-codec-crash.reference b/tests/queries/0_stateless/02896_gcd-codec-crash.reference deleted file mode 100644 index 56f0d46cddc..00000000000 --- a/tests/queries/0_stateless/02896_gcd-codec-crash.reference +++ /dev/null @@ -1,200 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02896_gcd-codec-crash.sql b/tests/queries/0_stateless/02896_gcd-codec-crash.sql deleted file mode 100644 index 764d7168b07..00000000000 --- a/tests/queries/0_stateless/02896_gcd-codec-crash.sql +++ /dev/null @@ -1,13 +0,0 @@ -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test2; - -CREATE TABLE test (a Nullable(Int64) CODEC (GCD,LZ4)) ENGINE=MergeTree Order by (); -INSERT INTO test SELECT 0 FROM numbers(1e2); -SELECT * FROM test; - -CREATE TABLE test2 (a Nullable(Int64) CODEC (GCD,LZ4)) ENGINE=MergeTree Order by (); -INSERT INTO test2 SELECT 1 FROM numbers(1e2); -SELECT * FROM test2; - -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test2; From 770a762317b88060f71bafdbff4f31a60e5ad8d1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 3 Nov 2023 19:01:33 +0100 Subject: [PATCH 291/813] aggressive timeout --- src/Backups/BackupIO_S3.cpp | 4 +++- src/Core/Settings.h | 1 + src/IO/ConnectionTimeouts.cpp | 20 ++++++++++++++++ src/IO/ConnectionTimeouts.h | 2 ++ src/IO/S3/Client.cpp | 1 + src/IO/S3/PocoHTTPClient.cpp | 43 +++++++++++++++++++++++++++++++---- src/IO/S3/PocoHTTPClient.h | 5 ++++ 7 files changed, 71 insertions(+), 5 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 8bb2f895e38..0b700665988 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -55,7 +55,9 @@ namespace static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false, request_settings.get_request_throttler, request_settings.put_request_throttler, + /* for_disk_s3 = */ false, + request_settings.get_request_throttler, + request_settings.put_request_throttler, s3_uri.uri.getScheme()); client_configuration.endpointOverride = s3_uri.endpoint; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3b90a3e068b..b1459b6f328 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,6 +94,7 @@ class IColumn; 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_aggressive_timeouts, true, "When aggressive 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) \ diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 01fbaa4f817..a9eebb1a755 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -133,4 +133,24 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings settings.http_receive_timeout); } +ConnectionTimeouts ConnectionTimeouts::aggressiveTimeouts(UInt32 attempt) const +{ + auto aggressive = *this; + + if (attempt == 2) + { + auto one_second = Poco::Timespan(1, 0); + aggressive.send_timeout = saturate(one_second, send_timeout); + aggressive.receive_timeout = saturate(one_second, receive_timeout); + } + else if (attempt == 1) + { + auto two_hundred_ms = Poco::Timespan(0, 200 * 1000); + aggressive.send_timeout = saturate(two_hundred_ms, send_timeout); + aggressive.receive_timeout = saturate(two_hundred_ms, receive_timeout); + } + + return aggressive; +} + } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 684af42827f..17ee1907d89 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -67,6 +67,8 @@ struct ConnectionTimeouts /// Timeouts for the case when we will try many addresses in a loop. static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings); static ConnectionTimeouts getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout); + + ConnectionTimeouts aggressiveTimeouts(UInt32 attempt) const; }; } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index ceb7d275299..4250342c49f 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -905,6 +905,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT s3_retry_attempts, enable_s3_requests_logging, for_disk_s3, + context->getGlobalContext()->getSettingsRef().s3_aggressive_timeouts, get_request_throttler, put_request_throttler, error_report); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index d0f248f48a6..08ba04ee875 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -99,6 +100,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( unsigned int s3_retry_attempts_, bool enable_s3_requests_logging_, bool for_disk_s3_, + bool s3_aggressive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, std::function error_report_) @@ -111,6 +113,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , for_disk_s3(for_disk_s3_) , get_request_throttler(get_request_throttler_) , put_request_throttler(put_request_throttler_) + , s3_aggressive_timeouts(s3_aggressive_timeouts_) , error_report(error_report_) { } @@ -157,6 +160,7 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config Poco::Timespan(client_configuration.http_keep_alive_timeout_ms * 1000))) /// flag indicating whether keep-alive is enabled is set to each session upon creation , remote_host_filter(client_configuration.remote_host_filter) , s3_max_redirects(client_configuration.s3_max_redirects) + , s3_aggressive_timeouts(client_configuration.s3_aggressive_timeouts) , enable_s3_requests_logging(client_configuration.enable_s3_requests_logging) , for_disk_s3(client_configuration.for_disk_s3) , get_request_throttler(client_configuration.get_request_throttler) @@ -268,6 +272,37 @@ void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricT ProfileEvents::increment(disk_s3_events_map[static_cast(type)][static_cast(kind)], amount); } +UInt32 extractAttempt(const Aws::String & request_info) +{ + static auto key = Aws::String("attempt="); + + auto key_begin = request_info.find(key, 0); + if (key_begin == Aws::String::npos) + return 1; + + auto val_begin = key_begin + key.size(); + auto val_end = request_info.find(';', val_begin); + if (val_end == Aws::String::npos) + val_end = request_info.size(); + + Aws::String value = request_info.substr(val_begin, val_end-val_begin); + + UInt32 attempt = 1; + ReadBufferFromString buf(value); + readIntText(attempt, buf); + return attempt; +} + +ConnectionTimeouts PocoHTTPClient::getTimeouts(Aws::Http::HttpRequest & request) const +{ + if (!s3_aggressive_timeouts) + return timeouts; + + const auto & request_info = request.GetHeaderValue(Aws::Http::SDK_REQUEST_HEADER); + auto attempt = extractAttempt(request_info); + return timeouts.aggressiveTimeouts(attempt); +} + void PocoHTTPClient::makeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, @@ -348,17 +383,17 @@ void PocoHTTPClient::makeRequestInternalImpl( /// This can lead to request signature difference on S3 side. if constexpr (pooled) session = makePooledHTTPSession( - target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit, proxy_configuration); + target_uri, getTimeouts(request), http_connection_pool_size, wait_on_pool_size_limit, proxy_configuration); else - session = makeHTTPSession(target_uri, timeouts, proxy_configuration); + session = makeHTTPSession(target_uri, getTimeouts(request), proxy_configuration); } else { if constexpr (pooled) session = makePooledHTTPSession( - target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit); + target_uri, getTimeouts(request), http_connection_pool_size, wait_on_pool_size_limit); else - session = makeHTTPSession(target_uri, timeouts); + session = makeHTTPSession(target_uri, getTimeouts(request)); } /// In case of error this address will be written to logs diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 2a449458360..6eeff431569 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -55,6 +55,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration size_t http_connection_pool_size = 0; /// See PoolBase::BehaviourOnLimit bool wait_on_pool_size_limit = true; + bool s3_aggressive_timeouts = false; std::function error_report; @@ -69,6 +70,7 @@ private: unsigned int s3_retry_attempts, bool enable_s3_requests_logging_, bool for_disk_s3_, + bool s3_aggressive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, std::function error_report_ @@ -169,6 +171,8 @@ private: Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; + ConnectionTimeouts getTimeouts(Aws::Http::HttpRequest & request) const; + protected: static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; @@ -178,6 +182,7 @@ protected: ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; + bool s3_aggressive_timeouts = false; bool enable_s3_requests_logging; bool for_disk_s3; From ab2594154e35a4fed769de14bfc2b720598dfaa7 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 6 Nov 2023 20:57:16 +0100 Subject: [PATCH 292/813] add tests --- src/Disks/ObjectStorages/S3/diskSettings.cpp | 6 +- .../configs/inf_s3_retries.xml | 1 + .../configs/s3_retries.xml | 1 + .../configs/storage_conf.xml | 15 +++- .../test_checking_s3_blobs_paranoid/test.py | 74 +++++++++++++++++-- 5 files changed, 87 insertions(+), 10 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index de88c876922..43618c64776 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -63,10 +63,12 @@ std::unique_ptr getClient( client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 3000); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100); client_configuration.endpointOverride = uri.endpoint; - client_configuration.http_keep_alive_timeout_ms - = config.getUInt(config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); + client_configuration.http_keep_alive_timeout_ms = config.getUInt( + config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); client_configuration.wait_on_pool_size_limit = false; + client_configuration.s3_aggressive_timeouts = config.getUInt( + config_prefix + ".aggressive_timeouts", client_configuration.s3_aggressive_timeouts); /* * Override proxy configuration for backwards compatibility with old configuration format. diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml index 206eb4f2bad..5f0860ac120 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml @@ -4,6 +4,7 @@ 1000000 + 1 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml index 556bf60d385..f215a89f613 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml @@ -4,6 +4,7 @@ 5 + 0 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml index b77e72d808b..264c411b59b 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -7,6 +7,12 @@ + + s3 + http://minio1:9001/root/data/ + minio + minio123 + s3 http://resolver:8083/root/data/ @@ -23,9 +29,16 @@ + + +

+ s3 +
+ + - broken_s3 + s3 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index d6bcb3fb8f4..7f8664f1648 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -64,6 +64,8 @@ def test_upload_after_check_works(cluster, broken_s3): data String ) ENGINE=MergeTree() ORDER BY id + SETTINGS + storage_policy='broken_s3' """ ) @@ -87,7 +89,8 @@ def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): SELECT ProfileEvents['S3CreateMultipartUpload'], ProfileEvents['S3UploadPart'], - ProfileEvents['S3WriteRequestsErrors'] + ProfileEvents['S3WriteRequestsErrors'], + ProfileEvents['S3PutObject'], FROM system.query_log WHERE query_id='{query_id}' AND type='{log_type}' @@ -129,7 +132,7 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( node, insert_query_id ) assert count_create_multi_part_uploads == 1 @@ -172,7 +175,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( node, insert_query_id ) assert count_create_multi_part_uploads == 1 @@ -207,7 +210,7 @@ def test_when_s3_connection_refused_is_retried(cluster, broken_s3): query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( node, insert_query_id, log_type="QueryFinish" ) assert count_create_multi_part_uploads == 1 @@ -279,7 +282,7 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( node, insert_query_id, log_type="QueryFinish" ) @@ -361,7 +364,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( node, insert_query_id, log_type="QueryFinish" ) @@ -438,7 +441,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( node, insert_query_id, log_type="QueryFinish" ) @@ -533,3 +536,60 @@ def test_query_is_canceled_with_inf_retries(cluster, broken_s3): retry_count=120, sleep_time=1, ) + + +@pytest.mark.parametrize("node_name", ["node", "node_with_inf_s3_retries"]) +def test_aggressive_timeouts(cluster, broken_s3, node_name): + node = cluster.instances[node_name] + + broken_s3.setup_fake_puts(part_length=1) + broken_s3.setup_slow_answers( + timeout=5, + count=1000000, + ) + + insert_query_id = f"TEST_AGGRESSIVE_TIMEOUTS_{node_name}" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/aggressive_timeouts', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1 + SETTINGS + s3_request_timeout_ms=30000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + broken_s3.reset() + + _, _, count_s3_errors, count_s3_puts = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert count_s3_puts == 1 + + s3_aggressive_timeouts_state = node.query( + f""" + SELECT + value + FROM system.settings + WHERE + name='s3_aggressive_timeouts' + """ + ).strip() + + if node_name == "node_with_inf_s3_retries": + # first 2 attempts failed + assert s3_aggressive_timeouts_state == "1" + assert count_s3_errors == 2 + else: + assert s3_aggressive_timeouts_state == "0" + assert count_s3_errors == 0 From e0edd165da4df6d700fcde818cc91492f295323f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 7 Nov 2023 11:21:46 +0100 Subject: [PATCH 293/813] add doc --- docs/en/operations/settings/settings.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index d0acad7b557..306529c4b96 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4820,3 +4820,10 @@ When set to `true` the metadata files are written with `VERSION_FULL_OBJECT_KEY` When set to `false` the metadata files are written with the previous format version, `VERSION_INLINE_DATA`. With that format only suffixes of object storage key names are are written to the metadata files. The prefix for all of object storage key names is set in configurations files at `storage_configuration.disks` section. Default value: `false`. + +## s3_aggressive_timeouts {#s3_aggressive_timeouts} + +When set to `true` than for all s3 requests first two attempts are made with low send and receive timeouts. +When set to `false` than all attempts are made with identical timeouts. + +Default value: `true`. From 338c51745f1709220a01a3684b5e305ff64ff788 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 7 Nov 2023 12:08:38 +0100 Subject: [PATCH 294/813] fix style --- .../test_checking_s3_blobs_paranoid/test.py | 75 ++++++++++++------- 1 file changed, 46 insertions(+), 29 deletions(-) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 7f8664f1648..441a5a541e8 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -80,7 +80,7 @@ def test_upload_after_check_works(cluster, broken_s3): assert "suddenly disappeared" in error, error -def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): +def get_multipart_counters(node, query_id, log_type="ExceptionWhileProcessing"): node.query("SYSTEM FLUSH LOGS") return [ int(x) @@ -90,7 +90,24 @@ def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): ProfileEvents['S3CreateMultipartUpload'], ProfileEvents['S3UploadPart'], ProfileEvents['S3WriteRequestsErrors'], + FROM system.query_log + WHERE query_id='{query_id}' + AND type='{log_type}' + """ + ).split() + if x + ] + + +def get_put_counters(node, query_id, log_type="ExceptionWhileProcessing"): + node.query("SYSTEM FLUSH LOGS") + return [ + int(x) + for x in node.query( + f""" + SELECT ProfileEvents['S3PutObject'], + ProfileEvents['S3WriteRequestsErrors'], FROM system.query_log WHERE query_id='{query_id}' AND type='{log_type}' @@ -132,12 +149,12 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 0 - assert count_s3_errors == 1 + assert create_multipart == 1 + assert upload_parts == 0 + assert s3_errors == 1 # Add "lz4" compression method in the list after https://github.com/ClickHouse/ClickHouse/issues/50975 is fixed @@ -175,12 +192,12 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts >= 2 - assert count_s3_errors >= 2 + assert create_multipart == 1 + assert upload_parts >= 2 + assert s3_errors >= 2 def test_when_s3_connection_refused_is_retried(cluster, broken_s3): @@ -210,12 +227,12 @@ def test_when_s3_connection_refused_is_retried(cluster, broken_s3): query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 39 - assert count_s3_errors == 3 + assert create_multipart == 1 + assert upload_parts == 39 + assert s3_errors == 3 broken_s3.setup_at_part_upload(count=1000, after=2, action="connection_refused") insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED_1" @@ -282,13 +299,13 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 39 - assert count_s3_errors == 3 + assert create_multipart == 1 + assert upload_parts == 39 + assert s3_errors == 3 broken_s3.setup_at_part_upload( count=1000, @@ -364,13 +381,13 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 39 - assert count_s3_errors == 3 + assert create_multipart == 1 + assert upload_parts == 39 + assert s3_errors == 3 broken_s3.setup_at_create_multi_part_upload( count=1000, @@ -441,13 +458,13 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors, _ = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 7 - assert count_s3_errors == 3 + assert create_multipart == 1 + assert upload_parts == 7 + assert s3_errors == 3 broken_s3.setup_at_part_upload( count=1000, @@ -570,11 +587,11 @@ def test_aggressive_timeouts(cluster, broken_s3, node_name): broken_s3.reset() - _, _, count_s3_errors, count_s3_puts = get_counters( + put_objects, s3_errors = get_put_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_s3_puts == 1 + assert put_objects == 1 s3_aggressive_timeouts_state = node.query( f""" @@ -589,7 +606,7 @@ def test_aggressive_timeouts(cluster, broken_s3, node_name): if node_name == "node_with_inf_s3_retries": # first 2 attempts failed assert s3_aggressive_timeouts_state == "1" - assert count_s3_errors == 2 + assert s3_errors == 2 else: assert s3_aggressive_timeouts_state == "0" - assert count_s3_errors == 0 + assert s3_errors == 0 From 1a7be21a66eb498a6d52718d9cd7f37ba9729213 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 7 Nov 2023 18:05:57 +0100 Subject: [PATCH 295/813] adjust tests --- .../configs/config.d/storage_conf.xml | 2 ++ tests/integration/test_storage_s3/configs/defaultS3.xml | 5 ----- tests/integration/test_storage_s3/configs/s3_retry.xml | 2 +- 3 files changed, 3 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml index 235b9a7b7a1..f51b854de75 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml @@ -11,6 +11,7 @@ true 0 + 0 20000 @@ -33,6 +34,7 @@ true 1 + 0 1 20000 diff --git a/tests/integration/test_storage_s3/configs/defaultS3.xml b/tests/integration/test_storage_s3/configs/defaultS3.xml index 37454ef6781..7dac6d9fbb5 100644 --- a/tests/integration/test_storage_s3/configs/defaultS3.xml +++ b/tests/integration/test_storage_s3/configs/defaultS3.xml @@ -1,9 +1,4 @@ - - - 5 - - http://resolver:8080 diff --git a/tests/integration/test_storage_s3/configs/s3_retry.xml b/tests/integration/test_storage_s3/configs/s3_retry.xml index 727e23273cf..581fc44c8d4 100644 --- a/tests/integration/test_storage_s3/configs/s3_retry.xml +++ b/tests/integration/test_storage_s3/configs/s3_retry.xml @@ -1,7 +1,7 @@ - 5 + 10 From 45de9beab4231a806ea247adef0a1fc5180748ba Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 8 Nov 2023 13:19:28 +0100 Subject: [PATCH 296/813] set new timeout for session from connection pool --- base/poco/Net/src/HTTPSession.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index d2663baaf9f..d30f5590280 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -94,8 +94,22 @@ void HTTPSession::setTimeout(const Poco::Timespan& timeout) void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco::Timespan& sendTimeout, const Poco::Timespan& receiveTimeout) { _connectionTimeout = connectionTimeout; - _sendTimeout = sendTimeout; - _receiveTimeout = receiveTimeout; + + if (_sendTimeout != sendTimeout) + { + _sendTimeout = sendTimeout; + + if (connected()) + _socket.setSendTimeout(_sendTimeout); + } + + if (_receiveTimeout != receiveTimeout) + { + _receiveTimeout = receiveTimeout; + + if (connected()) + _socket.setReceiveTimeout(_receiveTimeout); + } } From be01a5cd3e07eeba990a8dcc3e69e62f3492d05e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 8 Nov 2023 17:32:06 +0100 Subject: [PATCH 297/813] turn off agressive timeouts for heavy requests --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 6 +++++- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 1 + src/IO/S3/Client.cpp | 9 ++++++--- src/IO/S3/Client.h | 6 ++---- 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index b36185249af..aa4bcd7fbad 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -537,7 +537,11 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( } S3ObjectStorage::Clients::Clients(std::shared_ptr client_, const S3ObjectStorageSettings & settings) - : client(std::move(client_)), client_with_long_timeout(client->clone(std::nullopt, settings.request_settings.long_request_timeout_ms)) {} + : client(std::move(client_)) + , client_with_long_timeout(client->clone( + /*override_aggressive_timeouts*/ false, + settings.request_settings.long_request_timeout_ms)) +{} ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string &) const { diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index b1b3fb22366..37e491e21dc 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -184,6 +184,7 @@ private: std::string bucket; String object_key_prefix; + MultiVersion clients; MultiVersion s3_settings; S3Capabilities s3_capabilities; diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 4250342c49f..12a0cb8f93c 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -119,14 +119,17 @@ std::unique_ptr Client::create( } std::unique_ptr Client::clone( - std::optional> override_retry_strategy, + std::optional override_aggressive_timeouts, std::optional override_request_timeout_ms) const { PocoHTTPClientConfiguration new_configuration = client_configuration; - if (override_retry_strategy.has_value()) - new_configuration.retryStrategy = *override_retry_strategy; + if (override_request_timeout_ms.has_value()) new_configuration.requestTimeoutMs = *override_request_timeout_ms; + + if (override_aggressive_timeouts.has_value()) + new_configuration.s3_aggressive_timeouts = *override_aggressive_timeouts; + return std::unique_ptr(new Client(*this, new_configuration)); } diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 48310bc21af..81ab3854d3d 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -119,13 +119,11 @@ public: bool use_virtual_addressing); /// Create a client with adjusted settings: - /// * override_retry_strategy can be used to disable retries to avoid nested retries when we have - /// a retry loop outside of S3 client. Specifically, for read and write buffers. Currently not - /// actually used. /// * override_request_timeout_ms is used to increase timeout for CompleteMultipartUploadRequest /// because it often sits idle for 10 seconds: https://github.com/ClickHouse/ClickHouse/pull/42321 + /// * s3_aggressive_timeouts is used to turn off s3_aggressive_timeouts feature for CompleteMultipartUploadRequest std::unique_ptr clone( - std::optional> override_retry_strategy = std::nullopt, + std::optional override_aggressive_timeouts = std::nullopt, std::optional override_request_timeout_ms = std::nullopt) const; Client & operator=(const Client &) = delete; From 27fb25d056c420bca141ce2ecd83868d15fd07ef Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 9 Nov 2023 13:10:52 +0100 Subject: [PATCH 298/813] alter the naming, fix client_with_long_timeout in s3 storage --- src/Core/Settings.h | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 4 ++-- src/IO/S3/Client.cpp | 8 ++++---- src/IO/S3/Client.h | 4 ++-- src/IO/S3/PocoHTTPClient.cpp | 8 ++++---- src/IO/S3/PocoHTTPClient.h | 6 +++--- src/Storages/StorageS3.cpp | 2 +- 8 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b1459b6f328..3f80c83ff5f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,7 +94,7 @@ class IColumn; 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_aggressive_timeouts, true, "When aggressive timeouts are enabled first two attempts are made with low receive and send timeout", 0) \ + M(Bool, s3_use_adaptive_timeouts, true, "When aggressive 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) \ diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index aa4bcd7fbad..8a46bfd59d1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -539,7 +539,7 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( S3ObjectStorage::Clients::Clients(std::shared_ptr client_, const S3ObjectStorageSettings & settings) : client(std::move(client_)) , client_with_long_timeout(client->clone( - /*override_aggressive_timeouts*/ false, + /*override_use_adaptive_timeouts*/ false, settings.request_settings.long_request_timeout_ms)) {} diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 43618c64776..573fa744ce6 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -67,8 +67,8 @@ std::unique_ptr getClient( config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); client_configuration.wait_on_pool_size_limit = false; - client_configuration.s3_aggressive_timeouts = config.getUInt( - config_prefix + ".aggressive_timeouts", client_configuration.s3_aggressive_timeouts); + client_configuration.s3_use_adaptive_timeouts = config.getUInt( + config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); /* * Override proxy configuration for backwards compatibility with old configuration format. diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 12a0cb8f93c..90806852c1e 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -119,7 +119,7 @@ std::unique_ptr Client::create( } std::unique_ptr Client::clone( - std::optional override_aggressive_timeouts, + std::optional override_use_adaptive_timeouts, std::optional override_request_timeout_ms) const { PocoHTTPClientConfiguration new_configuration = client_configuration; @@ -127,8 +127,8 @@ std::unique_ptr Client::clone( if (override_request_timeout_ms.has_value()) new_configuration.requestTimeoutMs = *override_request_timeout_ms; - if (override_aggressive_timeouts.has_value()) - new_configuration.s3_aggressive_timeouts = *override_aggressive_timeouts; + if (override_use_adaptive_timeouts.has_value()) + new_configuration.s3_use_adaptive_timeouts = *override_use_adaptive_timeouts; return std::unique_ptr(new Client(*this, new_configuration)); } @@ -908,7 +908,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT s3_retry_attempts, enable_s3_requests_logging, for_disk_s3, - context->getGlobalContext()->getSettingsRef().s3_aggressive_timeouts, + context->getGlobalContext()->getSettingsRef().s3_use_adaptive_timeouts, get_request_throttler, put_request_throttler, error_report); diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 81ab3854d3d..be7235eb9f1 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -121,9 +121,9 @@ public: /// Create a client with adjusted settings: /// * override_request_timeout_ms is used to increase timeout for CompleteMultipartUploadRequest /// because it often sits idle for 10 seconds: https://github.com/ClickHouse/ClickHouse/pull/42321 - /// * s3_aggressive_timeouts is used to turn off s3_aggressive_timeouts feature for CompleteMultipartUploadRequest + /// * s3_use_adaptive_timeouts is used to turn off s3_use_adaptive_timeouts feature for CompleteMultipartUploadRequest std::unique_ptr clone( - std::optional override_aggressive_timeouts = std::nullopt, + std::optional override_use_adaptive_timeouts = std::nullopt, std::optional override_request_timeout_ms = std::nullopt) const; Client & operator=(const Client &) = delete; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 08ba04ee875..f783a886877 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -100,7 +100,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( unsigned int s3_retry_attempts_, bool enable_s3_requests_logging_, bool for_disk_s3_, - bool s3_aggressive_timeouts_, + bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, std::function error_report_) @@ -113,7 +113,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , for_disk_s3(for_disk_s3_) , get_request_throttler(get_request_throttler_) , put_request_throttler(put_request_throttler_) - , s3_aggressive_timeouts(s3_aggressive_timeouts_) + , s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_) , error_report(error_report_) { } @@ -160,7 +160,7 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config Poco::Timespan(client_configuration.http_keep_alive_timeout_ms * 1000))) /// flag indicating whether keep-alive is enabled is set to each session upon creation , remote_host_filter(client_configuration.remote_host_filter) , s3_max_redirects(client_configuration.s3_max_redirects) - , s3_aggressive_timeouts(client_configuration.s3_aggressive_timeouts) + , s3_use_adaptive_timeouts(client_configuration.s3_use_adaptive_timeouts) , enable_s3_requests_logging(client_configuration.enable_s3_requests_logging) , for_disk_s3(client_configuration.for_disk_s3) , get_request_throttler(client_configuration.get_request_throttler) @@ -295,7 +295,7 @@ UInt32 extractAttempt(const Aws::String & request_info) ConnectionTimeouts PocoHTTPClient::getTimeouts(Aws::Http::HttpRequest & request) const { - if (!s3_aggressive_timeouts) + if (!s3_use_adaptive_timeouts) return timeouts; const auto & request_info = request.GetHeaderValue(Aws::Http::SDK_REQUEST_HEADER); diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 6eeff431569..9ba5f4ffe64 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -55,7 +55,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration size_t http_connection_pool_size = 0; /// See PoolBase::BehaviourOnLimit bool wait_on_pool_size_limit = true; - bool s3_aggressive_timeouts = false; + bool s3_use_adaptive_timeouts = false; std::function error_report; @@ -70,7 +70,7 @@ private: unsigned int s3_retry_attempts, bool enable_s3_requests_logging_, bool for_disk_s3_, - bool s3_aggressive_timeouts_, + bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, std::function error_report_ @@ -182,7 +182,7 @@ protected: ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; - bool s3_aggressive_timeouts = false; + bool s3_use_adaptive_timeouts = false; bool enable_s3_requests_logging; bool for_disk_s3; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 63ed84680c9..231efb87e87 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1330,7 +1330,7 @@ void StorageS3::Configuration::connect(ContextPtr context) auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), }); - client_with_long_timeout = client->clone(std::nullopt, request_settings.long_request_timeout_ms); + client_with_long_timeout = client->clone(/*override_use_adaptive_timeouts*/ false, request_settings.long_request_timeout_ms); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) From bb0b6afe14319799028fbd8483b3bc4042e6f951 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 9 Nov 2023 13:12:38 +0100 Subject: [PATCH 299/813] reduce cuncurrent request number to the minio in test_storage_s3 --- tests/integration/test_storage_s3/configs/s3_retry.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_s3/configs/s3_retry.xml b/tests/integration/test_storage_s3/configs/s3_retry.xml index 581fc44c8d4..b7a7bbc8a9b 100644 --- a/tests/integration/test_storage_s3/configs/s3_retry.xml +++ b/tests/integration/test_storage_s3/configs/s3_retry.xml @@ -2,6 +2,7 @@ 10 + 5 From 76d11687a76ede0a0fd080fd76ff04da501e7af6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 9 Nov 2023 13:12:56 +0100 Subject: [PATCH 300/813] adjuct docs --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 306529c4b96..34ed85c773a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4821,7 +4821,7 @@ When set to `false` the metadata files are written with the previous format vers Default value: `false`. -## s3_aggressive_timeouts {#s3_aggressive_timeouts} +## s3_use_adaptive_timeouts {#s3_use_adaptive_timeouts} When set to `true` than for all s3 requests first two attempts are made with low send and receive timeouts. When set to `false` than all attempts are made with identical timeouts. From 8d36fd6e54cc66ab826d80fb6c4ec867fad4b731 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 9 Nov 2023 23:54:31 +0100 Subject: [PATCH 301/813] get rid off of client_with_long_timeout_ptr --- src/Backups/BackupIO_S3.cpp | 5 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 1 - src/Core/Settings.h | 2 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 62 ++++++------------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 14 +---- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 +- src/IO/ConnectionTimeouts.cpp | 53 ++++++++++++---- src/IO/ConnectionTimeouts.h | 3 +- src/IO/S3/Client.cpp | 14 +---- src/IO/S3/Client.h | 8 +-- src/IO/S3/PocoHTTPClient.cpp | 2 +- src/IO/S3/copyS3File.cpp | 26 +++----- src/IO/S3/copyS3File.h | 7 --- src/IO/S3/tests/gtest_aws_s3_client.cpp | 1 - src/IO/WriteBufferFromS3.cpp | 4 +- src/IO/WriteBufferFromS3.h | 3 - src/IO/tests/gtest_writebuffer_s3.cpp | 1 - src/Storages/StorageS3.cpp | 3 - src/Storages/StorageS3.h | 1 - src/Storages/StorageS3Settings.h | 3 +- 20 files changed, 83 insertions(+), 132 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 0b700665988..4f83158d07d 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -169,7 +169,6 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s blob_path.size(), mode); copyS3File( - client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -231,7 +230,6 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src { LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( - client, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], @@ -253,7 +251,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { - copyDataToS3File(create_read_buffer, start_pos, length, client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, s3_settings.request_settings, {}, + copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, s3_settings.request_settings, {}, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); } @@ -283,7 +281,6 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) { return std::make_unique( client, - client, // already has long timeout s3_uri.bucket, fs::path(s3_uri.key) / file_name, DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 302e05c8418..bedde0d7b39 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -148,7 +148,6 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapsh const auto create_writer = [&](const auto & key) { return WriteBufferFromS3( - s3_client->client, s3_client->client, s3_client->uri.bucket, key, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3f80c83ff5f..34547aded9c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -105,7 +105,7 @@ class IColumn; 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, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 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, 3000, "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_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_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. Only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 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) \ diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 8a46bfd59d1..75dd405f6aa 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -153,7 +153,7 @@ private: bool S3ObjectStorage::exists(const StoredObject & object) const { auto settings_ptr = s3_settings.get(); - return S3::objectExists(*clients.get()->client, bucket, object.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + return S3::objectExists(*client.get(), bucket, object.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); } std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT @@ -172,7 +172,7 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT (const std::string & path, size_t read_until_position) -> std::unique_ptr { return std::make_unique( - clients.get()->client, + client.get(), bucket, path, version_id, @@ -222,7 +222,7 @@ std::unique_ptr S3ObjectStorage::readObject( /// NOLINT { auto settings_ptr = s3_settings.get(); return std::make_unique( - clients.get()->client, + client.get(), bucket, object.remote_path, version_id, @@ -247,10 +247,8 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); - auto clients_ = clients.get(); return std::make_unique( - clients_->client, - clients_->client_with_long_timeout, + client.get(), bucket, object.remote_path, buf_size, @@ -264,15 +262,12 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix) const { auto settings_ptr = s3_settings.get(); - auto client_ptr = clients.get()->client; - - return std::make_shared(bucket, path_prefix, client_ptr, settings_ptr->list_object_keys_size); + return std::make_shared(bucket, path_prefix, client.get(), settings_ptr->list_object_keys_size); } void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto settings_ptr = s3_settings.get(); - auto client_ptr = clients.get()->client; S3::ListObjectsV2Request request; request.SetBucket(bucket); @@ -287,7 +282,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet { ProfileEvents::increment(ProfileEvents::S3ListObjects); ProfileEvents::increment(ProfileEvents::DiskS3ListObjects); - outcome = client_ptr->ListObjectsV2(request); + outcome = client.get()->ListObjectsV2(request); throwIfError(outcome); auto result = outcome.GetResult(); @@ -318,14 +313,12 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exists) { - auto client_ptr = clients.get()->client; - ProfileEvents::increment(ProfileEvents::S3DeleteObjects); ProfileEvents::increment(ProfileEvents::DiskS3DeleteObjects); S3::DeleteObjectRequest request; request.SetBucket(bucket); request.SetKey(object.remote_path); - auto outcome = client_ptr->DeleteObject(request); + auto outcome = client.get()->DeleteObject(request); throwIfUnexpectedError(outcome, if_exists); @@ -344,7 +337,6 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e } else { - auto client_ptr = clients.get()->client; auto settings_ptr = s3_settings.get(); size_t chunk_size_limit = settings_ptr->objects_chunk_size_to_delete; @@ -373,7 +365,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e S3::DeleteObjectsRequest request; request.SetBucket(bucket); request.SetDelete(delkeys); - auto outcome = client_ptr->DeleteObjects(request); + auto outcome = client.get()->DeleteObjects(request); throwIfUnexpectedError(outcome, if_exists); @@ -405,7 +397,7 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*clients.get()->client, bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); + auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) return {}; @@ -421,7 +413,7 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*clients.get()->client, bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true); + auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true); ObjectMetadata result; result.size_bytes = object_info.size; @@ -442,12 +434,12 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT /// Shortcut for S3 if (auto * dest_s3 = dynamic_cast(&object_storage_to); dest_s3 != nullptr) { - auto clients_ = clients.get(); + auto client_ = client.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*client_, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(clients_->client, - clients_->client_with_long_timeout, + copyS3File( + client.get(), bucket, object_from.remote_path, 0, @@ -471,12 +463,11 @@ void S3ObjectStorage::copyObject( // NOLINT const WriteSettings &, std::optional object_to_attributes) { - auto clients_ = clients.get(); + auto client_ = client.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*client_, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(clients_->client, - clients_->client_with_long_timeout, + copyS3File(client_, bucket, object_from.remote_path, 0, @@ -497,31 +488,25 @@ void S3ObjectStorage::setNewSettings(std::unique_ptr && void S3ObjectStorage::shutdown() { - auto clients_ptr = clients.get(); /// This call stops any next retry attempts for ongoing S3 requests. /// If S3 request is failed and the method below is executed S3 client immediately returns the last failed S3 request outcome. /// If S3 is healthy nothing wrong will be happened and S3 requests will be processed in a regular way without errors. /// This should significantly speed up shutdown process if S3 is unhealthy. - const_cast(*clients_ptr->client).DisableRequestProcessing(); - const_cast(*clients_ptr->client_with_long_timeout).DisableRequestProcessing(); + const_cast(*client.get()).DisableRequestProcessing(); } void S3ObjectStorage::startup() { - auto clients_ptr = clients.get(); - /// Need to be enabled if it was disabled during shutdown() call. - const_cast(*clients_ptr->client).EnableRequestProcessing(); - const_cast(*clients_ptr->client_with_long_timeout).EnableRequestProcessing(); + const_cast(*client.get()).EnableRequestProcessing(); } void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, 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); - auto new_clients = std::make_unique(std::move(new_client), *new_s3_settings); s3_settings.set(std::move(new_s3_settings)); - clients.set(std::move(new_clients)); + client.set(std::move(new_client)); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( @@ -536,13 +521,6 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( endpoint, object_key_prefix); } -S3ObjectStorage::Clients::Clients(std::shared_ptr client_, const S3ObjectStorageSettings & settings) - : client(std::move(client_)) - , client_with_long_timeout(client->clone( - /*override_use_adaptive_timeouts*/ false, - settings.request_settings.long_request_timeout_ms)) -{} - ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string &) const { /// Path to store the new S3 object. diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 37e491e21dc..7d14482311f 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -39,16 +39,6 @@ struct S3ObjectStorageSettings class S3ObjectStorage : public IObjectStorage { -public: - struct Clients - { - std::shared_ptr client; - std::shared_ptr client_with_long_timeout; - - Clients() = default; - Clients(std::shared_ptr client, const S3ObjectStorageSettings & settings); - }; - private: friend class S3PlainObjectStorage; @@ -63,7 +53,7 @@ private: String object_key_prefix_) : bucket(std::move(bucket_)) , object_key_prefix(std::move(object_key_prefix_)) - , clients(std::make_unique(std::move(client_), *s3_settings_)) + , client(std::move(client_)) , s3_settings(std::move(s3_settings_)) , s3_capabilities(s3_capabilities_) , version_id(std::move(version_id_)) @@ -185,7 +175,7 @@ private: String object_key_prefix; - MultiVersion clients; + MultiVersion client; MultiVersion s3_settings; S3Capabilities s3_capabilities; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 573fa744ce6..b0384daab2d 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -60,7 +60,7 @@ std::unique_ptr getClient( uri.uri.getScheme()); client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 1000); - client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 3000); + client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 30000); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100); client_configuration.endpointOverride = uri.endpoint; client_configuration.http_keep_alive_timeout_ms = config.getUInt( diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index a9eebb1a755..90406dcf409 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -133,22 +133,51 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings settings.http_receive_timeout); } -ConnectionTimeouts ConnectionTimeouts::aggressiveTimeouts(UInt32 attempt) const +ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(Aws::Http::HttpMethod method, UInt32 attempt) const { + constexpr size_t first_method_index = size_t(Aws::Http::HttpMethod::HTTP_GET); + constexpr size_t last_method_index = size_t(Aws::Http::HttpMethod::HTTP_PATCH); + constexpr size_t methods_count = last_method_index - first_method_index + 1; + + /// HTTP_POST is used for CompleteMultipartUpload requests. + /// These requests need longer timeout, especially when minio is used + /// The same assumption are made for HTTP_DELETE, HTTP_PATCH + /// That requests are more heavy that HTTP_GET, HTTP_HEAD, HTTP_PUT + + static const UInt32 first_attempt_send_receive_timeouts_ms[methods_count][2] = { + /*HTTP_GET*/ {200, 200}, + /*HTTP_POST*/ {200, 30000}, + /*HTTP_DELETE*/ {200, 1000}, + /*HTTP_PUT*/ {200, 200}, + /*HTTP_HEAD*/ {200, 200}, + /*HTTP_PATCH*/ {200, 1000}, + }; + + static const UInt32 second_attempt_send_receive_timeouts_ms[methods_count][2] = { + /*HTTP_GET*/ {1000, 1000}, + /*HTTP_POST*/ {1000, 30000}, + /*HTTP_DELETE*/ {1000, 10000}, + /*HTTP_PUT*/ {1000, 1000}, + /*HTTP_HEAD*/ {1000, 1000}, + /*HTTP_PATCH*/ {1000, 10000}, + }; + + static_assert(methods_count == 6); + static_assert(sizeof(first_attempt_send_receive_timeouts_ms) == sizeof(second_attempt_send_receive_timeouts_ms)); + static_assert(sizeof(first_attempt_send_receive_timeouts_ms) == methods_count * sizeof(UInt32) * 2); + auto aggressive = *this; + if (attempt > 2) + return aggressive; + + auto timeout_map = first_attempt_send_receive_timeouts_ms; if (attempt == 2) - { - auto one_second = Poco::Timespan(1, 0); - aggressive.send_timeout = saturate(one_second, send_timeout); - aggressive.receive_timeout = saturate(one_second, receive_timeout); - } - else if (attempt == 1) - { - auto two_hundred_ms = Poco::Timespan(0, 200 * 1000); - aggressive.send_timeout = saturate(two_hundred_ms, send_timeout); - aggressive.receive_timeout = saturate(two_hundred_ms, receive_timeout); - } + timeout_map = second_attempt_send_receive_timeouts_ms; + + const size_t method_index = size_t(method) - first_method_index; + aggressive.send_timeout = saturate(Poco::Timespan(timeout_map[method_index][0]), send_timeout); + aggressive.receive_timeout = saturate(Poco::Timespan(timeout_map[method_index][1]), receive_timeout); return aggressive; } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 17ee1907d89..0ef133c8378 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -4,6 +4,7 @@ #include #include +#include namespace DB { @@ -68,7 +69,7 @@ struct ConnectionTimeouts static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings); static ConnectionTimeouts getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout); - ConnectionTimeouts aggressiveTimeouts(UInt32 attempt) const; + ConnectionTimeouts getAdaptiveTimeouts(Aws::Http::HttpMethod method, UInt32 attempt) const; }; } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 90806852c1e..4630e68fbb6 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -118,19 +118,9 @@ std::unique_ptr Client::create( new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, use_virtual_addressing)); } -std::unique_ptr Client::clone( - std::optional override_use_adaptive_timeouts, - std::optional override_request_timeout_ms) const +std::unique_ptr Client::clone() const { - PocoHTTPClientConfiguration new_configuration = client_configuration; - - if (override_request_timeout_ms.has_value()) - new_configuration.requestTimeoutMs = *override_request_timeout_ms; - - if (override_use_adaptive_timeouts.has_value()) - new_configuration.s3_use_adaptive_timeouts = *override_use_adaptive_timeouts; - - return std::unique_ptr(new Client(*this, new_configuration)); + return std::unique_ptr(new Client(*this, client_configuration)); } namespace diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index be7235eb9f1..5ad57a9d827 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -118,13 +118,7 @@ public: Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, bool use_virtual_addressing); - /// Create a client with adjusted settings: - /// * override_request_timeout_ms is used to increase timeout for CompleteMultipartUploadRequest - /// because it often sits idle for 10 seconds: https://github.com/ClickHouse/ClickHouse/pull/42321 - /// * s3_use_adaptive_timeouts is used to turn off s3_use_adaptive_timeouts feature for CompleteMultipartUploadRequest - std::unique_ptr clone( - std::optional override_use_adaptive_timeouts = std::nullopt, - std::optional override_request_timeout_ms = std::nullopt) const; + std::unique_ptr clone() const; Client & operator=(const Client &) = delete; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index f783a886877..b26c36f8029 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -300,7 +300,7 @@ ConnectionTimeouts PocoHTTPClient::getTimeouts(Aws::Http::HttpRequest & request) const auto & request_info = request.GetHeaderValue(Aws::Http::SDK_REQUEST_HEADER); auto attempt = extractAttempt(request_info); - return timeouts.aggressiveTimeouts(attempt); + return timeouts.getAdaptiveTimeouts(request.GetMethod(), attempt); } void PocoHTTPClient::makeRequestInternal( diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index a16a1a41505..30da1c580c1 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -53,7 +53,6 @@ namespace public: UploadHelper( const std::shared_ptr & client_ptr_, - const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, @@ -62,7 +61,6 @@ namespace bool for_disk_s3_, const Poco::Logger * log_) : client_ptr(client_ptr_) - , client_with_long_timeout_ptr(client_with_long_timeout_ptr_) , dest_bucket(dest_bucket_) , dest_key(dest_key_) , request_settings(request_settings_) @@ -78,7 +76,6 @@ namespace protected: std::shared_ptr client_ptr; - std::shared_ptr client_with_long_timeout_ptr; const String & dest_bucket; const String & dest_key; const S3Settings::RequestSettings & request_settings; @@ -179,7 +176,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); - auto outcome = client_with_long_timeout_ptr->CompleteMultipartUpload(request); + auto outcome = client_ptr->CompleteMultipartUpload(request); if (outcome.IsSuccess()) { @@ -433,14 +430,13 @@ namespace size_t offset_, size_t size_, const std::shared_ptr & client_ptr_, - const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) + : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) , create_read_buffer(create_read_buffer_) , offset(offset_) , size(size_) @@ -602,7 +598,6 @@ namespace public: CopyFileHelper( const std::shared_ptr & client_ptr_, - const std::shared_ptr & client_with_long_timeout_ptr_, const String & src_bucket_, const String & src_key_, size_t src_offset_, @@ -614,7 +609,7 @@ namespace const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) + : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) , src_bucket(src_bucket_) , src_key(src_key_) , offset(src_offset_) @@ -677,7 +672,7 @@ namespace /// 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 request.SetContentType("binary/octet-stream"); - client_with_long_timeout_ptr->setKMSHeaders(request); + client_ptr->setKMSHeaders(request); } void processCopyRequest(const S3::CopyObjectRequest & request) @@ -689,7 +684,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CopyObject); - auto outcome = client_with_long_timeout_ptr->CopyObject(request); + auto outcome = client_ptr->CopyObject(request); if (outcome.IsSuccess()) { LOG_TRACE( @@ -714,7 +709,6 @@ namespace offset, size, client_ptr, - client_with_long_timeout_ptr, dest_bucket, dest_key, request_settings, @@ -788,7 +782,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy); - auto outcome = client_with_long_timeout_ptr->UploadPartCopy(req); + auto outcome = client_ptr->UploadPartCopy(req); if (!outcome.IsSuccess()) { abortMultipartUpload(); @@ -806,7 +800,6 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, - const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, @@ -814,14 +807,13 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule, bool for_disk_s3) { - CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } void copyS3File( const std::shared_ptr & s3_client, - const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -836,7 +828,7 @@ void copyS3File( { if (settings.allow_native_copy) { - CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, read_settings, object_metadata, schedule, for_disk_s3}; + CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, read_settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } else @@ -845,7 +837,7 @@ void copyS3File( { return std::make_unique(s3_client, src_bucket, src_key, "", settings, read_settings); }; - copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 1bcbfd7735e..33e22fdfba2 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -27,15 +27,9 @@ using CreateReadBuffer = std::function()>; /// because it is a known issue, it is fallbacks to read-write copy /// (copyDataToS3File()). /// -/// s3_client_with_long_timeout (may be equal to s3_client) is used for native copy and -/// CompleteMultipartUpload requests. These requests need longer timeout because S3 servers often -/// block on them for multiple seconds without sending or receiving data from us (maybe the servers -/// are copying data internally, or maybe throttling, idk). -/// /// read_settings - is used for throttling in case of native copy is not possible void copyS3File( const std::shared_ptr & s3_client, - const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -58,7 +52,6 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, - const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index c42f14e9a53..d4b9a017398 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -91,7 +91,6 @@ void doWriteRequest(std::shared_ptr client, const DB::S3:: DB::S3Settings::RequestSettings request_settings; request_settings.max_unexpected_write_error_retries = max_unexpected_write_error_retries; DB::WriteBufferFromS3 write_buffer( - client, client, uri.bucket, uri.key, diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index e1b9c17efe9..62d0c80f1f2 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -77,7 +77,6 @@ struct WriteBufferFromS3::PartData WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, - std::shared_ptr client_with_long_timeout_ptr_, const String & bucket_, const String & key_, size_t buf_size_, @@ -92,7 +91,6 @@ WriteBufferFromS3::WriteBufferFromS3( , upload_settings(request_settings.getUploadSettings()) , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) - , client_with_long_timeout_ptr(std::move(client_with_long_timeout_ptr_)) , object_metadata(std::move(object_metadata_)) , buffer_allocation_policy(ChooseBufferPolicy(upload_settings)) , task_tracker( @@ -566,7 +564,7 @@ void WriteBufferFromS3::completeMultipartUpload() ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); Stopwatch watch; - auto outcome = client_with_long_timeout_ptr->CompleteMultipartUpload(req); + auto outcome = client_ptr->CompleteMultipartUpload(req); watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 95148c49779..590342cc997 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -30,8 +30,6 @@ class WriteBufferFromS3 final : public WriteBufferFromFileBase public: WriteBufferFromS3( std::shared_ptr client_ptr_, - /// for CompleteMultipartUploadRequest, because it blocks on recv() for a few seconds on big uploads - std::shared_ptr client_with_long_timeout_ptr_, const String & bucket_, const String & key_, size_t buf_size_, @@ -90,7 +88,6 @@ private: const S3Settings::RequestSettings::PartUploadSettings & upload_settings; const WriteSettings write_settings; const std::shared_ptr client_ptr; - const std::shared_ptr client_with_long_timeout_ptr; const std::optional> object_metadata; Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 21bdd9a6f26..c82f97f8b20 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -549,7 +549,6 @@ public: getAsyncPolicy().setAutoExecute(false); return std::make_unique( - client, client, bucket, file_name, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 231efb87e87..b0cd40a2e05 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -824,7 +824,6 @@ public: write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique( configuration_.client, - configuration_.client_with_long_timeout, bucket, key, DBMS_DEFAULT_BUFFER_SIZE, @@ -1329,8 +1328,6 @@ void StorageS3::Configuration::connect(ContextPtr context) 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)), }); - - client_with_long_timeout = client->clone(/*override_use_adaptive_timeouts*/ false, request_settings.long_request_timeout_ms); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 3330ac6c210..3f35c578e19 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -311,7 +311,6 @@ public: HTTPHeaderEntries headers_from_ast; std::shared_ptr client; - std::shared_ptr client_with_long_timeout; std::vector keys; }; diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index e3d577ca0b3..728972c948c 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -69,8 +69,7 @@ struct S3Settings ThrottlerPtr get_request_throttler; ThrottlerPtr put_request_throttler; size_t retry_attempts = 10; - size_t request_timeout_ms = 3000; - size_t long_request_timeout_ms = 30000; // TODO: Take this from config like request_timeout_ms + size_t request_timeout_ms = 30000; bool allow_native_copy = true; bool throw_on_zero_files_match = false; From 3075bd97450d42f00320b18c1b177fd700a19bec Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 10 Nov 2023 15:15:24 +0100 Subject: [PATCH 302/813] track clickhouse high level retries --- base/poco/Net/src/HTTPSession.cpp | 4 +- src/IO/ConnectionTimeouts.cpp | 101 ++++++++++++------ src/IO/ConnectionTimeouts.h | 3 +- src/IO/HTTPCommon.cpp | 12 +-- src/IO/HTTPCommon.h | 2 + src/IO/ReadBufferFromS3.cpp | 24 +++-- src/IO/ReadBufferFromS3.h | 4 +- src/IO/S3/PocoHTTPClient.cpp | 89 ++++++++------- src/IO/S3/PocoHTTPClient.h | 2 +- src/IO/S3/tests/gtest_aws_s3_client.cpp | 2 + .../configs/inf_s3_retries.xml | 2 +- .../configs/s3_retries.xml | 2 +- .../configs/storage_conf.xml | 1 + .../test_checking_s3_blobs_paranoid/test.py | 16 +-- .../test_storage_s3/configs/s3_retry.xml | 1 + .../s3_mocks/unstable_server.py | 17 ++- tests/integration/test_storage_s3/test.py | 9 ++ 17 files changed, 181 insertions(+), 110 deletions(-) diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index d30f5590280..9ebbd7d04cd 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -95,7 +95,7 @@ void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco { _connectionTimeout = connectionTimeout; - if (_sendTimeout != sendTimeout) + if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) { _sendTimeout = sendTimeout; @@ -103,7 +103,7 @@ void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco _socket.setSendTimeout(_sendTimeout); } - if (_receiveTimeout != receiveTimeout) + if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) { _receiveTimeout = receiveTimeout; diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 90406dcf409..970afc75ec3 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -133,51 +133,84 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings settings.http_receive_timeout); } -ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(Aws::Http::HttpMethod method, UInt32 attempt) const +class SendReceiveTimeoutsForFirstAttempt { - constexpr size_t first_method_index = size_t(Aws::Http::HttpMethod::HTTP_GET); - constexpr size_t last_method_index = size_t(Aws::Http::HttpMethod::HTTP_PATCH); - constexpr size_t methods_count = last_method_index - first_method_index + 1; +private: + static constexpr size_t known_methods_count = 6; + using KnownMethodsArray = std::array; + static const KnownMethodsArray known_methods; - /// HTTP_POST is used for CompleteMultipartUpload requests. - /// These requests need longer timeout, especially when minio is used + /// HTTP_POST is used for CompleteMultipartUpload requests. Its latency could be high. + /// These requests need longer timeout, especially when minio is used. /// The same assumption are made for HTTP_DELETE, HTTP_PATCH /// That requests are more heavy that HTTP_GET, HTTP_HEAD, HTTP_PUT - static const UInt32 first_attempt_send_receive_timeouts_ms[methods_count][2] = { - /*HTTP_GET*/ {200, 200}, - /*HTTP_POST*/ {200, 30000}, - /*HTTP_DELETE*/ {200, 1000}, - /*HTTP_PUT*/ {200, 200}, - /*HTTP_HEAD*/ {200, 200}, - /*HTTP_PATCH*/ {200, 1000}, + static constexpr Poco::Timestamp::TimeDiff first_byte_ms[known_methods_count][2] = + { + /* GET */ {200, 200}, + /* POST */ {200, 200}, + /* DELETE */ {200, 200}, + /* PUT */ {200, 200}, + /* HEAD */ {200, 200}, + /* PATCH */ {200, 200}, }; - static const UInt32 second_attempt_send_receive_timeouts_ms[methods_count][2] = { - /*HTTP_GET*/ {1000, 1000}, - /*HTTP_POST*/ {1000, 30000}, - /*HTTP_DELETE*/ {1000, 10000}, - /*HTTP_PUT*/ {1000, 1000}, - /*HTTP_HEAD*/ {1000, 1000}, - /*HTTP_PATCH*/ {1000, 10000}, + static constexpr Poco::Timestamp::TimeDiff rest_bytes_ms[known_methods_count][2] = + { + /* GET */ {500, 500}, + /* POST */ {1000, 30000}, + /* DELETE */ {1000, 10000}, + /* PUT */ {1000, 3000}, + /* HEAD */ {500, 500}, + /* PATCH */ {1000, 10000}, }; - static_assert(methods_count == 6); - static_assert(sizeof(first_attempt_send_receive_timeouts_ms) == sizeof(second_attempt_send_receive_timeouts_ms)); - static_assert(sizeof(first_attempt_send_receive_timeouts_ms) == methods_count * sizeof(UInt32) * 2); + static_assert(sizeof(first_byte_ms) == sizeof(rest_bytes_ms)); + static_assert(sizeof(first_byte_ms) == known_methods_count * sizeof(Poco::Timestamp::TimeDiff) * 2); + + static size_t getMethodIndex(const String & method) + { + KnownMethodsArray::const_iterator it = std::find(known_methods.begin(), known_methods.end(), method); + chassert(it != known_methods.end()); + if (it == known_methods.end()) + return 0; + return std::distance(known_methods.begin(), it); + } + +public: + static std::pair getSendReceiveTimeout(const String & method, bool first_byte) + { + auto idx = getMethodIndex(method); + + if (first_byte) + return std::make_pair( + Poco::Timespan(first_byte_ms[idx][0] * 1000), + Poco::Timespan(first_byte_ms[idx][1] * 1000) + ); + + return std::make_pair( + Poco::Timespan(rest_bytes_ms[idx][0] * 1000), + Poco::Timespan(rest_bytes_ms[idx][1] * 1000) + ); + } +}; + +const SendReceiveTimeoutsForFirstAttempt::KnownMethodsArray SendReceiveTimeoutsForFirstAttempt::known_methods = +{ + "GET", "POST", "DELETE", "PUT", "HEAD", "PATCH" +}; + + +ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(const String & method, bool first_attempt, bool first_byte) const +{ + if (!first_attempt) + return *this; + + auto [send, recv] = SendReceiveTimeoutsForFirstAttempt::getSendReceiveTimeout(method, first_byte); auto aggressive = *this; - - if (attempt > 2) - return aggressive; - - auto timeout_map = first_attempt_send_receive_timeouts_ms; - if (attempt == 2) - timeout_map = second_attempt_send_receive_timeouts_ms; - - const size_t method_index = size_t(method) - first_method_index; - aggressive.send_timeout = saturate(Poco::Timespan(timeout_map[method_index][0]), send_timeout); - aggressive.receive_timeout = saturate(Poco::Timespan(timeout_map[method_index][1]), receive_timeout); + aggressive.send_timeout = saturate(send, send_timeout); + aggressive.receive_timeout = saturate(recv, receive_timeout); return aggressive; } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 0ef133c8378..aabebdb836d 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -4,7 +4,6 @@ #include #include -#include namespace DB { @@ -69,7 +68,7 @@ struct ConnectionTimeouts static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings); static ConnectionTimeouts getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout); - ConnectionTimeouts getAdaptiveTimeouts(Aws::Http::HttpMethod method, UInt32 attempt) const; + ConnectionTimeouts getAdaptiveTimeouts(const String & method, bool first_attempt, bool first_byte) const; }; } diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 65ffa51a466..cce394c67c9 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -50,12 +50,6 @@ namespace ErrorCodes namespace { - void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) - { - session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); - session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); - } - Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const ProxyConfiguration & proxy_configuration) { Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; @@ -359,6 +353,12 @@ namespace }; } +void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) +{ + session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); + session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); +} + void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout) { if (!response.getKeepAlive()) diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index de62b5d5c16..c9968fc6915 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -113,4 +113,6 @@ std::istream * receiveResponse( void assertResponseIsOk( const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false); + +void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index f19978ccb47..c9c9319c44c 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -167,9 +167,9 @@ bool ReadBufferFromS3::nextImpl() } size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; !next_result; ++attempt) + for (size_t attempt = 1; !next_result; ++attempt) { - bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + bool last_attempt = attempt >= request_settings.max_single_read_retries; ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); @@ -177,7 +177,7 @@ bool ReadBufferFromS3::nextImpl() { if (!impl) { - impl = initialize(); + impl = initialize(attempt); if (use_external_buffer) { @@ -232,9 +232,9 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons { size_t initial_n = n; size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; n > 0; ++attempt) + for (size_t attempt = 1; n > 0; ++attempt) { - bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + bool last_attempt = attempt >= request_settings.max_single_read_retries; size_t bytes_copied = 0; ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); @@ -266,7 +266,7 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons try { - result = sendRequest(range_begin, range_begin + n - 1); + result = sendRequest(attempt, range_begin, range_begin + n - 1); std::istream & istr = result->GetBody(); copyFromIStreamWithProgressCallback(istr, to, n, progress_callback, &bytes_copied); @@ -304,8 +304,8 @@ bool ReadBufferFromS3::processException(Poco::Exception & e, size_t read_offset, LOG_DEBUG( log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, " - "Attempt: {}, Message: {}", - bucket, key, version_id.empty() ? "Latest" : version_id, read_offset, attempt, e.message()); + "Attempt: {}/{}, Message: {}", + bucket, key, version_id.empty() ? "Latest" : version_id, read_offset, attempt, request_settings.max_single_read_retries, e.message()); if (auto * s3_exception = dynamic_cast(&e)) @@ -463,7 +463,7 @@ ReadBufferFromS3::~ReadBufferFromS3() } } -std::unique_ptr ReadBufferFromS3::initialize() +std::unique_ptr ReadBufferFromS3::initialize(size_t attempt) { resetSessionIfNeeded(readAllRangeSuccessfully(), read_result); read_all_range_successfully = false; @@ -475,13 +475,13 @@ std::unique_ptr ReadBufferFromS3::initialize() if (read_until_position && offset >= read_until_position) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); - read_result = sendRequest(offset, read_until_position ? std::make_optional(read_until_position - 1) : std::nullopt); + read_result = sendRequest(attempt, offset, read_until_position ? std::make_optional(read_until_position - 1) : std::nullopt); size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; return std::make_unique(read_result->GetBody(), buffer_size); } -Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin, std::optional range_end_incl) const +Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, size_t range_begin, std::optional range_end_incl) const { S3::GetObjectRequest req; req.SetBucket(bucket); @@ -489,6 +489,8 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin if (!version_id.empty()) req.SetVersionId(version_id); + req.SetAdditionalCustomHeaderValue("clickhouse-request", fmt::format("attempt={}", attempt)); + if (range_end_incl) { req.SetRange(fmt::format("bytes={}-{}", range_begin, *range_end_incl)); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 0835e52a5b2..101e25f8b43 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -79,7 +79,7 @@ public: bool supportsReadAt() override { return true; } private: - std::unique_ptr initialize(); + std::unique_ptr initialize(size_t attempt); /// If true, if we destroy impl now, no work was wasted. Just for metrics. bool atEndOfRequestedRangeGuess(); @@ -88,7 +88,7 @@ private: /// Returns true if the error looks retriable. bool processException(Poco::Exception & e, size_t read_offset, size_t attempt) const; - Aws::S3::Model::GetObjectResult sendRequest(size_t range_begin, std::optional range_end_incl) const; + Aws::S3::Model::GetObjectResult sendRequest(size_t attempt, size_t range_begin, std::optional range_end_incl) const; bool readAllRangeSuccessfully() const; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index b26c36f8029..904e2324145 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -272,35 +272,36 @@ void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricT ProfileEvents::increment(disk_s3_events_map[static_cast(type)][static_cast(kind)], amount); } -UInt32 extractAttempt(const Aws::String & request_info) +String extractAttemptFromInfo(const Aws::String & request_info) { static auto key = Aws::String("attempt="); auto key_begin = request_info.find(key, 0); if (key_begin == Aws::String::npos) - return 1; + return "1"; auto val_begin = key_begin + key.size(); auto val_end = request_info.find(';', val_begin); if (val_end == Aws::String::npos) val_end = request_info.size(); - Aws::String value = request_info.substr(val_begin, val_end-val_begin); - - UInt32 attempt = 1; - ReadBufferFromString buf(value); - readIntText(attempt, buf); - return attempt; + return request_info.substr(val_begin, val_end-val_begin); } -ConnectionTimeouts PocoHTTPClient::getTimeouts(Aws::Http::HttpRequest & request) const +String getOrEmpty(const Aws::Http::HeaderValueCollection & map, const String & key) +{ + auto it = map.find(key); + if (it == map.end()) + return {}; + return it->second; +} + +ConnectionTimeouts PocoHTTPClient::getTimeouts(const String & method, bool first_attempt, bool first_byte) const { if (!s3_use_adaptive_timeouts) return timeouts; - const auto & request_info = request.GetHeaderValue(Aws::Http::SDK_REQUEST_HEADER); - auto attempt = extractAttempt(request_info); - return timeouts.getAdaptiveTimeouts(request.GetMethod(), attempt); + return timeouts.getAdaptiveTimeouts(method, first_attempt, first_byte); } void PocoHTTPClient::makeRequestInternal( @@ -317,6 +318,25 @@ void PocoHTTPClient::makeRequestInternal( makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); } +String getMethod(const Aws::Http::HttpRequest & request) +{ + switch (request.GetMethod()) + { + case Aws::Http::HttpMethod::HTTP_GET: + return Poco::Net::HTTPRequest::HTTP_GET; + case Aws::Http::HttpMethod::HTTP_POST: + return Poco::Net::HTTPRequest::HTTP_POST; + case Aws::Http::HttpMethod::HTTP_DELETE: + return Poco::Net::HTTPRequest::HTTP_DELETE; + case Aws::Http::HttpMethod::HTTP_PUT: + return Poco::Net::HTTPRequest::HTTP_PUT; + case Aws::Http::HttpMethod::HTTP_HEAD: + return Poco::Net::HTTPRequest::HTTP_HEAD; + case Aws::Http::HttpMethod::HTTP_PATCH: + return Poco::Net::HTTPRequest::HTTP_PATCH; + } +} + template void PocoHTTPClient::makeRequestInternalImpl( Aws::Http::HttpRequest & request, @@ -330,9 +350,14 @@ void PocoHTTPClient::makeRequestInternalImpl( Poco::Logger * log = &Poco::Logger::get("AWSClient"); auto uri = request.GetUri().GetURIString(); + auto method = getMethod(request); + + auto sdk_attempt = extractAttemptFromInfo(getOrEmpty(request.GetHeaders(), Aws::Http::SDK_REQUEST_HEADER)); + auto ch_attempt = extractAttemptFromInfo(getOrEmpty(request.GetHeaders(), "clickhouse-request")); + bool first_attempt = ch_attempt == "1" && sdk_attempt == "1"; if (enable_s3_requests_logging) - LOG_TEST(log, "Make request to: {}", uri); + LOG_TEST(log, "Make request to: {}, aws sdk attempt: {}, clickhouse attempt: {}", uri, sdk_attempt, ch_attempt); switch (request.GetMethod()) { @@ -383,17 +408,17 @@ void PocoHTTPClient::makeRequestInternalImpl( /// This can lead to request signature difference on S3 side. if constexpr (pooled) session = makePooledHTTPSession( - target_uri, getTimeouts(request), http_connection_pool_size, wait_on_pool_size_limit, proxy_configuration); + target_uri, getTimeouts(method, first_attempt), http_connection_pool_size, wait_on_pool_size_limit, proxy_configuration); else - session = makeHTTPSession(target_uri, getTimeouts(request), proxy_configuration); + session = makeHTTPSession(target_uri, getTimeouts(method, first_attempt), proxy_configuration); } else { if constexpr (pooled) session = makePooledHTTPSession( - target_uri, getTimeouts(request), http_connection_pool_size, wait_on_pool_size_limit); + target_uri, getTimeouts(method, first_attempt), http_connection_pool_size, wait_on_pool_size_limit); else - session = makeHTTPSession(target_uri, getTimeouts(request)); + session = makeHTTPSession(target_uri, getTimeouts(method, first_attempt)); } /// In case of error this address will be written to logs @@ -427,28 +452,7 @@ void PocoHTTPClient::makeRequestInternalImpl( path_and_query = "/"; poco_request.setURI(path_and_query); - - switch (request.GetMethod()) - { - case Aws::Http::HttpMethod::HTTP_GET: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_GET); - break; - case Aws::Http::HttpMethod::HTTP_POST: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_POST); - break; - case Aws::Http::HttpMethod::HTTP_DELETE: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); - break; - case Aws::Http::HttpMethod::HTTP_PUT: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); - break; - case Aws::Http::HttpMethod::HTTP_HEAD: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); - break; - case Aws::Http::HttpMethod::HTTP_PATCH: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); - break; - } + poco_request.setMethod(method); /// Headers coming from SDK are lower-cased. for (const auto & [header_name, header_value] : request.GetHeaders()) @@ -473,6 +477,7 @@ void PocoHTTPClient::makeRequestInternalImpl( request.GetContentBody()->clear(); request.GetContentBody()->seekg(0); + setTimeouts(*session, getTimeouts(method, first_attempt, /*first_byte*/ false)); auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); if (enable_s3_requests_logging) LOG_TEST(log, "Written {} bytes to request body", size); @@ -482,6 +487,8 @@ void PocoHTTPClient::makeRequestInternalImpl( LOG_TEST(log, "Receiving response..."); auto & response_body_stream = session->receiveResponse(poco_response); + setTimeouts(*session, getTimeouts(method, first_attempt, /*first_byte*/ false)); + watch.stop(); addMetric(request, S3MetricType::Microseconds, watch.elapsedMicroseconds()); @@ -533,6 +540,7 @@ void PocoHTTPClient::makeRequestInternalImpl( /// Request is successful but for some special requests we can have actual error message in body if (status_code >= SUCCESS_RESPONSE_MIN && status_code <= SUCCESS_RESPONSE_MAX && checkRequestCanReturn2xxAndErrorInBody(request)) { + /// reading the full response std::string response_string((std::istreambuf_iterator(response_body_stream)), std::istreambuf_iterator()); @@ -547,7 +555,6 @@ void PocoHTTPClient::makeRequestInternalImpl( addMetric(request, S3MetricType::Errors); if (error_report) error_report(proxy_configuration); - } /// Set response from string @@ -566,6 +573,8 @@ void PocoHTTPClient::makeRequestInternalImpl( if (status_code >= 500 && error_report) error_report(proxy_configuration); } + + /// expose stream, after that client reads data from that stream without built-in retries response->SetResponseBody(response_body_stream, session); } diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 9ba5f4ffe64..14c4fec5dd7 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -171,7 +171,7 @@ private: Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; - ConnectionTimeouts getTimeouts(Aws::Http::HttpRequest & request) const; + ConnectionTimeouts getTimeouts(const String & method, bool first_attempt, bool first_byte = true) const; protected: static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index d4b9a017398..bff9ca6fa7b 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -170,6 +170,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeadersRead) "authorization: ... SignedHeaders=" "amz-sdk-invocation-id;" "amz-sdk-request;" + "clickhouse-request;" "content-type;" "host;" "x-amz-api-version;" @@ -215,6 +216,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersRead) "authorization: ... SignedHeaders=" "amz-sdk-invocation-id;" "amz-sdk-request;" + "clickhouse-request;" "content-type;" "host;" "x-amz-api-version;" diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml index 5f0860ac120..4210c13b727 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml @@ -4,7 +4,7 @@ 1000000 - 1 + 1 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml index f215a89f613..95a313ea4f2 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml @@ -4,7 +4,7 @@ 5 - 0 + 0 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml index 264c411b59b..7b1f503ed55 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -18,6 +18,7 @@ http://resolver:8083/root/data/ minio minio123 + 1 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 441a5a541e8..b000ccabcf4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -556,7 +556,7 @@ def test_query_is_canceled_with_inf_retries(cluster, broken_s3): @pytest.mark.parametrize("node_name", ["node", "node_with_inf_s3_retries"]) -def test_aggressive_timeouts(cluster, broken_s3, node_name): +def test_adaptive_timeouts(cluster, broken_s3, node_name): node = cluster.instances[node_name] broken_s3.setup_fake_puts(part_length=1) @@ -565,12 +565,12 @@ def test_aggressive_timeouts(cluster, broken_s3, node_name): count=1000000, ) - insert_query_id = f"TEST_AGGRESSIVE_TIMEOUTS_{node_name}" + insert_query_id = f"TEST_ADAPTIVE_TIMEOUTS_{node_name}" node.query( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/aggressive_timeouts', + 'http://resolver:8083/root/data/adaptive_timeouts', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -593,20 +593,20 @@ def test_aggressive_timeouts(cluster, broken_s3, node_name): assert put_objects == 1 - s3_aggressive_timeouts_state = node.query( + s3_use_adaptive_timeouts = node.query( f""" SELECT value FROM system.settings WHERE - name='s3_aggressive_timeouts' + name='s3_use_adaptive_timeouts' """ ).strip() if node_name == "node_with_inf_s3_retries": # first 2 attempts failed - assert s3_aggressive_timeouts_state == "1" - assert s3_errors == 2 + assert s3_use_adaptive_timeouts == "1" + assert s3_errors == 1 else: - assert s3_aggressive_timeouts_state == "0" + assert s3_use_adaptive_timeouts == "0" assert s3_errors == 0 diff --git a/tests/integration/test_storage_s3/configs/s3_retry.xml b/tests/integration/test_storage_s3/configs/s3_retry.xml index b7a7bbc8a9b..3171da051d0 100644 --- a/tests/integration/test_storage_s3/configs/s3_retry.xml +++ b/tests/integration/test_storage_s3/configs/s3_retry.xml @@ -1,6 +1,7 @@ + 1 10 5 diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 103dd30340c..5ef781bdc9e 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -4,6 +4,7 @@ import re import socket import struct import sys +import time def gen_n_digit_number(n): @@ -39,14 +40,14 @@ random.seed("Unstable server/1.0") # Generating some "random" data and append a line which contains sum of numbers in column 4. lines = ( - b"".join((gen_line() for _ in range(500000))) + b"".join([gen_line() for _ in range(500000)]) + f"0,0,0,{-sum_in_4_column}\n".encode() ) class RequestHandler(http.server.BaseHTTPRequestHandler): def do_HEAD(self): - if self.path == "/root/test.csv": + if self.path == "/root/test.csv" or self.path == "/root/slow_send_test.csv": self.from_bytes = 0 self.end_bytes = len(lines) self.size = self.end_bytes @@ -101,6 +102,18 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): print("Dropping connection") break + if self.path == "/root/slow_send_test.csv": + self.send_block_size = 81920 + + for c, i in enumerate( + range(self.from_bytes, self.end_bytes, self.send_block_size) + ): + self.wfile.write( + lines[i : min(i + self.send_block_size, self.end_bytes)] + ) + self.wfile.flush() + time.sleep(1) + elif self.path == "/": self.wfile.write(b"OK") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 01ade1acc4d..8c79ad02445 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -818,6 +818,15 @@ def test_storage_s3_get_unstable(started_cluster): assert result.splitlines() == ["500001,500000,0"] +def test_storage_s3_get_slow(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + table_format = "column1 Int64, column2 Int64, column3 Int64, column4 Int64" + get_query = f"SELECT count(), sum(column3), sum(column4) FROM s3('http://resolver:8081/{started_cluster.minio_bucket}/slow_send_test.csv', 'CSV', '{table_format}') FORMAT CSV" + result = run_query(instance, get_query) + assert result.splitlines() == ["500001,500000,0"] + + def test_storage_s3_put_uncompressed(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] From 3b73780bf6a749cb22536ffd62d8aa5c8c0638b0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Nov 2023 15:17:25 +0000 Subject: [PATCH 303/813] Calculate sets from (array) joined actions --- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 33 ++++++++++++++++--- src/Planner/PlannerJoinTree.h | 1 + .../02916_analyzer_set_in_join.reference | 2 ++ .../02916_analyzer_set_in_join.sql | 11 +++++++ 5 files changed, 44 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02916_analyzer_set_in_join.reference create mode 100644 tests/queries/0_stateless/02916_analyzer_set_in_join.sql diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 5b354ccda46..89166316261 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1390,7 +1390,7 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info); - std::vector result_actions_to_execute; + std::vector result_actions_to_execute = std::move(join_tree_query_plan.actions_dags); for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData()) { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index f8770ca7c9c..de808d9de6d 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -952,8 +952,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, - JoinTreeQueryPlan left_join_tree_query_plan, - JoinTreeQueryPlan right_join_tree_query_plan, + JoinTreeQueryPlan && left_join_tree_query_plan, + JoinTreeQueryPlan && right_join_tree_query_plan, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context) { @@ -1410,7 +1410,23 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); - return {std::move(result_plan), QueryProcessingStage::FetchColumns, std::move(left_join_tree_query_plan.used_row_policies)}; + std::vector result_actions_to_execute; + + std::move(left_join_tree_query_plan.actions_dags.begin(), left_join_tree_query_plan.actions_dags.end(), + std::back_inserter(result_actions_to_execute)); + std::move(right_join_tree_query_plan.actions_dags.begin(), right_join_tree_query_plan.actions_dags.end(), + std::back_inserter(result_actions_to_execute)); + if (join_clauses_and_actions.left_join_expressions_actions) + result_actions_to_execute.emplace_back(std::move(join_clauses_and_actions.left_join_expressions_actions)); + if (join_clauses_and_actions.right_join_expressions_actions) + result_actions_to_execute.emplace_back(std::move(join_clauses_and_actions.right_join_expressions_actions)); + + return JoinTreeQueryPlan{ + .query_plan = std::move(result_plan), + .from_stage = QueryProcessingStage::FetchColumns, + .used_row_policies = std::move(left_join_tree_query_plan.used_row_policies), + .actions_dags = std::move(result_actions_to_execute), + }; } JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_join_table_expression, @@ -1450,6 +1466,10 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ } array_join_action_dag->projectInput(); + + std::vector result_actions_to_execute; + result_actions_to_execute.push_back(array_join_action_dag); + auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), array_join_action_dag); array_join_actions->setStepDescription("ARRAY JOIN actions"); plan.addStep(std::move(array_join_actions)); @@ -1488,7 +1508,12 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_step->setStepDescription("ARRAY JOIN"); plan.addStep(std::move(array_join_step)); - return {std::move(plan), QueryProcessingStage::FetchColumns, std::move(join_tree_query_plan.used_row_policies)}; + return JoinTreeQueryPlan{ + .query_plan = std::move(plan), + .from_stage = QueryProcessingStage::FetchColumns, + .used_row_policies = std::move(join_tree_query_plan.used_row_policies), + .actions_dags = std::move(result_actions_to_execute), + }; } } diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index d4d6c173847..06acf4cd2fd 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -16,6 +16,7 @@ struct JoinTreeQueryPlan QueryPlan query_plan; QueryProcessingStage::Enum from_stage; std::set used_row_policies; + std::vector actions_dags = {}; }; /// Build JOIN TREE query plan for query node diff --git a/tests/queries/0_stateless/02916_analyzer_set_in_join.reference b/tests/queries/0_stateless/02916_analyzer_set_in_join.reference new file mode 100644 index 00000000000..a063ea39893 --- /dev/null +++ b/tests/queries/0_stateless/02916_analyzer_set_in_join.reference @@ -0,0 +1,2 @@ +1 0 +42 1 diff --git a/tests/queries/0_stateless/02916_analyzer_set_in_join.sql b/tests/queries/0_stateless/02916_analyzer_set_in_join.sql new file mode 100644 index 00000000000..cae17d74a97 --- /dev/null +++ b/tests/queries/0_stateless/02916_analyzer_set_in_join.sql @@ -0,0 +1,11 @@ + +SELECT 1, b +FROM numbers(1) +ARRAY JOIN [materialize(3) IN (SELECT 42)] AS b +; + +SELECT * +FROM (SELECT materialize(42) as a) as t1 +JOIN (SELECT materialize(1) as a) as t2 +ON t1.a IN (SELECT 42) = t2.a +; From daa60acb19aa8dfe1b6bc76a34ef08dd1d81cfc6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 14 Nov 2023 11:03:00 +0000 Subject: [PATCH 304/813] Bug: #56673 --- src/Compression/CompressionCodecGCD.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index 70fff01ebf7..8b923d61650 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -246,7 +246,7 @@ UInt8 getGCDBytesSize(const IDataType * column_type) { WhichDataType which(column_type); if (!(which.isInt() || which.isUInt() || which.isDecimal() || which.isDateOrDate32() || which.isDateTime() ||which.isDateTime64())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec GCD is not applicable for {} because the data type is not of fixed size", + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec GCD cannot be applied to column {} because it can only be used with Int*, UInt*, Decimal*, Date* or DateTime* types.", column_type->getName()); size_t max_size = column_type->getSizeOfValueInMemory(); From 11b8395fe7aa734c9ec8d4ee9f40ede995d230d6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 14 Nov 2023 11:05:36 +0000 Subject: [PATCH 305/813] Address review comments --- src/Planner/PlannerJoinTree.cpp | 30 +++++++++++++++--------------- src/Planner/PlannerJoinTree.h | 2 +- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index de808d9de6d..7c1e6ded1e0 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -948,12 +948,16 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } } - return {std::move(query_plan), from_stage, std::move(used_row_policies)}; + return JoinTreeQueryPlan{ + .query_plan = std::move(query_plan), + .from_stage = from_stage, + .used_row_policies = std::move(used_row_policies), + }; } JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, - JoinTreeQueryPlan && left_join_tree_query_plan, - JoinTreeQueryPlan && right_join_tree_query_plan, + JoinTreeQueryPlan left_join_tree_query_plan, + JoinTreeQueryPlan right_join_tree_query_plan, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context) { @@ -1410,22 +1414,19 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); - std::vector result_actions_to_execute; - - std::move(left_join_tree_query_plan.actions_dags.begin(), left_join_tree_query_plan.actions_dags.end(), - std::back_inserter(result_actions_to_execute)); - std::move(right_join_tree_query_plan.actions_dags.begin(), right_join_tree_query_plan.actions_dags.end(), - std::back_inserter(result_actions_to_execute)); + /// Collect all required actions dags in `left_join_tree_query_plan.actions_dags` + for (auto && action_dag : right_join_tree_query_plan.actions_dags) + left_join_tree_query_plan.actions_dags.emplace_back(action_dag); if (join_clauses_and_actions.left_join_expressions_actions) - result_actions_to_execute.emplace_back(std::move(join_clauses_and_actions.left_join_expressions_actions)); + left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.left_join_expressions_actions)); if (join_clauses_and_actions.right_join_expressions_actions) - result_actions_to_execute.emplace_back(std::move(join_clauses_and_actions.right_join_expressions_actions)); + left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.right_join_expressions_actions)); return JoinTreeQueryPlan{ .query_plan = std::move(result_plan), .from_stage = QueryProcessingStage::FetchColumns, .used_row_policies = std::move(left_join_tree_query_plan.used_row_policies), - .actions_dags = std::move(result_actions_to_execute), + .actions_dags = std::move(left_join_tree_query_plan.actions_dags), }; } @@ -1467,8 +1468,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_action_dag->projectInput(); - std::vector result_actions_to_execute; - result_actions_to_execute.push_back(array_join_action_dag); + join_tree_query_plan.actions_dags.push_back(array_join_action_dag); auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), array_join_action_dag); array_join_actions->setStepDescription("ARRAY JOIN actions"); @@ -1512,7 +1512,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ .query_plan = std::move(plan), .from_stage = QueryProcessingStage::FetchColumns, .used_row_policies = std::move(join_tree_query_plan.used_row_policies), - .actions_dags = std::move(result_actions_to_execute), + .actions_dags = std::move(join_tree_query_plan.actions_dags), }; } diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index 06acf4cd2fd..c5a7d14fa55 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -16,7 +16,7 @@ struct JoinTreeQueryPlan QueryPlan query_plan; QueryProcessingStage::Enum from_stage; std::set used_row_policies; - std::vector actions_dags = {}; + std::vector actions_dags; }; /// Build JOIN TREE query plan for query node From 8e88bf1348d66d7196475c8bc5bf2708af2a5125 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 13 Nov 2023 22:28:04 +0100 Subject: [PATCH 306/813] Revert "Revert "Add a setting max_execution_time_leaf to limit the execution time on shard for distributed query"" --- .../operations/settings/query-complexity.md | 22 ++++++++++++++++++- src/Core/Settings.h | 16 ++++++++------ .../ClusterProxy/SelectStreamFactory.cpp | 13 ++++++----- .../ClusterProxy/executeQuery.cpp | 8 +++++++ .../02786_max_execution_time_leaf.reference | 0 .../02786_max_execution_time_leaf.sql | 4 ++++ 6 files changed, 50 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02786_max_execution_time_leaf.reference create mode 100644 tests/queries/0_stateless/02786_max_execution_time_leaf.sql diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 2211b0c2de2..9e36aa26946 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -172,7 +172,27 @@ If you set `timeout_before_checking_execution_speed `to 0, ClickHouse will use c ## timeout_overflow_mode {#timeout-overflow-mode} -What to do if the query is run longer than ‘max_execution_time’: ‘throw’ or ‘break’. By default, throw. +What to do if the query is run longer than `max_execution_time`: `throw` or `break`. By default, `throw`. + +# max_execution_time_leaf + +Similar semantic to `max_execution_time` but only apply on leaf node for distributed or remote queries. + +For example, if we want to limit execution time on leaf node to `10s` but no limit on the initial node, instead of having `max_execution_time` in the nested subquery settings: + +``` sql +SELECT count() FROM cluster(cluster, view(SELECT * FROM t SETTINGS max_execution_time = 10)); +``` + +We can use `max_execution_time_leaf` as the query settings: + +``` sql +SELECT count() FROM cluster(cluster, view(SELECT * FROM t)) SETTINGS max_execution_time_leaf = 10; +``` + +# timeout_overflow_mode_leaf + +What to do when the query in leaf node run longer than `max_execution_time_leaf`: `throw` or `break`. By default, `throw`. ## min_execution_speed {#min-execution-speed} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3b90a3e068b..b7d1af17a64 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -364,16 +364,16 @@ class IColumn; M(UInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \ M(OverflowMode, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ - M(UInt64, max_rows_to_read_leaf, 0, "Limit on read rows on the leaf nodes for distributed queries. Limit is applied for local reads only excluding the final merge stage on the root node. Note, the setting is unstable with prefer_localhost_replica=1.", 0) \ - M(UInt64, max_bytes_to_read_leaf, 0, "Limit on read bytes (after decompression) on the leaf nodes for distributed queries. Limit is applied for local reads only excluding the final merge stage on the root node. Note, the setting is unstable with prefer_localhost_replica=1.", 0) \ + M(UInt64, max_rows_to_read_leaf, 0, "Limit on read rows on the leaf nodes for distributed queries. Limit is applied for local reads only, excluding the final merge stage on the root node. Note, the setting is unstable with prefer_localhost_replica=1.", 0) \ + M(UInt64, max_bytes_to_read_leaf, 0, "Limit on read bytes (after decompression) on the leaf nodes for distributed queries. Limit is applied for local reads only, excluding the final merge stage on the root node. Note, the setting is unstable with prefer_localhost_replica=1.", 0) \ M(OverflowMode, read_overflow_mode_leaf, OverflowMode::THROW, "What to do when the leaf limit is exceeded.", 0) \ \ - M(UInt64, max_rows_to_group_by, 0, "If aggregation during GROUP BY is generating more than specified number of rows (unique GROUP BY keys), the behavior will be determined by the 'group_by_overflow_mode' which by default is - throw an exception, but can be also switched to an approximate GROUP BY mode.", 0) \ + M(UInt64, max_rows_to_group_by, 0, "If aggregation during GROUP BY is generating more than the specified number of rows (unique GROUP BY keys), the behavior will be determined by the 'group_by_overflow_mode' which by default is - throw an exception, but can be also switched to an approximate GROUP BY mode.", 0) \ M(OverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(UInt64, max_bytes_before_external_group_by, 0, "If memory usage during GROUP BY operation is exceeding this threshold in bytes, activate the 'external aggregation' mode (spill data to disk). Recommended value is half of available system memory.", 0) \ \ - M(UInt64, max_rows_to_sort, 0, "If more than specified amount of records have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ - M(UInt64, max_bytes_to_sort, 0, "If more than specified amount of (uncompressed) bytes have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ + M(UInt64, max_rows_to_sort, 0, "If more than the specified amount of records have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ + M(UInt64, max_bytes_to_sort, 0, "If more than the specified amount of (uncompressed) bytes have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ M(OverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(UInt64, max_bytes_before_external_sort, 0, "If memory usage during ORDER BY operation is exceeding this threshold in bytes, activate the 'external sorting' mode (spill data to disk). Recommended value is half of available system memory.", 0) \ M(UInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.", 0) \ @@ -384,8 +384,10 @@ class IColumn; M(OverflowMode, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ /* TODO: Check also when merging and finalizing aggregate functions. */ \ - M(Seconds, max_execution_time, 0, "If query run time exceeded the specified number of seconds, the behavior will be determined by the 'timeout_overflow_mode' which by default is - throw an exception. Note that the timeout is checked and query can stop only in designated places during data processing. It currently cannot stop during merging of aggregation states or during query analysis, and the actual run time will be higher than the value of this setting.", 0) \ + M(Seconds, max_execution_time, 0, "If query runtime exceeds the specified number of seconds, the behavior will be determined by the 'timeout_overflow_mode', which by default is - throw an exception. Note that the timeout is checked and query can stop only in designated places during data processing. It currently cannot stop during merging of aggregation states or during query analysis, and the actual run time will be higher than the value of this setting.", 0) \ M(OverflowMode, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(Seconds, max_execution_time_leaf, 0, "Similar semantic to max_execution_time but only apply on leaf node for distributed queries, the time out behavior will be determined by 'timeout_overflow_mode_leaf' which by default is - throw an exception", 0) \ + M(OverflowMode, timeout_overflow_mode_leaf, OverflowMode::THROW, "What to do when the leaf limit is exceeded.", 0) \ \ M(UInt64, min_execution_speed, 0, "Minimum number of execution rows per second.", 0) \ M(UInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \ @@ -399,7 +401,7 @@ class IColumn; \ M(UInt64, max_sessions_for_user, 0, "Maximum number of simultaneous sessions for a user.", 0) \ \ - M(UInt64, max_subquery_depth, 100, "If a query has more than specified number of nested subqueries, throw an exception. This allows you to have a sanity check to protect the users of your cluster from going insane with their queries.", 0) \ + M(UInt64, max_subquery_depth, 100, "If a query has more than the specified number of nested subqueries, throw an exception. This allows you to have a sanity check to protect the users of your cluster from going insane with their queries.", 0) \ M(UInt64, max_analyze_depth, 5000, "Maximum number of analyses performed by interpreter.", 0) \ M(UInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \ M(UInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \ diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 3935028f27c..4edc9d4d4e5 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -1,20 +1,21 @@ -#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 @@ -22,6 +23,7 @@ #include #include + namespace ProfileEvents { extern const Event DistributedConnectionMissingTable; @@ -121,6 +123,7 @@ void SelectStreamFactory::createForShard( if (it != objects_by_shard.end()) replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, query_ast); + auto emplace_local_stream = [&]() { local_plans.emplace_back(createLocalPlan( diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 41235d107cd..420bb447027 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -141,6 +141,14 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, new_settings.allow_experimental_parallel_reading_from_replicas = false; } + if (settings.max_execution_time_leaf.value > 0) + { + /// Replace 'max_execution_time' of this sub-query with 'max_execution_time_leaf' and 'timeout_overflow_mode' + /// with 'timeout_overflow_mode_leaf' + new_settings.max_execution_time = settings.max_execution_time_leaf; + new_settings.timeout_overflow_mode = settings.timeout_overflow_mode_leaf; + } + auto new_context = Context::createCopy(context); new_context->setSettings(new_settings); return new_context; diff --git a/tests/queries/0_stateless/02786_max_execution_time_leaf.reference b/tests/queries/0_stateless/02786_max_execution_time_leaf.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02786_max_execution_time_leaf.sql b/tests/queries/0_stateless/02786_max_execution_time_leaf.sql new file mode 100644 index 00000000000..1d02e82569c --- /dev/null +++ b/tests/queries/0_stateless/02786_max_execution_time_leaf.sql @@ -0,0 +1,4 @@ +-- Tags: no-fasttest +SELECT count() FROM cluster('test_cluster_two_shards', view( SELECT * FROM numbers(100000000000) )) SETTINGS max_execution_time_leaf = 1; -- { serverError 159 } +-- Can return partial result +SELECT count() FROM cluster('test_cluster_two_shards', view( SELECT * FROM numbers(100000000000) )) FORMAT Null SETTINGS max_execution_time_leaf = 1, timeout_overflow_mode_leaf = 'break'; From ae05f4f604e2eead6ac4bbe41d30cb06b3784cd7 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 13 Nov 2023 21:35:15 +0000 Subject: [PATCH 307/813] Fix use_structure_from_insertion_table_in_table_functions with new Analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 175 +++++++++++----------- 1 file changed, 88 insertions(+), 87 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7855c4f34a8..d6c9c7ab807 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6327,6 +6327,94 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, table_function_name); } + QueryTreeNodes result_table_function_arguments; + + auto skip_analysis_arguments_indexes = table_function_ptr->skipAnalysisForArguments(table_function_node, scope_context); + + auto & table_function_arguments = table_function_node_typed.getArguments().getNodes(); + size_t table_function_arguments_size = table_function_arguments.size(); + + for (size_t table_function_argument_index = 0; table_function_argument_index < table_function_arguments_size; ++table_function_argument_index) + { + auto & table_function_argument = table_function_arguments[table_function_argument_index]; + + auto skip_argument_index_it = std::find(skip_analysis_arguments_indexes.begin(), + skip_analysis_arguments_indexes.end(), + table_function_argument_index); + if (skip_argument_index_it != skip_analysis_arguments_indexes.end()) + { + result_table_function_arguments.push_back(table_function_argument); + continue; + } + + if (auto * identifier_node = table_function_argument->as()) + { + const auto & unresolved_identifier = identifier_node->getIdentifier(); + auto identifier_resolve_result = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::EXPRESSION}, scope); + auto resolved_identifier = std::move(identifier_resolve_result.resolved_identifier); + + if (resolved_identifier && resolved_identifier->getNodeType() == QueryTreeNodeType::CONSTANT) + result_table_function_arguments.push_back(std::move(resolved_identifier)); + else + result_table_function_arguments.push_back(table_function_argument); + + continue; + } + else if (auto * table_function_argument_function = table_function_argument->as()) + { + const auto & table_function_argument_function_name = table_function_argument_function->getFunctionName(); + if (TableFunctionFactory::instance().isTableFunctionName(table_function_argument_function_name)) + { + auto table_function_node_to_resolve_typed = std::make_shared(table_function_argument_function_name); + table_function_node_to_resolve_typed->getArgumentsNode() = table_function_argument_function->getArgumentsNode(); + + QueryTreeNodePtr table_function_node_to_resolve = std::move(table_function_node_to_resolve_typed); + resolveTableFunction(table_function_node_to_resolve, scope, expressions_visitor, true /*nested_table_function*/); + + result_table_function_arguments.push_back(std::move(table_function_node_to_resolve)); + continue; + } + } + + /** Table functions arguments can contain expressions with invalid identifiers. + * We cannot skip analysis for such arguments, because some table functions cannot provide + * information if analysis for argument should be skipped until other arguments will be resolved. + * + * Example: SELECT key from remote('127.0.0.{1,2}', view(select number AS key from numbers(2)), cityHash64(key)); + * Example: SELECT id from remote('127.0.0.{1,2}', 'default', 'test_table', cityHash64(id)); + */ + try + { + resolveExpressionNode(table_function_argument, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + } + catch (const Exception & exception) + { + if (exception.code() == ErrorCodes::UNKNOWN_IDENTIFIER) + { + result_table_function_arguments.push_back(table_function_argument); + continue; + } + + throw; + } + + if (auto * expression_list = table_function_argument->as()) + { + for (auto & expression_list_node : expression_list->getNodes()) + result_table_function_arguments.push_back(expression_list_node); + } + else + { + result_table_function_arguments.push_back(table_function_argument); + } + } + + table_function_node_typed.getArguments().getNodes() = std::move(result_table_function_arguments); + + auto table_function_ast = table_function_node_typed.toAST(); + table_function_ptr->parseArguments(table_function_ast, scope_context); + + uint64_t use_structure_from_insertion_table_in_table_functions = scope_context->getSettingsRef().use_structure_from_insertion_table_in_table_functions; if (!nested_table_function && use_structure_from_insertion_table_in_table_functions && @@ -6468,93 +6556,6 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, } } - QueryTreeNodes result_table_function_arguments; - - auto skip_analysis_arguments_indexes = table_function_ptr->skipAnalysisForArguments(table_function_node, scope_context); - - auto & table_function_arguments = table_function_node_typed.getArguments().getNodes(); - size_t table_function_arguments_size = table_function_arguments.size(); - - for (size_t table_function_argument_index = 0; table_function_argument_index < table_function_arguments_size; ++table_function_argument_index) - { - auto & table_function_argument = table_function_arguments[table_function_argument_index]; - - auto skip_argument_index_it = std::find(skip_analysis_arguments_indexes.begin(), - skip_analysis_arguments_indexes.end(), - table_function_argument_index); - if (skip_argument_index_it != skip_analysis_arguments_indexes.end()) - { - result_table_function_arguments.push_back(table_function_argument); - continue; - } - - if (auto * identifier_node = table_function_argument->as()) - { - const auto & unresolved_identifier = identifier_node->getIdentifier(); - auto identifier_resolve_result = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::EXPRESSION}, scope); - auto resolved_identifier = std::move(identifier_resolve_result.resolved_identifier); - - if (resolved_identifier && resolved_identifier->getNodeType() == QueryTreeNodeType::CONSTANT) - result_table_function_arguments.push_back(std::move(resolved_identifier)); - else - result_table_function_arguments.push_back(table_function_argument); - - continue; - } - else if (auto * table_function_argument_function = table_function_argument->as()) - { - const auto & table_function_argument_function_name = table_function_argument_function->getFunctionName(); - if (TableFunctionFactory::instance().isTableFunctionName(table_function_argument_function_name)) - { - auto table_function_node_to_resolve_typed = std::make_shared(table_function_argument_function_name); - table_function_node_to_resolve_typed->getArgumentsNode() = table_function_argument_function->getArgumentsNode(); - - QueryTreeNodePtr table_function_node_to_resolve = std::move(table_function_node_to_resolve_typed); - resolveTableFunction(table_function_node_to_resolve, scope, expressions_visitor, true /*nested_table_function*/); - - result_table_function_arguments.push_back(std::move(table_function_node_to_resolve)); - continue; - } - } - - /** Table functions arguments can contain expressions with invalid identifiers. - * We cannot skip analysis for such arguments, because some table functions cannot provide - * information if analysis for argument should be skipped until other arguments will be resolved. - * - * Example: SELECT key from remote('127.0.0.{1,2}', view(select number AS key from numbers(2)), cityHash64(key)); - * Example: SELECT id from remote('127.0.0.{1,2}', 'default', 'test_table', cityHash64(id)); - */ - try - { - resolveExpressionNode(table_function_argument, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - } - catch (const Exception & exception) - { - if (exception.code() == ErrorCodes::UNKNOWN_IDENTIFIER) - { - result_table_function_arguments.push_back(table_function_argument); - continue; - } - - throw; - } - - if (auto * expression_list = table_function_argument->as()) - { - for (auto & expression_list_node : expression_list->getNodes()) - result_table_function_arguments.push_back(expression_list_node); - } - else - { - result_table_function_arguments.push_back(table_function_argument); - } - } - - table_function_node_typed.getArguments().getNodes() = std::move(result_table_function_arguments); - - auto table_function_ast = table_function_node_typed.toAST(); - table_function_ptr->parseArguments(table_function_ast, scope_context); - auto table_function_storage = scope_context->getQueryContext()->executeTableFunction(table_function_ast, table_function_ptr); table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context, std::move(skip_analysis_arguments_indexes)); } From cf7f155ce9dab0ea0a40d5104ae35562eded282e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 12 Nov 2023 14:41:19 +0000 Subject: [PATCH 308/813] Bump gRPC to v1.55.4 and protobuf to v22.5 --- contrib/google-protobuf | 2 +- contrib/google-protobuf-cmake/CMakeLists.txt | 199 +++++++++++++------ contrib/grpc | 2 +- 3 files changed, 144 insertions(+), 59 deletions(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index 2a4fa1a4e95..089b89c8d41 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit 2a4fa1a4e95012d754ac55d43c8bc462dd1c78a8 +Subproject commit 089b89c8d4140f0d49fe4222b047a8ea814bc752 diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 268f0fbe0e4..fc5bf2c0241 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -30,17 +30,70 @@ include_directories( ${protobuf_binary_dir} ${protobuf_source_dir}/src) +add_library(utf8_range + ${protobuf_source_dir}/third_party/utf8_range/naive.c + ${protobuf_source_dir}/third_party/utf8_range/range2-neon.c + ${protobuf_source_dir}/third_party/utf8_range/range2-sse.c +) +include_directories(${protobuf_source_dir}/third_party/utf8_range) + +add_library(utf8_validity + ${protobuf_source_dir}/third_party/utf8_range/utf8_validity.cc +) +target_link_libraries(utf8_validity PUBLIC absl::strings) + +set(protobuf_absl_used_targets + absl::absl_check + absl::absl_log + absl::algorithm + absl::base + absl::bind_front + absl::bits + absl::btree + absl::cleanup + absl::cord + absl::core_headers + absl::debugging + absl::die_if_null + absl::dynamic_annotations + absl::flags + absl::flat_hash_map + absl::flat_hash_set + absl::function_ref + absl::hash + absl::layout + absl::log_initialize + absl::log_severity + absl::memory + absl::node_hash_map + absl::node_hash_set + absl::optional + absl::span + absl::status + absl::statusor + absl::strings + absl::synchronization + absl::time + absl::type_traits + absl::utility + absl::variant +) + set(libprotobuf_lite_files ${protobuf_source_dir}/src/google/protobuf/any_lite.cc ${protobuf_source_dir}/src/google/protobuf/arena.cc + ${protobuf_source_dir}/src/google/protobuf/arena_align.cc + ${protobuf_source_dir}/src/google/protobuf/arena_config.cc ${protobuf_source_dir}/src/google/protobuf/arenastring.cc + ${protobuf_source_dir}/src/google/protobuf/arenaz_sampler.cc ${protobuf_source_dir}/src/google/protobuf/extension_set.cc ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_tctable_lite.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc ${protobuf_source_dir}/src/google/protobuf/implicit_weak_message.cc + ${protobuf_source_dir}/src/google/protobuf/inlined_string_field.cc ${protobuf_source_dir}/src/google/protobuf/io/coded_stream.cc ${protobuf_source_dir}/src/google/protobuf/io/io_win32.cc - ${protobuf_source_dir}/src/google/protobuf/io/strtod.cc ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream.cc ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl.cc ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl_lite.cc @@ -48,21 +101,15 @@ set(libprotobuf_lite_files ${protobuf_source_dir}/src/google/protobuf/message_lite.cc ${protobuf_source_dir}/src/google/protobuf/parse_context.cc ${protobuf_source_dir}/src/google/protobuf/repeated_field.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/bytestream.cc + ${protobuf_source_dir}/src/google/protobuf/repeated_ptr_field.cc ${protobuf_source_dir}/src/google/protobuf/stubs/common.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/int128.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/status.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/statusor.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/stringpiece.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/stringprintf.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/structurally_valid.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/strutil.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/time.cc ${protobuf_source_dir}/src/google/protobuf/wire_format_lite.cc ) add_library(_libprotobuf-lite ${libprotobuf_lite_files}) -target_link_libraries(_libprotobuf-lite pthread) +target_link_libraries(_libprotobuf-lite + pthread + utf8_validity) if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") target_link_libraries(_libprotobuf-lite log) endif() @@ -71,67 +118,89 @@ add_library(protobuf::libprotobuf-lite ALIAS _libprotobuf-lite) set(libprotobuf_files - ${protobuf_source_dir}/src/google/protobuf/any.cc ${protobuf_source_dir}/src/google/protobuf/any.pb.cc ${protobuf_source_dir}/src/google/protobuf/api.pb.cc + ${protobuf_source_dir}/src/google/protobuf/duration.pb.cc + ${protobuf_source_dir}/src/google/protobuf/empty.pb.cc + ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc + ${protobuf_source_dir}/src/google/protobuf/source_context.pb.cc + ${protobuf_source_dir}/src/google/protobuf/struct.pb.cc + ${protobuf_source_dir}/src/google/protobuf/timestamp.pb.cc + ${protobuf_source_dir}/src/google/protobuf/type.pb.cc + ${protobuf_source_dir}/src/google/protobuf/wrappers.pb.cc + ${protobuf_source_dir}/src/google/protobuf/any.cc + ${protobuf_source_dir}/src/google/protobuf/any_lite.cc + ${protobuf_source_dir}/src/google/protobuf/arena.cc + ${protobuf_source_dir}/src/google/protobuf/arena_align.cc + ${protobuf_source_dir}/src/google/protobuf/arena_config.cc + ${protobuf_source_dir}/src/google/protobuf/arenastring.cc + ${protobuf_source_dir}/src/google/protobuf/arenaz_sampler.cc ${protobuf_source_dir}/src/google/protobuf/compiler/importer.cc ${protobuf_source_dir}/src/google/protobuf/compiler/parser.cc ${protobuf_source_dir}/src/google/protobuf/descriptor.cc ${protobuf_source_dir}/src/google/protobuf/descriptor.pb.cc ${protobuf_source_dir}/src/google/protobuf/descriptor_database.cc - ${protobuf_source_dir}/src/google/protobuf/duration.pb.cc ${protobuf_source_dir}/src/google/protobuf/dynamic_message.cc - ${protobuf_source_dir}/src/google/protobuf/empty.pb.cc + ${protobuf_source_dir}/src/google/protobuf/extension_set.cc ${protobuf_source_dir}/src/google/protobuf/extension_set_heavy.cc - ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc + ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_bases.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_tctable_full.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_tctable_gen.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_tctable_lite.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc + ${protobuf_source_dir}/src/google/protobuf/implicit_weak_message.cc + ${protobuf_source_dir}/src/google/protobuf/inlined_string_field.cc + ${protobuf_source_dir}/src/google/protobuf/io/coded_stream.cc ${protobuf_source_dir}/src/google/protobuf/io/gzip_stream.cc + ${protobuf_source_dir}/src/google/protobuf/io/io_win32.cc ${protobuf_source_dir}/src/google/protobuf/io/printer.cc + ${protobuf_source_dir}/src/google/protobuf/io/strtod.cc ${protobuf_source_dir}/src/google/protobuf/io/tokenizer.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_sink.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl_lite.cc + ${protobuf_source_dir}/src/google/protobuf/json/internal/lexer.cc + ${protobuf_source_dir}/src/google/protobuf/json/internal/message_path.cc + ${protobuf_source_dir}/src/google/protobuf/json/internal/parser.cc + ${protobuf_source_dir}/src/google/protobuf/json/internal/unparser.cc + ${protobuf_source_dir}/src/google/protobuf/json/internal/untyped_message.cc + ${protobuf_source_dir}/src/google/protobuf/json/internal/writer.cc + ${protobuf_source_dir}/src/google/protobuf/json/internal/zero_copy_buffered_stream.cc + ${protobuf_source_dir}/src/google/protobuf/json/json.cc + ${protobuf_source_dir}/src/google/protobuf/map.cc ${protobuf_source_dir}/src/google/protobuf/map_field.cc ${protobuf_source_dir}/src/google/protobuf/message.cc + ${protobuf_source_dir}/src/google/protobuf/message_lite.cc + ${protobuf_source_dir}/src/google/protobuf/parse_context.cc ${protobuf_source_dir}/src/google/protobuf/reflection_ops.cc + ${protobuf_source_dir}/src/google/protobuf/repeated_field.cc ${protobuf_source_dir}/src/google/protobuf/repeated_ptr_field.cc ${protobuf_source_dir}/src/google/protobuf/service.cc - ${protobuf_source_dir}/src/google/protobuf/source_context.pb.cc - ${protobuf_source_dir}/src/google/protobuf/struct.pb.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/substitute.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/common.cc ${protobuf_source_dir}/src/google/protobuf/text_format.cc - ${protobuf_source_dir}/src/google/protobuf/timestamp.pb.cc - ${protobuf_source_dir}/src/google/protobuf/type.pb.cc ${protobuf_source_dir}/src/google/protobuf/unknown_field_set.cc ${protobuf_source_dir}/src/google/protobuf/util/delimited_message_util.cc ${protobuf_source_dir}/src/google/protobuf/util/field_comparator.cc ${protobuf_source_dir}/src/google/protobuf/util/field_mask_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/datapiece.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/default_value_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/error_listener.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/field_mask_utility.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_escaping.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_stream_parser.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/object_writer.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/proto_writer.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectsource.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info_test_helper.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/utility.cc - ${protobuf_source_dir}/src/google/protobuf/util/json_util.cc ${protobuf_source_dir}/src/google/protobuf/util/message_differencer.cc ${protobuf_source_dir}/src/google/protobuf/util/time_util.cc ${protobuf_source_dir}/src/google/protobuf/util/type_resolver_util.cc ${protobuf_source_dir}/src/google/protobuf/wire_format.cc - ${protobuf_source_dir}/src/google/protobuf/wrappers.pb.cc + ${protobuf_source_dir}/src/google/protobuf/wire_format_lite.cc ) add_library(_libprotobuf ${libprotobuf_lite_files} ${libprotobuf_files}) if (ENABLE_FUZZING) target_compile_options(_libprotobuf PRIVATE "-fsanitize-recover=all") endif() -target_link_libraries(_libprotobuf pthread) -target_link_libraries(_libprotobuf ch_contrib::zlib) +target_link_libraries(_libprotobuf + pthread + ch_contrib::zlib + utf8_validity + ${protobuf_absl_used_targets}) if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") target_link_libraries(_libprotobuf log) endif() @@ -143,20 +212,21 @@ set(libprotoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/code_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/command_line_interface.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/enum_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/extension.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/string_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/file.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/map_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/message_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/padding_optimizer.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/parse_function_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/primitive_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/service.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/string_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/tracker.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_doc_comment.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum_field.cc @@ -173,6 +243,7 @@ set(libprotoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_primitive_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_source_generator_base.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_wrapper_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/names.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/context.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/doc_comment.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/enum.cc @@ -195,38 +266,48 @@ set(libprotoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_field_lite.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_serialization.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/name_resolver.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/names.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/primitive_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/primitive_field_lite.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/service.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/shared_code_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/string_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/java/string_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_oneof.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/import_writer.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/line_consumer.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/names.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/oneof.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/text_format_decode_data.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/php/names.cc ${protobuf_source_dir}/src/google/protobuf/compiler/php/php_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.cc ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.pb.cc ${protobuf_source_dir}/src/google/protobuf/compiler/python/generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/python/helpers.cc ${protobuf_source_dir}/src/google/protobuf/compiler/python/pyi_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/retention.cc ${protobuf_source_dir}/src/google/protobuf/compiler/ruby/ruby_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/subprocess.cc ${protobuf_source_dir}/src/google/protobuf/compiler/zip_writer.cc ) add_library(_libprotoc ${libprotoc_files}) -target_link_libraries(_libprotoc _libprotobuf) +target_link_libraries(_libprotoc + _libprotobuf + ${protobuf_absl_used_targets}) add_library(protobuf::libprotoc ALIAS _libprotoc) set(protoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/main.cc) @@ -235,7 +316,11 @@ if (CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR) add_executable(protoc ${protoc_files}) - target_link_libraries(protoc _libprotoc _libprotobuf pthread) + target_link_libraries(protoc _libprotoc + _libprotobuf + pthread + utf8_validity + ${protobuf_absl_used_targets}) add_executable(protobuf::protoc ALIAS protoc) if (ENABLE_FUZZING) diff --git a/contrib/grpc b/contrib/grpc index 6e5e645de7c..a08fe1a3407 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 6e5e645de7cb0604e3ad4ba55abff2eca38c1803 +Subproject commit a08fe1a34075c93bb2d606dd608b9a3953288b81 From 9e1357dd7eb733466fcfe00715490b00ef7d44ee Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Tue, 7 Nov 2023 13:44:52 +0200 Subject: [PATCH 309/813] Update `query_masking_rules` when reloading the config Fixes #56449 --- programs/server/Server.cpp | 2 + src/Common/SensitiveDataMasker.cpp | 4 + src/Interpreters/Context.cpp | 16 ++++ src/Interpreters/Context.h | 2 + .../__init__.py | 0 .../configs/changed_settings.xml | 19 +++++ .../configs/empty_settings.xml | 12 +++ .../test_reload_query_masking_rules/test.py | 74 +++++++++++++++++++ 8 files changed, 129 insertions(+) create mode 100644 tests/integration/test_reload_query_masking_rules/__init__.py create mode 100644 tests/integration/test_reload_query_masking_rules/configs/changed_settings.xml create mode 100644 tests/integration/test_reload_query_masking_rules/configs/empty_settings.xml create mode 100644 tests/integration/test_reload_query_masking_rules/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9e974e796e0..ca091dbeceb 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1372,6 +1372,8 @@ try global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); + global_context->reloadQueryMaskingRulesIfChanged(config); + std::lock_guard lock(servers_lock); updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); } diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 2b21c223bd8..5fc5c3618cc 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -104,6 +104,10 @@ void SensitiveDataMasker::setInstance(std::unique_ptr sensi { sensitive_data_masker = std::move(sensitive_data_masker_); } + else + { + sensitive_data_masker.reset(); + } } SensitiveDataMasker * SensitiveDataMasker::getInstance() diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 75cc5f8366c..8bd4d619349 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -196,6 +197,9 @@ struct ContextSharedPart : boost::noncopyable mutable zkutil::ZooKeeperPtr zookeeper TSA_GUARDED_BY(zookeeper_mutex); /// Client for ZooKeeper. ConfigurationPtr zookeeper_config TSA_GUARDED_BY(zookeeper_mutex); /// Stores zookeeper configs + mutable std::mutex sensitive_data_masker_mutex; + ConfigurationPtr sensitive_data_masker_config; + #if USE_NURAFT mutable std::mutex keeper_dispatcher_mutex; mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); @@ -3198,6 +3202,18 @@ bool Context::hasAuxiliaryZooKeeper(const String & name) const return getConfigRef().has("auxiliary_zookeepers." + name); } +void Context::reloadQueryMaskingRulesIfChanged(const ConfigurationPtr & config) const +{ + std::lock_guard lock(shared->sensitive_data_masker_mutex); + + const auto old_config = shared->sensitive_data_masker_config; + if (old_config && isSameConfiguration(*config, *old_config, "query_masking_rules")) + return; + + SensitiveDataMasker::setInstance(std::make_unique(*config, "query_masking_rules")); + shared->sensitive_data_masker_config = config; +} + InterserverCredentialsPtr Context::getInterserverCredentials() const { return shared->interserver_io_credentials.get(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e12a5c4b69b..f90812df8c1 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -946,6 +946,8 @@ public: // Reload Zookeeper void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const; + void reloadQueryMaskingRulesIfChanged(const ConfigurationPtr & config) const; + void setSystemZooKeeperLogAfterInitializationIfNeeded(); /// --- Caches ------------------------------------------------------------------------------------------ diff --git a/tests/integration/test_reload_query_masking_rules/__init__.py b/tests/integration/test_reload_query_masking_rules/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_reload_query_masking_rules/configs/changed_settings.xml b/tests/integration/test_reload_query_masking_rules/configs/changed_settings.xml new file mode 100644 index 00000000000..d681496d843 --- /dev/null +++ b/tests/integration/test_reload_query_masking_rules/configs/changed_settings.xml @@ -0,0 +1,19 @@ + + + system + query_log
+ toYYYYMM(event_date) + 7500 + 1048576 + 8192 + 524288 + false +
+ + + + TOPSECRET.TOPSECRET + [hidden] + + +
diff --git a/tests/integration/test_reload_query_masking_rules/configs/empty_settings.xml b/tests/integration/test_reload_query_masking_rules/configs/empty_settings.xml new file mode 100644 index 00000000000..82647ff82b5 --- /dev/null +++ b/tests/integration/test_reload_query_masking_rules/configs/empty_settings.xml @@ -0,0 +1,12 @@ + + + system + query_log
+ toYYYYMM(event_date) + 7500 + 1048576 + 8192 + 524288 + false +
+
diff --git a/tests/integration/test_reload_query_masking_rules/test.py b/tests/integration/test_reload_query_masking_rules/test.py new file mode 100644 index 00000000000..0f29bd0825e --- /dev/null +++ b/tests/integration/test_reload_query_masking_rules/test.py @@ -0,0 +1,74 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, assert_logs_contain_with_retry + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", user_configs=["configs/empty_settings.xml"]) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def reset_to_normal_settings_after_test(): + try: + node.copy_file_to_container( + os.path.join(SCRIPT_DIR, "configs/empty_settings.xml"), + "/etc/clickhouse-server/config.d/z.xml", + ) + node.query("SYSTEM RELOAD CONFIG") + yield + finally: + pass + + +# @pytest.mark.parametrize("reload_strategy", ["force", "timeout"]) +def test_reload_query_masking_rules(): + # At first, empty configuration is fed to ClickHouse. The query + # "SELECT 'TOPSECRET.TOPSECRET'" will not be redacted, and the new masking + # event will not be registered + node.query("SELECT 'TOPSECRET.TOPSECRET'") + assert_logs_contain_with_retry(node, "SELECT 'TOPSECRET.TOPSECRET'") + + # If there were no 'QueryMaskingRulesMatch' events, the query below returns + # 0 rows + assert ( + node.query( + "SELECT count(value) FROM system.events WHERE name = 'QueryMaskingRulesMatch'" + ) + == "0\n" + ) + + node.copy_file_to_container( + os.path.join(SCRIPT_DIR, "configs/changed_settings.xml"), + "/etc/clickhouse-server/config.d/z.xml", + ) + + node.query("SYSTEM RELOAD CONFIG") + + # Now the same query will be redacted in the logs and the counter of events + # will be incremented + node.query("SELECT 'TOPSECRET.TOPSECRET'") + + assert_eq_with_retry( + node, + "SELECT count(value) FROM system.events WHERE name = 'QueryMaskingRulesMatch'", + "1", + ) + assert_logs_contain_with_retry(node, r"SELECT '\[hidden\]'") + assert ( + node.query( + "SELECT value FROM system.events WHERE name = 'QueryMaskingRulesMatch'" + ) + == "1\n" + ) + + node.rotate_logs() From dfdd1a761e74d49150f927534fa8f8d31bcc1b43 Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Mon, 13 Nov 2023 13:00:52 +0200 Subject: [PATCH 310/813] Move mutex from Context to SensitiveDataMasker --- src/Common/SensitiveDataMasker.cpp | 5 +++++ src/Common/SensitiveDataMasker.h | 2 ++ src/Interpreters/Context.cpp | 3 --- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 5fc5c3618cc..fd7f45b11c6 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -1,5 +1,6 @@ #include "SensitiveDataMasker.h" +#include #include #include #include @@ -94,9 +95,12 @@ public: SensitiveDataMasker::~SensitiveDataMasker() = default; std::unique_ptr SensitiveDataMasker::sensitive_data_masker = nullptr; +std::mutex SensitiveDataMasker::instance_mutex; void SensitiveDataMasker::setInstance(std::unique_ptr sensitive_data_masker_) { + std::lock_guard lock(instance_mutex); + if (!sensitive_data_masker_) throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: the 'sensitive_data_masker' is not set"); @@ -112,6 +116,7 @@ void SensitiveDataMasker::setInstance(std::unique_ptr sensi SensitiveDataMasker * SensitiveDataMasker::getInstance() { + std::lock_guard lock(instance_mutex); return sensitive_data_masker.get(); } diff --git a/src/Common/SensitiveDataMasker.h b/src/Common/SensitiveDataMasker.h index de5cc125dcc..abb613043d1 100644 --- a/src/Common/SensitiveDataMasker.h +++ b/src/Common/SensitiveDataMasker.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -45,6 +46,7 @@ class SensitiveDataMasker private: class MaskingRule; std::vector> all_masking_rules; + static std::mutex instance_mutex; static std::unique_ptr sensitive_data_masker; public: diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8bd4d619349..d687b48a970 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -197,7 +197,6 @@ struct ContextSharedPart : boost::noncopyable mutable zkutil::ZooKeeperPtr zookeeper TSA_GUARDED_BY(zookeeper_mutex); /// Client for ZooKeeper. ConfigurationPtr zookeeper_config TSA_GUARDED_BY(zookeeper_mutex); /// Stores zookeeper configs - mutable std::mutex sensitive_data_masker_mutex; ConfigurationPtr sensitive_data_masker_config; #if USE_NURAFT @@ -3204,8 +3203,6 @@ bool Context::hasAuxiliaryZooKeeper(const String & name) const void Context::reloadQueryMaskingRulesIfChanged(const ConfigurationPtr & config) const { - std::lock_guard lock(shared->sensitive_data_masker_mutex); - const auto old_config = shared->sensitive_data_masker_config; if (old_config && isSameConfiguration(*config, *old_config, "query_masking_rules")) return; From 2fddc176428893867cff92ebbaaf3b91f7229b8c Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 13 Nov 2023 17:03:55 +0100 Subject: [PATCH 311/813] Add test --- tests/integration/test_config_substitutions/test.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/integration/test_config_substitutions/test.py b/tests/integration/test_config_substitutions/test.py index 692b36f1fae..d2a7a4a7ae4 100644 --- a/tests/integration/test_config_substitutions/test.py +++ b/tests/integration/test_config_substitutions/test.py @@ -30,6 +30,12 @@ node6 = cluster.add_instance( }, main_configs=["configs/include_from_source.xml"], ) +node7 = cluster.add_instance( + "node7", + user_configs=["configs/000-config_with_env_subst.xml", "configs/010-env_subst_override.xml"], + env_variables={"MAX_QUERY_SIZE": "121212"}, + instance_env_variables=True, +) # overridden with 424242 @pytest.fixture(scope="module") @@ -78,6 +84,10 @@ def test_config(start_cluster): node6.query("select value from system.settings where name = 'max_query_size'") == "99999\n" ) + assert ( + node7.query("select value from system.settings where name = 'max_query_size'") + == "424242\n" + ) def test_include_config(start_cluster): From 5bb4eb2383c9d6f98c2b49f1059ab4dab5ef3ec9 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 13 Nov 2023 17:04:26 +0100 Subject: [PATCH 312/813] Fix --- src/Common/Config/ConfigProcessor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index fe16313c0bf..3d64eac6fa6 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -326,6 +326,11 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, NodePtr new_node = config->importNode(with_node, true); config_root->replaceChild(new_node, config_node); } + else if (with_element.hasChildNodes() && with_element.firstChild()->nodeType() == Node::TEXT_NODE) + { + NodePtr new_node = config->importNode(with_node, true); + config_root->replaceChild(new_node, config_node); + } else { Element & config_element = dynamic_cast(*config_node); From cc3dfafd7b1b1203662c068aa120f87bf9584aae Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 13 Nov 2023 19:05:20 +0000 Subject: [PATCH 313/813] Automatic style fix --- tests/integration/test_config_substitutions/test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_config_substitutions/test.py b/tests/integration/test_config_substitutions/test.py index d2a7a4a7ae4..46961e5da71 100644 --- a/tests/integration/test_config_substitutions/test.py +++ b/tests/integration/test_config_substitutions/test.py @@ -32,10 +32,13 @@ node6 = cluster.add_instance( ) node7 = cluster.add_instance( "node7", - user_configs=["configs/000-config_with_env_subst.xml", "configs/010-env_subst_override.xml"], + user_configs=[ + "configs/000-config_with_env_subst.xml", + "configs/010-env_subst_override.xml", + ], env_variables={"MAX_QUERY_SIZE": "121212"}, instance_env_variables=True, -) # overridden with 424242 +) # overridden with 424242 @pytest.fixture(scope="module") From 3e6a5d82086842cf66431906a4fefdceb740648e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 14 Nov 2023 12:22:32 +0000 Subject: [PATCH 314/813] merge_row_policy: some comments --- src/Storages/StorageMerge.cpp | 2 ++ src/Storages/StorageMerge.h | 9 ++++++--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5d5c4b716ec..1eb93f13e9b 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -634,6 +634,8 @@ void ReadFromMerge::processAliases( storage_metadata_snapshot->getColumns().getAll(), context); column_expr = setAlias(column_expr, column); + /// use storage type for transient columns that are not represented in result + /// e.g. for columns that needed to evaluate row policy auto type = sample_block.has(column) ? sample_block.getByName(column).type : column_description.type; aliases.push_back({ .name = column, .type = type, .expression = column_expr->clone() }); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 71680b90072..d6b4c5d8ad2 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -170,9 +170,9 @@ private: struct AliasData { - String name; - DataTypePtr type; - ASTPtr expression; + String name; /// "size" in "size String Alias formatReadableSize(size_bytes)" + DataTypePtr type; /// String in "size String Alias formatReadableSize(size_bytes)", or something different came from query + ASTPtr expression; /// formatReadableSize(size_bytes) in "size String Alias formatReadableSize(size_bytes)" }; using Aliases = std::vector; @@ -184,6 +184,9 @@ private: const StorageWithLockAndName & storage_with_lock_and_name, const StorageSnapshotPtr & storage_snapshot); + /// Populates AliasData structures for further processing + /// using types from result query if possible + /// and removes alias columns from real_column_names void processAliases( Names & real_column_names, const StorageWithLockAndName & storage_with_lock, From d5907e10de95bc9a3e50f877338fae29dfc3d3ea Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Nov 2023 12:51:25 +0000 Subject: [PATCH 315/813] Fixing tests. --- .../PredicateExpressionsOptimizer.cpp | 4 ++- src/Storages/StorageInput.cpp | 25 +++++++++++++------ 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index 885c99aeb90..8dc8c1c92cc 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -84,8 +84,10 @@ std::vector PredicateExpressionsOptimizer::extractTablesPredicates(const A return {}; /// Not optimized when predicate contains stateful function or indeterministic function or window functions } + /// Skip predicate like `... IN (SELECT ... FROM input())` because + /// it can be duplicated but we can't execute `input()` twice. if (hasInputTableFunction(predicate_expression)) - return {}; /// Not optimized when predicate contains input table function + return {}; if (!expression_info.is_array_join) { diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 2314d3fb581..7f1eeaedfb1 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include #include @@ -60,16 +61,16 @@ public: ReadFromInput( Block sample_block, - //StorageSnapshotPtr storage_snapshot_, + Pipe pipe_, StorageInput & storage_) : ISourceStep(DataStream{.header = std::move(sample_block)}) - //, storage_snapshot(std::move(storage_snapshot_)) + , pipe(std::move(pipe_)) , storage(storage_) { } private: - //StorageSnapshotPtr storage_snapshot; + Pipe pipe; StorageInput & storage; }; @@ -85,21 +86,20 @@ void StorageInput::read( { storage_snapshot->check(column_names); Block sample_block = storage_snapshot->metadata->getSampleBlock(); + Pipe input_source_pipe; auto query_context = context->getQueryContext(); /// It is TCP request if we have callbacks for input(). - if (!was_pipe_initialized && query_context->getInputBlocksReaderCallback()) + if (query_context->getInputBlocksReaderCallback()) { /// Send structure to the client. query_context->initializeInput(shared_from_this()); + input_source_pipe = Pipe(std::make_shared(query_context, sample_block)); } - if (!was_pipe_initialized) - throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "Input stream is not initialized, input() must be used only in INSERT SELECT query"); - auto reading = std::make_unique( std::move(sample_block), - //storage_snapshot, + std::move(input_source_pipe), *this); query_plan.addStep(std::move(reading)); @@ -107,6 +107,15 @@ void StorageInput::read( void ReadFromInput::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + if (!pipe.empty()) + { + pipeline.init(std::move(pipe)); + return; + } + + if (!storage.was_pipe_initialized) + throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "Input stream is not initialized, input() must be used only in INSERT SELECT query"); + if (storage.was_pipe_used) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to read from input() twice."); From 22a9f71393e0087ed45f28b81a007a153b3f5cd7 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 14 Nov 2023 13:15:14 +0000 Subject: [PATCH 316/813] fixed due to review --- .../Formats/Impl/NpyRowInputFormat.cpp | 39 +++++++++--------- .../Formats/Impl/NpyRowInputFormat.h | 5 ++- .../0_stateless/02895_npy_format.reference | 3 ++ tests/queries/0_stateless/02895_npy_format.sh | 2 + .../0_stateless/data_npy/npy_inf_nan_null.npy | Bin 0 -> 134 bytes 5 files changed, 29 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/data_npy/npy_inf_nan_null.npy diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index 9e985dffe0c..a7fb15fa375 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -32,6 +32,8 @@ namespace float convertFloat16ToFloat32(uint16_t float16_value) { + if (float16_value == 0000000000000000) + return float(0); uint16_t sign = (float16_value >> 15) & 0x1; uint16_t exponent = (float16_value >> 10) & 0x1F; uint16_t fraction = float16_value & 0x3FF; @@ -298,26 +300,25 @@ NpyRowInputFormat::NpyRowInputFormat(ReadBuffer & in_, Block header_, Params par } template -void NpyRowInputFormat::readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness, bool isFloat16) +void NpyRowInputFormat::readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness) { - if (!isFloat16) - { - DataValue value; - if (endianness == NumpyDataType::Endianness::BIG) - readBinaryBigEndian(value, *in); - else - readBinaryLittleEndian(value, *in); - assert_cast &>(*column).insertValue((static_cast(value))); - } + DataValue value; + if (endianness == NumpyDataType::Endianness::BIG) + readBinaryBigEndian(value, *in); else - { - uint16_t value; - if (endianness == NumpyDataType::Endianness::BIG) - readBinaryBigEndian(value, *in); - else - readBinaryLittleEndian(value, *in); - assert_cast &>(*column).insertValue(static_cast(convertFloat16ToFloat32(value))); - } + readBinaryLittleEndian(value, *in); + assert_cast &>(*column).insertValue((static_cast(value))); +} + +template +void NpyRowInputFormat::readBinaryValueAndInsertFloat16(MutableColumnPtr column, NumpyDataType::Endianness endianness) +{ + uint16_t value; + if (endianness == NumpyDataType::Endianness::BIG) + readBinaryBigEndian(value, *in); + else + readBinaryLittleEndian(value, *in); + assert_cast &>(*column).insertValue(static_cast(convertFloat16ToFloat32(value))); } template @@ -344,7 +345,7 @@ void NpyRowInputFormat::readAndInsertFloat(IColumn * column, const DataTypePtr & { switch (npy_type.getTypeIndex()) { - case NumpyDataTypeIndex::Float16: readBinaryValueAndInsert(column->getPtr(), npy_type.getEndianness(), true); break; + case NumpyDataTypeIndex::Float16: readBinaryValueAndInsertFloat16(column->getPtr(), npy_type.getEndianness()); break; case NumpyDataTypeIndex::Float32: readBinaryValueAndInsert(column->getPtr(), npy_type.getEndianness()); break; case NumpyDataTypeIndex::Float64: readBinaryValueAndInsert(column->getPtr(), npy_type.getEndianness()); break; default: diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.h b/src/Processors/Formats/Impl/NpyRowInputFormat.h index 2bc28121d11..358dac72f5a 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.h +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.h @@ -43,7 +43,10 @@ private: void readAndInsertString(MutableColumnPtr column, const DataTypePtr & data_type, const NumpyDataType & npy_type, bool is_fixed); template - void readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness, bool isFloat16 = false); + void readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness); + + template + void readBinaryValueAndInsertFloat16(MutableColumnPtr column, NumpyDataType::Endianness endianness); void readRows(MutableColumns & columns); diff --git a/tests/queries/0_stateless/02895_npy_format.reference b/tests/queries/0_stateless/02895_npy_format.reference index e1c168d1c67..76c8a7a2abf 100644 --- a/tests/queries/0_stateless/02895_npy_format.reference +++ b/tests/queries/0_stateless/02895_npy_format.reference @@ -86,3 +86,6 @@ c 1 [2.199219,1.099609,3.300781] [4.25,3.34961,6.628906] +inf +nan +0 diff --git a/tests/queries/0_stateless/02895_npy_format.sh b/tests/queries/0_stateless/02895_npy_format.sh index b60f324467d..c4fb2e2f67d 100755 --- a/tests/queries/0_stateless/02895_npy_format.sh +++ b/tests/queries/0_stateless/02895_npy_format.sh @@ -58,3 +58,5 @@ $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_unicode.npy', $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/complex.npy')" 2>&1 | grep -c "BAD_ARGUMENTS" $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/float_16.npy')" + +$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/npy_inf_nan_null.npy')" diff --git a/tests/queries/0_stateless/data_npy/npy_inf_nan_null.npy b/tests/queries/0_stateless/data_npy/npy_inf_nan_null.npy new file mode 100644 index 0000000000000000000000000000000000000000..12ee359f6658be20810b5c40a53874753eed1280 GIT binary patch literal 134 zcmbR27wQ`j$;eQ~P_3SlTAW;@Zl$1ZlV+r@qoAIaUsO_*m=~X4l#&V(cT3DEP6dh= bXCxM+0{I%oI+{8PwF(pfE`}P0ItB&+`TQJG literal 0 HcmV?d00001 From 9ce9dc86d38662aa4163a5047047639b88c21945 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 14 Nov 2023 13:27:32 +0000 Subject: [PATCH 317/813] fixed due to review --- .../02908_Npy_files_caching.reference | 6 +++++- .../0_stateless/02908_Npy_files_caching.sh | 6 ++++++ .../queries/0_stateless/data_npy/npy_big.npy | Bin 0 -> 8000128 bytes 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/data_npy/npy_big.npy diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.reference b/tests/queries/0_stateless/02908_Npy_files_caching.reference index 6159dd69b57..5f5e5f6282d 100644 --- a/tests/queries/0_stateless/02908_Npy_files_caching.reference +++ b/tests/queries/0_stateless/02908_Npy_files_caching.reference @@ -2,4 +2,8 @@ 3 3 array Int64 -3 +\N +1000000 +1000000 +array Int64 +\N diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.sh b/tests/queries/0_stateless/02908_Npy_files_caching.sh index f77351f1180..4845f740972 100755 --- a/tests/queries/0_stateless/02908_Npy_files_caching.sh +++ b/tests/queries/0_stateless/02908_Npy_files_caching.sh @@ -11,3 +11,9 @@ $CLICKHOUSE_LOCAL -nm -q " desc file('$CURDIR/data_npy/one_dim.npy'); select number_of_rows from system.schema_inference_cache where format='Npy'; " +$CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/npy_big.npy') settings optimize_count_from_files=0" +$CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/npy_big.npy') settings optimize_count_from_files=1" +$CLICKHOUSE_LOCAL -nm -q " +desc file('$CURDIR/data_npy/npy_big.npy'); +select number_of_rows from system.schema_inference_cache where format='Npy'; +" diff --git a/tests/queries/0_stateless/data_npy/npy_big.npy b/tests/queries/0_stateless/data_npy/npy_big.npy new file mode 100644 index 0000000000000000000000000000000000000000..7dc3c2bf6009c9f3c09d70779e4c35a45c9ac111 GIT binary patch literal 8000128 zcmeFxu?_)26a~=L`ikiW*@8|)ts@$RN;Y$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8 zaDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0W zzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N@sGojM=^0000G^#64`0v(|TJm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b z4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw% zJm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5 z@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVK zzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_ z0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=> z9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kj zc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A z-~kVKzyluefCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzylue zfCoI_0S|b<10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b< z10L{z2Rz^b4|u=>9`Jw%Jm3Kjc)$Z5@PG$A-~kVKzyluefCoI_0S|b<10L{z2Rz^b T4|u=>9`Jw%Jm3Kjc;NX!bS59p literal 0 HcmV?d00001 From bd52e9833e1935caaa4b6cd26b9d002363d4a75a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 14 Nov 2023 14:33:20 +0000 Subject: [PATCH 318/813] Try to fix stateful test --- tests/queries/1_stateful/00178_gcd_codec.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/1_stateful/00178_gcd_codec.sql b/tests/queries/1_stateful/00178_gcd_codec.sql index 443075b1b24..04715939d15 100644 --- a/tests/queries/1_stateful/00178_gcd_codec.sql +++ b/tests/queries/1_stateful/00178_gcd_codec.sql @@ -7,7 +7,7 @@ CREATE TABLE hits_gcd (`WatchID` UInt64 CODEC (GCD,LZ4), `JavaEnable` UInt8 CODE SAMPLE BY intHash32(UserID); -INSERT INTO hits_gcd SELECT * FROM hits; +INSERT INTO hits_gcd SELECT * FROM test.hits; SELECT * FROM hits_gcd FORMAT Null; -DROP TABLE IF EXISTS hits_gcd; \ No newline at end of file +DROP TABLE IF EXISTS hits_gcd; From 2f9ac9b49cc8b2f8fbd7e824138855f73403ce38 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 14 Nov 2023 14:33:34 +0000 Subject: [PATCH 319/813] Address comments --- src/Backups/BackupCoordinationFileInfos.cpp | 81 +++++++++++++++++-- src/Backups/BackupFileInfo.h | 4 + src/Backups/BackupIO.h | 2 + src/Backups/BackupIO_Default.cpp | 2 + src/Backups/BackupIO_Disk.cpp | 9 +++ src/Backups/BackupIO_Disk.h | 2 + src/Backups/BackupIO_File.cpp | 10 +++ src/Backups/BackupIO_File.h | 2 + src/Backups/BackupIO_S3.cpp | 18 +++++ src/Backups/BackupIO_S3.h | 2 + src/Backups/BackupImpl.cpp | 27 ++----- .../test_backup_restore_keeper_map/test.py | 28 ++++--- 12 files changed, 149 insertions(+), 38 deletions(-) diff --git a/src/Backups/BackupCoordinationFileInfos.cpp b/src/Backups/BackupCoordinationFileInfos.cpp index ac2cbc337aa..b17b755b966 100644 --- a/src/Backups/BackupCoordinationFileInfos.cpp +++ b/src/Backups/BackupCoordinationFileInfos.cpp @@ -80,24 +80,76 @@ void BackupCoordinationFileInfos::prepare() const if (plain_backup) { + std::vector unresolved_references; + std::unordered_map file_name_to_info; + + const auto try_resolve_reference = [&](BackupFileInfo & reference) + { + auto it = file_name_to_info.find(reference.reference_target); + + if (it == file_name_to_info.end()) + return false; + + auto & target_info = it->second; + target_info->reference_sources.push_back(reference.file_name); + reference.size = target_info->size; + total_size_of_files += reference.size; + reference.checksum = target_info->checksum; + return true; + }; + /// For plain backup all file infos are stored as is, without checking for duplicates or skipping empty files. for (size_t i = 0; i != file_infos_for_all_hosts.size(); ++i) { auto & info = *(file_infos_for_all_hosts[i]); - - if (!info.reference_target.empty()) - continue; - info.data_file_name = info.file_name; info.data_file_index = i; info.base_size = 0; /// Base backup must not be used while creating a plain backup. info.base_checksum = 0; - total_size_of_files += info.size; + + if (info.reference_target.empty()) + { + file_name_to_info.emplace(info.file_name, &info); + total_size_of_files += info.size; + } + else if (!try_resolve_reference(info)) + { + unresolved_references.push_back(&info); + } } + + for (auto * reference : unresolved_references) + { + if (!try_resolve_reference(*reference)) + throw DB::Exception( + ErrorCodes::LOGICAL_ERROR, + "Couldn't resolve reference {} with target {}", + reference->file_name, + reference->reference_target); + } + num_files = file_infos_for_all_hosts.size(); } else { + std::vector unresolved_references; + std::unordered_map file_name_to_info; + + const auto try_resolve_reference = [&](BackupFileInfo & reference) + { + auto it = file_name_to_info.find(reference.reference_target); + + if (it == file_name_to_info.end()) + return false; + + auto & target_info = it->second; + reference.size = target_info->size; + reference.checksum = target_info->checksum; + reference.data_file_name = target_info->data_file_name; + reference.data_file_index = target_info->data_file_index; + return true; + }; + /// For non-plain backups files with the same size and checksum are stored only once, /// in order to find those files we'll use this map. std::map data_file_index_by_checksum; @@ -107,7 +159,12 @@ void BackupCoordinationFileInfos::prepare() const auto & info = *(file_infos_for_all_hosts[i]); if (!info.reference_target.empty()) + { + if (!try_resolve_reference(info)) + unresolved_references.push_back(&info); + continue; + } if (info.size == info.base_size) { @@ -134,7 +191,21 @@ void BackupCoordinationFileInfos::prepare() const info.data_file_name = file_infos_for_all_hosts[it->second]->data_file_name; } } + + file_name_to_info.emplace(info.file_name, &info); } + + for (auto * reference : unresolved_references) + { + if (!try_resolve_reference(*reference)) + throw DB::Exception( + ErrorCodes::LOGICAL_ERROR, + "Couldn't resolve reference {} with target {}", + reference->file_name, + reference->reference_target); + } + + num_files = file_infos_for_all_hosts.size(); } prepared = true; diff --git a/src/Backups/BackupFileInfo.h b/src/Backups/BackupFileInfo.h index 1d5607fd418..42bda3aa6ed 100644 --- a/src/Backups/BackupFileInfo.h +++ b/src/Backups/BackupFileInfo.h @@ -42,6 +42,10 @@ struct BackupFileInfo /// Set if this file is just a reference to another file String reference_target; + /// List of files that are referencing this file + /// Used for plain backup which needs to resolve all references + Strings reference_sources; + struct LessByFileName { bool operator()(const BackupFileInfo & lhs, const BackupFileInfo & rhs) const { return (lhs.file_name < rhs.file_name); } diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index e4a82a604e8..91d57e5ab0a 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -61,6 +61,8 @@ public: virtual void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) = 0; + virtual void copyFile(const String & destination, const String & source, size_t size) = 0; + virtual void removeFile(const String & file_name) = 0; virtual void removeFiles(const Strings & file_names) = 0; diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp index 5ac522695ce..95f2c66b6b9 100644 --- a/src/Backups/BackupIO_Default.cpp +++ b/src/Backups/BackupIO_Default.cpp @@ -91,4 +91,6 @@ void BackupWriterDefault::copyFileFromDisk(const String & path_in_backup, DiskPt copyDataToFile(path_in_backup, create_read_buffer, start_pos, length); } + + } diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 1e260ad22d9..91e8b97bc20 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -128,4 +128,13 @@ void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr s BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } +void BackupWriterDisk::copyFile(const String & destination, const String & source, size_t /*size*/) +{ + LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); + auto dest_file_path = root_path / destination; + auto src_file_path = root_path / source; + disk->createDirectories(dest_file_path.parent_path()); + disk->copyFile(src_file_path, *disk, dest_file_path, read_settings, write_settings); +} + } diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index 70d31eacc1a..575ec3f5707 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -44,6 +44,8 @@ public: void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void copyFile(const String & destination, const String & source, size_t size) override; + void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 2bedb5470fb..5384637a969 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -152,4 +152,14 @@ void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr s BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } +void BackupWriterFile::copyFile(const String & destination, const String & source, size_t /*size*/) +{ + LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); + + auto abs_source_path = root_path / source; + auto abs_dest_path = root_path / destination; + fs::create_directories(abs_dest_path.parent_path()); + fs::copy(abs_source_path, abs_dest_path, fs::copy_options::overwrite_existing); +} + } diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index 6bb4b11e134..ebe9a0f02cb 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -38,6 +38,8 @@ public: void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void copyFile(const String & destination, const String & source, size_t size) override; + void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 8bb2f895e38..9688d7f0730 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -249,6 +249,24 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } +void BackupWriterS3::copyFile(const String & destination, const String & source, size_t size) +{ + LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); + copyS3File( + client, + client, + /* src_bucket */ s3_uri.bucket, + /* src_key= */ fs::path(s3_uri.key) / source, + 0, + size, + s3_uri.bucket, + fs::path(s3_uri.key) / destination, + s3_settings.request_settings, + read_settings, + {}, + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); +} + void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { copyDataToS3File(create_read_buffer, start_pos, length, client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, s3_settings.request_settings, {}, diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 4abcbedf89f..c00ce747ff5 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -49,6 +49,8 @@ public: void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void copyFile(const String & destination, const String & source, size_t size) override; + void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index bf1853828df..56c30fab5c2 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -362,10 +362,10 @@ void BackupImpl::writeBackupMetadata() *out << ""; *out << "" << xml << info.file_name << ""; + *out << "" << info.size << ""; if (info.size) { - *out << "" << info.size << ""; *out << "" << hexChecksum(info.checksum) << ""; if (info.base_size) { @@ -381,10 +381,6 @@ void BackupImpl::writeBackupMetadata() if (info.encrypted_by_disk) *out << "true"; } - else if (!info.reference_target.empty()) - *out << "" << xml << info.reference_target << ""; - else - *out << "" << info.size << ""; total_size += info.size; bool has_entry = !deduplicate_files || (info.size && (info.size != info.base_size) && (info.data_file_name.empty() || (info.data_file_name == info.file_name))); @@ -465,13 +461,6 @@ void BackupImpl::readBackupMetadata() BackupFileInfo info; info.file_name = getString(file_config, "name"); - info.reference_target = getString(file_config, "reference_target", ""); - if (!info.reference_target.empty()) - { - reference_files.emplace_back(std::move(info.file_name), std::move(info.reference_target)); - continue; - } - info.size = getUInt64(file_config, "size"); if (info.size) { @@ -521,14 +510,6 @@ void BackupImpl::readBackupMetadata() } } - for (auto & [source_file, target_file] : reference_files) - { - auto it = file_names.find(target_file); - if (it == file_names.end()) - throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup entry {} referenced by {} not found", target_file, source_file); - file_names.emplace(std::move(source_file), it->second); - } - uncompressed_size = size_of_entries + str.size(); compressed_size = uncompressed_size; if (!use_archive) @@ -954,6 +935,12 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) writer->copyDataToFile(info.data_file_name, create_read_buffer, info.base_size, info.size - info.base_size); } + if (!deduplicate_files) + { + for (const auto & reference : info.reference_sources) + writer->copyFile(reference, info.data_file_name, info.size - info.base_size); + } + { std::lock_guard lock{mutex}; ++num_entries; diff --git a/tests/integration/test_backup_restore_keeper_map/test.py b/tests/integration/test_backup_restore_keeper_map/test.py index 8343ad3177f..c401f482c3f 100644 --- a/tests/integration/test_backup_restore_keeper_map/test.py +++ b/tests/integration/test_backup_restore_keeper_map/test.py @@ -65,22 +65,24 @@ def new_backup_name(base_name): return f"Disk('backups', '{base_name}{backup_id_counter}')" -def test_on_cluster(): - node1.query_with_retry("CREATE DATABASE keeper_backup ON CLUSTER cluster") +@pytest.mark.parametrize("deduplicate_files", [0, 1]) +def test_on_cluster(deduplicate_files): + database_name = f"keeper_backup{deduplicate_files}" + node1.query_with_retry(f"CREATE DATABASE {database_name} ON CLUSTER cluster") node1.query_with_retry( - "CREATE TABLE keeper_backup.keeper1 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster1') PRIMARY KEY key" + f"CREATE TABLE {database_name}.keeper1 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/{database_name}/test_on_cluster1') PRIMARY KEY key" ) node1.query_with_retry( - "CREATE TABLE keeper_backup.keeper2 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster1') PRIMARY KEY key" + f"CREATE TABLE {database_name}.keeper2 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/{database_name}/test_on_cluster1') PRIMARY KEY key" ) node1.query_with_retry( - "CREATE TABLE keeper_backup.keeper3 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/test_on_cluster2') PRIMARY KEY key" + f"CREATE TABLE {database_name}.keeper3 ON CLUSTER cluster (key UInt64, value String) Engine=KeeperMap('/{database_name}/test_on_cluster2') PRIMARY KEY key" ) node1.query_with_retry( - "INSERT INTO keeper_backup.keeper2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5" + f"INSERT INTO {database_name}.keeper2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5" ) node1.query_with_retry( - "INSERT INTO keeper_backup.keeper3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5" + f"INSERT INTO {database_name}.keeper3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5" ) expected_result = "".join(f"{i}\ttest{i}\n" for i in range(5)) @@ -89,7 +91,7 @@ def test_on_cluster(): for node in [node1, node2, node3]: for i in range(1, 4): result = node.query_with_retry( - f"SELECT key, value FROM keeper_backup.keeper{i} ORDER BY key FORMAT TSV" + f"SELECT key, value FROM {database_name}.keeper{i} ORDER BY key FORMAT TSV" ) assert result == expected_result @@ -97,10 +99,10 @@ def test_on_cluster(): backup_name = new_backup_name("test_on_cluster") node1.query( - f"BACKUP DATABASE keeper_backup ON CLUSTER cluster TO {backup_name} SETTINGS async = false;" + f"BACKUP DATABASE {database_name} ON CLUSTER cluster TO {backup_name} SETTINGS async = false, deduplicate_files = {deduplicate_files};" ) - node1.query("DROP DATABASE keeper_backup ON CLUSTER cluster SYNC;") + node1.query(f"DROP DATABASE {database_name} ON CLUSTER cluster SYNC;") def apply_for_all_nodes(f): for node in [node1, node2, node3]: @@ -121,14 +123,14 @@ def test_on_cluster(): apply_for_all_nodes(lambda node: node.start_clickhouse()) node1.query( - f"RESTORE DATABASE keeper_backup ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;" + f"RESTORE DATABASE {database_name} ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;" ) verify_data() - node1.query("DROP TABLE keeper_backup.keeper3 ON CLUSTER cluster SYNC;") + node1.query(f"DROP TABLE {database_name}.keeper3 ON CLUSTER cluster SYNC;") node1.query( - f"RESTORE TABLE keeper_backup.keeper3 ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;" + f"RESTORE TABLE {database_name}.keeper3 ON CLUSTER cluster FROM {backup_name} SETTINGS async = false;" ) verify_data() From ecef6db1fe862e561edd22148da0c3ff43f150ba Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Tue, 14 Nov 2023 15:36:50 +0100 Subject: [PATCH 320/813] Fix: RabbitMQ OpenSSL dynamic loading issue (#56703) * Fix: RabbitMQ OpenSSL dynamic loading issue * Update RabbitMQ * Fix minor glitch with TLS_client_method() https://github.com/ClickHouse/AMQP-CPP/commit/00f09897ce020a84e38f87dc416af4a19c5da9ae --------- Co-authored-by: Robert Schulze --- contrib/AMQP-CPP | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/AMQP-CPP b/contrib/AMQP-CPP index 818c2d8ad96..00f09897ce0 160000 --- a/contrib/AMQP-CPP +++ b/contrib/AMQP-CPP @@ -1 +1 @@ -Subproject commit 818c2d8ad96a08a5d20fece7d1e1e8855a2b0860 +Subproject commit 00f09897ce020a84e38f87dc416af4a19c5da9ae From 094eebde4fd783023887267227e1837483204d40 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 8 Nov 2023 13:14:05 +0100 Subject: [PATCH 321/813] Update the runner version --- tests/ci/worker/prepare-ci-ami.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 20e7e3fd53e..35a98b8e7d1 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -9,7 +9,7 @@ set -xeuo pipefail echo "Running prepare script" export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.304.0 +export RUNNER_VERSION=2.311.0 export RUNNER_HOME=/home/ubuntu/actions-runner deb_arch() { From 0c0f70d26d822ca85166fad6b27c847c55cd5095 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 8 Nov 2023 13:14:46 +0100 Subject: [PATCH 322/813] Add the installing of azure cli --- tests/ci/worker/prepare-ci-ami.sh | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 35a98b8e7d1..7aec9c5dfdd 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -56,12 +56,12 @@ apt-get install --yes --no-install-recommends \ qemu-user-static \ unzip +# Install docker curl -fsSL https://download.docker.com/linux/ubuntu/gpg | gpg --dearmor -o /usr/share/keyrings/docker-archive-keyring.gpg echo "deb [arch=$(deb_arch) signed-by=/usr/share/keyrings/docker-archive-keyring.gpg] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable" | tee /etc/apt/sources.list.d/docker.list > /dev/null apt-get update - apt-get install --yes --no-install-recommends docker-ce docker-buildx-plugin docker-ce-cli containerd.io usermod -aG docker ubuntu @@ -81,6 +81,14 @@ cat < /etc/docker/daemon.json } EOT +# Install azure-cli +curl -sLS https://packages.microsoft.com/keys/microsoft.asc | gpg --dearmor -o /etc/apt/keyrings/microsoft.gpg +AZ_DIST=$(lsb_release -cs) +echo "deb [arch=$(dpkg --print-architecture) signed-by=/etc/apt/keyrings/microsoft.gpg] https://packages.microsoft.com/repos/azure-cli/ $AZ_DIST main" | tee /etc/apt/sources.list.d/azure-cli.list + +apt-get update +apt-get install --yes --no-install-recommends azure-cli + # Increase the limit on number of virtual memory mappings to aviod 'Cannot mmap' error echo "vm.max_map_count = 2097152" > /etc/sysctl.d/01-increase-map-counts.conf From e77c9fbd53ee89febe788600a47a9667b7457409 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 8 Nov 2023 14:02:28 +0100 Subject: [PATCH 323/813] Add TOE in the comment, find and fix possible issues for the second run --- tests/ci/worker/prepare-ci-ami.sh | 43 +++++++++++++++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 7aec9c5dfdd..c27d956c834 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -96,10 +96,12 @@ systemctl restart docker # buildx builder is user-specific sudo -u ubuntu docker buildx version +sudo -u ubuntu docker buildx rm default-builder || : # if it's the second attempt sudo -u ubuntu docker buildx create --use --name default-builder pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory +rm -rf $RUNNER_HOME # if it's the second attempt mkdir -p $RUNNER_HOME && cd $RUNNER_HOME RUNNER_ARCHIVE="actions-runner-linux-$(runner_arch)-$RUNNER_VERSION.tar.gz" @@ -138,3 +140,44 @@ systemctl enable amazon-cloudwatch-agent.service # The following line is used in aws TOE check. touch /var/tmp/clickhouse-ci-ami.success +# END OF THE SCRIPT + +# TOE description +# name: CIInfrastructurePrepare +# description: instals the infrastructure for ClickHouse CI runners +# schemaVersion: 1.0 +# +# phases: +# - name: build +# steps: +# - name: DownloadRemoteScript +# maxAttempts: 3 +# action: WebDownload +# onFailure: Abort +# inputs: +# - source: https://github.com/ClickHouse/ClickHouse/raw/653da5f00219c088af66d97a8f1ea3e35e798268/tests/ci/worker/prepare-ci-ami.sh +# destination: /tmp/prepare-ci-ami.sh +# - name: RunScript +# maxAttempts: 3 +# action: ExecuteBash +# onFailure: Abort +# inputs: +# commands: +# - bash -x '{{build.DownloadRemoteScript.inputs[0].destination}}' +# +# +# - name: validate +# steps: +# - name: RunScript +# maxAttempts: 3 +# action: ExecuteBash +# onFailure: Abort +# inputs: +# commands: +# - ls /var/tmp/clickhouse-ci-ami.success +# - name: Cleanup +# action: DeleteFile +# onFailure: Abort +# maxAttempts: 3 +# inputs: +# - path: /var/tmp/clickhouse-ci-ami.success From 5f9704a3c1485b6297e42405141091526def82a4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 13 Nov 2023 16:39:48 +0100 Subject: [PATCH 324/813] Kill forgotten Runner.Listener too --- tests/ci/worker/init_runner.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index d452c985407..eed4bfdd223 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -104,7 +104,7 @@ check_proceed_spot_termination() { if [ -n "$runner_pid" ]; then # Kill the runner to not allow it cancelling the job # shellcheck disable=SC2046 - kill -9 $(list_children "$runner_pid") + kill -9 "$runner_pid" $(list_children "$runner_pid") fi sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" terminate_and_exit From b4cc55ea34aceb664b95259db77030bdcf20b82f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 14 Nov 2023 12:48:27 +0100 Subject: [PATCH 325/813] Deploy cloud-init script to S3 for following usage --- tests/ci/worker/.gitignore | 1 + tests/ci/worker/deploy-runner-init.sh | 85 +++++++++++++++++++++++++++ tests/ci/worker/init_runner.sh | 39 ++++++++++++ 3 files changed, 125 insertions(+) create mode 100644 tests/ci/worker/.gitignore create mode 100755 tests/ci/worker/deploy-runner-init.sh diff --git a/tests/ci/worker/.gitignore b/tests/ci/worker/.gitignore new file mode 100644 index 00000000000..4ed18989e78 --- /dev/null +++ b/tests/ci/worker/.gitignore @@ -0,0 +1 @@ +generated_*init_runner.sh diff --git a/tests/ci/worker/deploy-runner-init.sh b/tests/ci/worker/deploy-runner-init.sh new file mode 100755 index 00000000000..06edede48fa --- /dev/null +++ b/tests/ci/worker/deploy-runner-init.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash + +usage() { + echo "Usage: $0 ENVIRONMENT" >&2 + echo "Valid values for ENVIRONMENT: staging, production" >&2 + exit 1 +} + +case "$1" in + staging|production) + ENVIRONMENT="$1" ;; + --help) + usage ;; + *) + echo "Invalid argument" >&2 + usage ;; +esac + +cd "$(dirname "$0")" || exit 1 +SOURCE_SCRIPT='init_runner.sh' + +check_response() { + # Are we even in the interactive shell? + [ -t 1 ] || return 1 + local request + request="$1" + read -rp "$request (y/N): " response + case "$response" in + [Yy]) + return 0 + # Your code to continue goes here + ;; + *) + return 1 + ;; + esac +} + +check_dirty() { + if [ -n "$(git status --porcelain=v2 "$SOURCE_SCRIPT")" ]; then + echo "The $SOURCE_SCRIPT has uncommited changes, won't deploy it" >&2 + exit 1 + fi +} +GIT_HASH=$(git log -1 --format=format:%H) + +header() { + cat << EOF +#!/usr/bin/env bash + +echo 'The $ENVIRONMENT script is generated from $SOURCE_SCRIPT, commit $GIT_HASH' + +EOF +} + +body() { + local first_line + first_line=$(sed -n '/^# THE SCRIPT START$/{=;q}' "$SOURCE_SCRIPT") + if [ -z "$first_line" ]; then + echo "The pattern '# THE SCRIPT START' is not found in $SOURCE_SCRIPT" >&2 + exit 1 + fi + tail "+$first_line" "$SOURCE_SCRIPT" +} + +GENERATED_FILE="generated_${ENVIRONMENT}_${SOURCE_SCRIPT}" + +{ header && body; } > "$GENERATED_FILE" + +echo "The file $GENERATED_FILE is generated" + +if check_response "Display the content of $GENERATED_FILE?"; then + if [ -z "$PAGER" ]; then + less "$GENERATED_FILE" + else + $PAGER "$GENERATED_FILE" + fi +fi + +check_dirty + +S3_OBJECT=${S3_OBJECT:-s3://github-runners-data/cloud-init/${ENVIRONMENT}.sh} +if check_response "Deploy the generated script to $S3_OBJECT?"; then + aws s3 mv "$GENERATED_FILE" "$S3_OBJECT" +fi diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index eed4bfdd223..5d7b009bf3d 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -1,4 +1,43 @@ #!/usr/bin/env bash + +cat > /dev/null << 'EOF' +The following content is embedded into the s3 object via the script +deploy-runner-init.sh {staging,production} +with additional helping information + +In the `user data` you should define as the following +with appropriate as 'staging' or 'production': + +### COPY AFTER +Content-Type: multipart/mixed; boundary="//" +MIME-Version: 1.0 + +--// +Content-Type: text/cloud-config; charset="us-ascii" +MIME-Version: 1.0 +Content-Transfer-Encoding: 7bit +Content-Disposition: attachment; filename="cloud-config.txt" + +#cloud-config +cloud_final_modules: +- [scripts-user, always] + +--// +Content-Type: text/x-shellscript; charset="us-ascii" +MIME-Version: 1.0 +Content-Transfer-Encoding: 7bit +Content-Disposition: attachment; filename="userdata.txt" + +#!/bin/bash +aws s3 cp s3://github-runners-data/cloud-init/.sh /tmp/cloud-init.sh +chmod 0700 /tmp/cloud-init.sh +exec bash /tmp/cloud-init.sh +--// +### COPY BEFORE +EOF + +# THE SCRIPT START + set -uo pipefail #################################### From 556e0faad2bd314a5d6d814fc7eb69c21aa26062 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 14 Nov 2023 13:25:02 +0100 Subject: [PATCH 326/813] Make sleep in init_runner loop global --- tests/ci/worker/init_runner.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 5d7b009bf3d..ad79d398b95 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -314,7 +314,7 @@ while true; do ACTIONS_RUNNER_HOOK_JOB_STARTED=/tmp/actions-hooks/pre-run.sh \ ACTIONS_RUNNER_HOOK_JOB_COMPLETED=/tmp/actions-hooks/post-run.sh \ ./run.sh & - sleep 15 + sleep 10 else echo "Runner is working with pid $runner_pid, checking the metadata in background" check_proceed_spot_termination @@ -330,8 +330,8 @@ while true; do terminate_and_exit fi fi - sleep 5 fi + sleep 5 done # vim:ts=4:sw=4 From 8fbc0d3d519ad986ed6fcc3f6b28f4cb9e9a0f83 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 14 Nov 2023 15:47:47 +0100 Subject: [PATCH 327/813] Disable autoupdate for GH runners to speed-up boot --- tests/ci/worker/init_runner.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index ad79d398b95..a2737575955 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -302,7 +302,8 @@ while true; do check_proceed_spot_termination echo "Going to configure runner" - sudo -u ubuntu ./config.sh --url $RUNNER_URL --token "$(get_runner_token)" --ephemeral \ + sudo -u ubuntu ./config.sh --url $RUNNER_URL --token "$(get_runner_token)" \ + --ephemeral --disableupdate --unattended \ --runnergroup Default --labels "$LABELS" --work _work --name "$INSTANCE_ID" echo "Another one check to avoid race between runner and infrastructure" From 1356dc2eaa76ca63d7b6ff982201fe0bd26550bc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Nov 2023 14:55:21 +0000 Subject: [PATCH 328/813] Fixing style. --- src/Storages/StorageInput.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 7f1eeaedfb1..4c319ed9414 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { extern const int INVALID_USAGE_OF_INPUT; + extern const int LOGICAL_ERROR; } StorageInput::StorageInput(const StorageID & table_id, const ColumnsDescription & columns_) From e8eb425ff86ffd77a7f646401ebf600742d57cc5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 14 Nov 2023 15:13:37 +0000 Subject: [PATCH 329/813] Try to fix new calamities with cross-compilation --- contrib/google-protobuf-cmake/CMakeLists.txt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index fc5bf2c0241..f6955a3d8ce 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -20,7 +20,6 @@ endif() set(protobuf_source_dir "${ClickHouse_SOURCE_DIR}/contrib/google-protobuf") set(protobuf_binary_dir "${ClickHouse_BINARY_DIR}/contrib/google-protobuf") - add_definitions(-DGOOGLE_PROTOBUF_CMAKE_BUILD) add_definitions(-DHAVE_PTHREAD) @@ -340,6 +339,8 @@ else () # This is quite ugly but I cannot make dependencies work propery. + set(abseil_source_dir "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") + execute_process( COMMAND mkdir -p ${PROTOC_BUILD_DIR} COMMAND_ECHO STDOUT) @@ -354,6 +355,8 @@ else () "-Dprotobuf_BUILD_CONFORMANCE=0" "-Dprotobuf_BUILD_EXAMPLES=0" "-Dprotobuf_BUILD_PROTOC_BINARIES=1" + "-DABSL_ROOT_DIR=${abseil_source_dir}" + "-DABSL_ENABLE_INSTALL=0" "${protobuf_source_dir}/cmake" WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" COMMAND_ECHO STDOUT) From 80dffe5823217be0ebcefc3fccd9f36ec389a1e0 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 14 Nov 2023 15:55:46 +0000 Subject: [PATCH 330/813] fixed null due to review --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index a7fb15fa375..8b40cd12282 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -32,12 +32,13 @@ namespace float convertFloat16ToFloat32(uint16_t float16_value) { - if (float16_value == 0000000000000000) - return float(0); uint16_t sign = (float16_value >> 15) & 0x1; uint16_t exponent = (float16_value >> 10) & 0x1F; uint16_t fraction = float16_value & 0x3FF; + if (exponent == 0 && fraction == 0) + return float(sign << 31); + // Handling special cases for exponent if (exponent == 0x1F) { From d2aaac686d0cbf40f62b9d8b9bd182d6b5fc1e18 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 14 Nov 2023 18:32:37 +0100 Subject: [PATCH 331/813] Update 02908_Npy_files_caching.reference --- tests/queries/0_stateless/02908_Npy_files_caching.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.reference b/tests/queries/0_stateless/02908_Npy_files_caching.reference index 5f5e5f6282d..db9adf2d9c1 100644 --- a/tests/queries/0_stateless/02908_Npy_files_caching.reference +++ b/tests/queries/0_stateless/02908_Npy_files_caching.reference @@ -2,8 +2,8 @@ 3 3 array Int64 -\N +3 1000000 1000000 array Int64 -\N +1000000 From 03c3e968595512558f2cd4a3fddeec0ede75990c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 14 Nov 2023 17:32:34 +0000 Subject: [PATCH 332/813] Small test changes --- src/Coordination/Changelog.cpp | 1 - src/Coordination/tests/gtest_coordination.cpp | 27 ++++++++----------- 2 files changed, 11 insertions(+), 17 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 933eda485dd..c28cc368ac0 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -14,7 +14,6 @@ #include #include #include -#include "IO/CompressionMethod.h" #include diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 1dc4ae1382c..2b5fd3424c0 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1104,20 +1104,15 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) } /// Truncating only some entries from the end -TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) +/// For compressed logs we have no reliable way of knowing how many log entries were lost +/// after we truncate some bytes from the end +TEST_F(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) { - auto params = GetParam(); - - /// For compressed logs we have no reliable way of knowing how many log entries were lost - /// after we truncate some bytes from the end - if (!params.extension.empty()) - return; - ChangelogDirTest test("./logs"); setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::LogFileSettings{.force_sync = true, .compress_logs = false, .rotate_interval = 20}, DB::FlushSettings(), keeper_context); changelog.init(1, 0); @@ -1131,23 +1126,23 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) changelog.end_of_append_batch(0, 0); waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin")); DB::WriteBufferFromFile plain_buf( - "./logs/changelog_1_20.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + "./logs/changelog_1_20.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(plain_buf.size() - 30); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::LogFileSettings{.force_sync = true, .compress_logs = false, .rotate_interval = 20}, DB::FlushSettings(), keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 19); - EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); - assertBrokenLogRemoved("./logs", "changelog_21_40.bin" + params.extension); - EXPECT_TRUE(fs::exists("./logs/changelog_20_39.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin")); + assertBrokenLogRemoved("./logs", "changelog_21_40.bin"); + EXPECT_TRUE(fs::exists("./logs/changelog_20_39.bin")); auto entry = getLogEntry("hello_world", 7777); changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); From 9df2775f08d5ab377ba3aa2dd05010329b67ef20 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 14 Nov 2023 02:28:09 +0000 Subject: [PATCH 333/813] reduce timeout and setTimeout earlier. Signed-off-by: Jianfei Hu --- src/Coordination/KeeperContext.cpp | 7 ++++--- src/IO/S3/Credentials.cpp | 10 +++++----- tests/integration/helpers/keeper_config2.xml | 1 + .../integration/test_keeper_availability_zone/test.py | 6 ++++-- .../integration/test_keeper_four_word_command/test.py | 8 ++++---- 5 files changed, 18 insertions(+), 14 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 9745a53d1ab..c3cb166abee 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -39,8 +39,9 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, if (config.hasProperty("keeper_server.availability_zone")) { - auto keeper_az = config.getString("keeper_server.availability_zone.value"); - if (config.getBool("keeper_server.availability_zone.enable_auto_detection_on_cloud", false)) + auto keeper_az = config.getString("keeper_server.availability_zone.value", ""); + const auto auto_detect_for_cloud = config.getBool("keeper_server.availability_zone.enable_auto_detection_on_cloud", false); + if (keeper_az.empty() && auto_detect_for_cloud) { try { @@ -54,7 +55,7 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, if (!keeper_az.empty()) { system_nodes_with_data[keeper_availability_zone_path] = keeper_az; - LOG_INFO(&Poco::Logger::get("KeeperContext"), "Initialize the KeeperContext with availability zone: '{}'.'. ", keeper_az); + LOG_INFO(&Poco::Logger::get("KeeperContext"), "Initialize the KeeperContext with availability zone: '{}'", keeper_az); } } diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 4ba14572589..7d6ed094486 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -65,7 +65,7 @@ bool areCredentialsEmptyOrExpired(const Aws::Auth::AWSCredentials & credentials, } const char SSO_CREDENTIALS_PROVIDER_LOG_TAG[] = "SSOCredentialsProvider"; -const int AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS = 5; +const int AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS = 3; } @@ -241,11 +241,11 @@ String AWSEC2MetadataClient::getAvailabilityZoneOrException() { Poco::URI uri(getAWSMetadataEndpoint() + EC2_AVAILABILITY_ZONE_RESOURCE); Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort()); + session.setTimeout(Poco::Timespan(AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS, 0)); Poco::Net::HTTPResponse response; Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri.getPath()); session.sendRequest(request); - session.setTimeout(Poco::Timespan(AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS, 0)); std::istream & rs = session.receiveResponse(response); if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) @@ -287,17 +287,17 @@ String getRunningAvailabilityZoneImpl() auto aws_az = AWSEC2MetadataClient::getAvailabilityZoneOrException(); return aws_az; } - catch (const DB::Exception & aws_ex) + catch (const std::exception & aws_ex) { try { auto gcp_zone = getGCPAvailabilityZoneOrException(); return gcp_zone; } - catch (const DB::Exception & gcp_ex) + catch (const std::exception & gcp_ex) { throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Failed to find the availability zone, tried AWS and GCP. AWS Error: {}\nGCP Error: {}", aws_ex.displayText(), gcp_ex.displayText()); + "Failed to find the availability zone, tried AWS and GCP. AWS Error: {}\nGCP Error: {}", aws_ex.what(), gcp_ex.what()); } } } diff --git a/tests/integration/helpers/keeper_config2.xml b/tests/integration/helpers/keeper_config2.xml index 0c58aaceb1c..2afff2f5e59 100644 --- a/tests/integration/helpers/keeper_config2.xml +++ b/tests/integration/helpers/keeper_config2.xml @@ -14,6 +14,7 @@ 2 az-zoo2 + 1 diff --git a/tests/integration/test_keeper_availability_zone/test.py b/tests/integration/test_keeper_availability_zone/test.py index b78e776f3c6..a2003f8539e 100644 --- a/tests/integration/test_keeper_availability_zone/test.py +++ b/tests/integration/test_keeper_availability_zone/test.py @@ -27,10 +27,12 @@ def test_get_availability_zone(): with KeeperClient.from_cluster(cluster, "zoo1") as client1: assert client1.get("/keeper/availability_zone") == "az-zoo1" + # Keeper2 set enable_auto_detection_on_cloud to true, but is ignored and az-zoo2 is used. with KeeperClient.from_cluster(cluster, "zoo2") as client2: assert client2.get("/keeper/availability_zone") == "az-zoo2" - + assert "availability_zone" in client2.ls("/keeper") + # keeper3 is not configured with availability_zone value. with KeeperClient.from_cluster(cluster, "zoo3") as client3: with pytest.raises(Exception): - client3.get("/keeper/availability_zone") \ No newline at end of file + client3.get("/keeper/availability_zone") diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 25c4bc55327..71501133ae7 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -183,8 +183,8 @@ def test_cmd_mntr(started_cluster): # contains: # 10 nodes created by test # 3 nodes created by clickhouse "/clickhouse/task_queue/ddl" - # 1 root node, 4 keeper system nodes - assert int(result["zk_znode_count"]) == 15 + # 1 root node, 3 keeper system nodes + assert int(result["zk_znode_count"]) == 14 assert int(result["zk_watch_count"]) == 2 assert int(result["zk_ephemerals_count"]) == 2 assert int(result["zk_approximate_data_size"]) > 0 @@ -333,7 +333,7 @@ def test_cmd_srvr(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"], 16) > 10 assert result["Mode"] == "leader" - assert result["Node count"] == "15" + assert result["Node count"] == "14" finally: destroy_zk_client(zk) @@ -373,7 +373,7 @@ def test_cmd_stat(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"], 16) >= 10 assert result["Mode"] == "leader" - assert result["Node count"] == "15" + assert result["Node count"] == "14" # filter connection statistics cons = [n for n in data.split("\n") if "=" in n] From 79ed0f8a21c156c578f8863830c9e9cc11cd8f12 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 14 Nov 2023 19:21:56 +0100 Subject: [PATCH 334/813] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 8cb921bdd19..230c1e99151 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -34,7 +34,6 @@ 02352_grouby_shadows_arg 02354_annoy 02428_parameterized_view -02458_use_structure_from_insertion_table 02479_race_condition_between_insert_and_droppin_mv 02493_inconsistent_hex_and_binary_number 02554_fix_grouping_sets_predicate_push_down @@ -42,12 +41,9 @@ 01009_global_array_join_names 00917_multiple_joins_denny_crane 00636_partition_key_parts_pruning -01825_type_json_multiple_files 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET 02404_memory_bound_merging 02725_agg_projection_resprect_PK -02721_url_cluster -02534_s3_cluster_insert_select_schema_inference 02765_parallel_replicas_final_modifier 02784_parallel_replicas_automatic_decision_join 02818_parameterized_view_with_cte_multiple_usage From d24eb8bebd5cb827ad321006c250d48af67f77a8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 9 Nov 2023 12:44:32 +0300 Subject: [PATCH 335/813] Analyzer remove unused projection columns --- .../RemoveUnusedProjectionColumnsPass.cpp | 89 +++++++++++++++++++ .../RemoveUnusedProjectionColumnsPass.h | 24 +++++ src/Analyzer/QueryNode.cpp | 64 +++++++++++++ src/Analyzer/QueryNode.h | 11 ++- src/Analyzer/QueryTreePassManager.cpp | 2 + src/Analyzer/UnionNode.cpp | 35 ++++++++ src/Analyzer/UnionNode.h | 6 ++ ...remove_unused_projection_columns.reference | 8 ++ ...lyzer_remove_unused_projection_columns.sql | 22 +++++ 9 files changed, 257 insertions(+), 4 deletions(-) create mode 100644 src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp create mode 100644 src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.h create mode 100644 tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.reference create mode 100644 tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.sql diff --git a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp new file mode 100644 index 00000000000..223ccf67380 --- /dev/null +++ b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp @@ -0,0 +1,89 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class CollectUsedColumnsVisitor : public InDepthQueryTreeVisitor +{ +public: + bool needChildVisit(QueryTreeNodePtr &, QueryTreeNodePtr & child) + { + auto node_type = child->getNodeType(); + if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + { + subqueries_nodes_to_visit.insert(child); + return false; + } + + return true; + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto node_type = node->getNodeType(); + if (node_type != QueryTreeNodeType::COLUMN) + return; + + auto & column_node = node->as(); + auto column_source_node = column_node.getColumnSource(); + auto column_source_node_type = column_source_node->getNodeType(); + + if (column_source_node_type == QueryTreeNodeType::QUERY || column_source_node_type == QueryTreeNodeType::UNION) + { + auto * column_source_node_ptr = column_source_node.get(); + query_or_union_node_to_used_columns[column_source_node_ptr].insert(column_node.getColumnName()); + } + } + + void reset() + { + subqueries_nodes_to_visit.clear(); + query_or_union_node_to_used_columns.clear(); + } + + std::unordered_set subqueries_nodes_to_visit; + std::unordered_map> query_or_union_node_to_used_columns; +}; + +} + +void RemoveUnusedProjectionColumnsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +{ + std::vector nodes_to_visit; + nodes_to_visit.push_back(query_tree_node); + + CollectUsedColumnsVisitor visitor; + + while (!nodes_to_visit.empty()) + { + auto node_to_visit = std::move(nodes_to_visit.back()); + nodes_to_visit.pop_back(); + + visitor.visit(node_to_visit); + + for (auto & [query_or_union_node, used_columns] : visitor.query_or_union_node_to_used_columns) + { + if (auto * union_node = query_or_union_node->as()) + union_node->removeUnusedProjectionColumns(used_columns); + else if (auto * query_node = query_or_union_node->as()) + query_node->removeUnusedProjectionColumns(used_columns); + } + + for (const auto & subquery_node_to_visit : visitor.subqueries_nodes_to_visit) + nodes_to_visit.push_back(subquery_node_to_visit); + + visitor.reset(); + } +} + +} diff --git a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.h b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.h new file mode 100644 index 00000000000..2c6768ed4bf --- /dev/null +++ b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +namespace DB +{ + +/** Remove unused projection columns in subqueries. + * + * Example: SELECT a FROM (SELECT a, b FROM test_table); + * Result: SELECT a FROM (SELECT a FROM test_table); + */ +class RemoveUnusedProjectionColumnsPass final : public IQueryTreePass +{ +public: + String getName() override { return "RemoveUnusedProjectionColumnsPass"; } + + String getDescription() override { return "Remove unused projection columns in subqueries."; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 51e3dac781d..529631f045d 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -46,6 +46,70 @@ QueryNode::QueryNode(ContextMutablePtr context_) : QueryNode(std::move(context_), {} /*settings_changes*/) {} +void QueryNode::resolveProjectionColumns(NamesAndTypes projection_columns_value) +{ + if (projection_columns_value.size() != getProjection().getNodes().size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected projection columns size to match projection nodes size"); + + projection_columns = std::move(projection_columns_value); +} + +void QueryNode::removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns) +{ + auto & projection_nodes = getProjection().getNodes(); + + if (used_projection_columns.empty()) + { + /// Keep at least 1 column if used columns are empty + projection_nodes.erase(projection_nodes.begin() + 1, projection_nodes.end()); + projection_columns.erase(projection_columns.begin() + 1, projection_columns.end()); + } + + size_t projection_columns_size = projection_columns.size(); + size_t write_index = 0; + + for (size_t i = 0; i < projection_columns_size; ++i) + { + if (!used_projection_columns.contains(projection_columns[i].name)) + continue; + + projection_nodes[write_index] = projection_nodes[i]; + projection_columns[write_index] = projection_columns[i]; + ++write_index; + } + + projection_nodes.erase(projection_nodes.begin() + write_index, projection_nodes.end()); + projection_columns.erase(projection_columns.begin() + write_index, projection_columns.end()); +} + +void QueryNode::removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns_indexes) +{ + auto & projection_nodes = getProjection().getNodes(); + + if (used_projection_columns_indexes.empty()) + { + /// Keep at least 1 column if used columns are empty + projection_nodes.erase(projection_nodes.begin() + 1, projection_nodes.end()); + projection_columns.erase(projection_columns.begin() + 1, projection_columns.end()); + } + + size_t projection_columns_size = projection_columns.size(); + size_t write_index = 0; + + for (size_t i = 0; i < projection_columns_size; ++i) + { + if (!used_projection_columns_indexes.contains(i)) + continue; + + projection_nodes[write_index] = projection_nodes[i]; + projection_columns[write_index] = projection_columns[i]; + ++write_index; + } + + projection_nodes.erase(projection_nodes.begin() + write_index, projection_nodes.end()); + projection_columns.erase(projection_columns.begin() + write_index, projection_columns.end()); +} + void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "QUERY id: " << format_state.getNodeId(this); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 277d6404965..82bc72b7411 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -556,10 +556,13 @@ public: } /// Resolve query node projection columns - void resolveProjectionColumns(NamesAndTypes projection_columns_value) - { - projection_columns = std::move(projection_columns_value); - } + void resolveProjectionColumns(NamesAndTypes projection_columns_value); + + /// Remove unused projection columns + void removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns); + + /// Remove unused projection columns + void removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns_indexes); QueryTreeNodeType getNodeType() const override { diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 08474c4100a..254bd81c030 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -243,6 +244,7 @@ void QueryTreePassManager::dump(WriteBuffer & buffer, size_t up_to_pass_index) void addQueryTreePasses(QueryTreePassManager & manager) { manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 2bc3daeef36..5d2ac128abe 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -88,6 +88,41 @@ NamesAndTypes UnionNode::computeProjectionColumns() const return result_columns; } +void UnionNode::removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns) +{ + auto projection_columns = computeProjectionColumns(); + size_t projection_columns_size = projection_columns.size(); + std::unordered_set used_projection_column_indexes; + + for (size_t i = 0; i < projection_columns_size; ++i) + { + const auto & projection_column = projection_columns[i]; + if (used_projection_columns.contains(projection_column.name)) + used_projection_column_indexes.insert(i); + } + + auto & query_nodes = getQueries().getNodes(); + for (auto & query_node : query_nodes) + { + if (auto * query_node_typed = query_node->as()) + query_node_typed->removeUnusedProjectionColumns(used_projection_column_indexes); + else if (auto * union_node_typed = query_node->as()) + union_node_typed->removeUnusedProjectionColumns(used_projection_column_indexes); + } +} + +void UnionNode::removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns_indexes) +{ + auto & query_nodes = getQueries().getNodes(); + for (auto & query_node : query_nodes) + { + if (auto * query_node_typed = query_node->as()) + query_node_typed->removeUnusedProjectionColumns(used_projection_columns_indexes); + else if (auto * union_node_typed = query_node->as()) + union_node_typed->removeUnusedProjectionColumns(used_projection_columns_indexes); + } +} + void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "UNION id: " << format_state.getNodeId(this); diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index 0045b0c334f..7686b73f5e0 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -129,6 +129,12 @@ public: /// Compute union node projection columns NamesAndTypes computeProjectionColumns() const; + /// Remove unused projection columns + void removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns); + + /// Remove unused projection columns + void removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns_indexes); + QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::UNION; diff --git a/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.reference b/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.reference new file mode 100644 index 00000000000..405d3348775 --- /dev/null +++ b/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.reference @@ -0,0 +1,8 @@ +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.sql b/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.sql new file mode 100644 index 00000000000..70de63c592d --- /dev/null +++ b/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.sql @@ -0,0 +1,22 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE = MergeTree ORDER BY id; + +INSERT INTO test_table VALUES (0, 'Value_0'); + +SET max_columns_to_read = 1; + +SELECT id FROM (SELECT * FROM test_table); +SELECT id FROM (SELECT * FROM (SELECT * FROM test_table)); +SELECT id FROM (SELECT * FROM test_table UNION ALL SELECT * FROM test_table); + +SELECT id FROM (SELECT id, value FROM test_table); +SELECT id FROM (SELECT id, value FROM (SELECT id, value FROM test_table)); +SELECT id FROM (SELECT id, value FROM test_table UNION ALL SELECT id, value FROM test_table); + +DROP TABLE test_table; From 3104939cff4349daf867467c5d106a904fde952f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 9 Nov 2023 16:38:16 +0300 Subject: [PATCH 336/813] Updated tests --- .../02227_union_match_by_name.reference | 21 ++++------- .../02476_fuse_sum_count.reference | 23 +++--------- .../02477_fuse_quantiles.reference | 36 +++++-------------- ...8_distinct_to_count_optimization.reference | 36 +++++-------------- 4 files changed, 29 insertions(+), 87 deletions(-) diff --git a/tests/queries/0_stateless/02227_union_match_by_name.reference b/tests/queries/0_stateless/02227_union_match_by_name.reference index 685b3c83b05..42b9b01a529 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.reference +++ b/tests/queries/0_stateless/02227_union_match_by_name.reference @@ -14,36 +14,29 @@ Header: avgWeighted(x, y) Nullable(Float64) Header: x_0 Nullable(UInt8) y_1 UInt8 Union - Header: NULL Nullable(UInt8) - x Nullable(UInt8) + Header: x Nullable(UInt8) y UInt8 Expression (Conversion before UNION) - Header: NULL Nullable(UInt8) - x Nullable(UInt8) + Header: x Nullable(UInt8) y UInt8 Expression (Project names) - Header: NULL Nullable(Nothing) - x UInt8 + Header: x UInt8 y UInt8 Expression (Projection) - Header: NULL_Nullable(Nothing) Nullable(Nothing) - 255_UInt8 UInt8 + Header: 255_UInt8 UInt8 1_UInt8 UInt8 Expression (Change column names to column identifiers) Header: dummy_0 UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 Expression (Conversion before UNION) - Header: NULL Nullable(UInt8) - x Nullable(UInt8) + Header: x Nullable(UInt8) y UInt8 Expression (Project names) - Header: y UInt8 - x Nullable(Nothing) + Header: x Nullable(Nothing) y UInt8 Expression (Projection) - Header: 1_UInt8 UInt8 - NULL_Nullable(Nothing) Nullable(Nothing) + Header: NULL_Nullable(Nothing) Nullable(Nothing) 1_UInt8 UInt8 Expression (Change column names to column identifiers) Header: dummy_0 UInt8 diff --git a/tests/queries/0_stateless/02476_fuse_sum_count.reference b/tests/queries/0_stateless/02476_fuse_sum_count.reference index 43a39e8b7e5..c62583e8c88 100644 --- a/tests/queries/0_stateless/02476_fuse_sum_count.reference +++ b/tests/queries/0_stateless/02476_fuse_sum_count.reference @@ -256,27 +256,14 @@ QUERY id: 0 QUERY id: 14, is_subquery: 1 PROJECTION COLUMNS x Int64 - count(b) UInt64 PROJECTION - LIST id: 15, nodes: 2 - FUNCTION id: 16, function_name: tupleElement, function_type: ordinary, result_type: Int64 + LIST id: 15, nodes: 1 + FUNCTION id: 16, function_name: sum, function_type: aggregate, result_type: Int64 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: sumCount, function_type: aggregate, result_type: Tuple(Int64, UInt64) - ARGUMENTS - LIST id: 19, nodes: 1 - COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21 - CONSTANT id: 22, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 23, function_name: tupleElement, function_type: ordinary, result_type: UInt64 - ARGUMENTS - LIST id: 24, nodes: 2 - FUNCTION id: 18, function_name: sumCount, function_type: aggregate, result_type: Tuple(Int64, UInt64) - ARGUMENTS - LIST id: 19, nodes: 1 - COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21 - CONSTANT id: 25, constant_value: UInt64_2, constant_value_type: UInt8 + LIST id: 17, nodes: 1 + COLUMN id: 18, column_name: b, result_type: Int8, source_id: 19 JOIN TREE - TABLE id: 21, table_name: default.fuse_tbl + TABLE id: 19, table_name: default.fuse_tbl 0 0 nan 0 0 nan 45 10 4.5 Decimal(38, 0) UInt64 Float64 diff --git a/tests/queries/0_stateless/02477_fuse_quantiles.reference b/tests/queries/0_stateless/02477_fuse_quantiles.reference index 7c7d581f7fb..8384df9f04a 100644 --- a/tests/queries/0_stateless/02477_fuse_quantiles.reference +++ b/tests/queries/0_stateless/02477_fuse_quantiles.reference @@ -48,35 +48,17 @@ QUERY id: 0 QUERY id: 19, is_subquery: 1 PROJECTION COLUMNS x Float64 - quantile(0.9)(b) Float64 PROJECTION - LIST id: 21, nodes: 2 - FUNCTION id: 22, function_name: arrayElement, function_type: ordinary, result_type: Float64 + LIST id: 21, nodes: 1 + FUNCTION id: 22, function_name: quantile, function_type: aggregate, result_type: Float64 + PARAMETERS + LIST id: 23, nodes: 1 + CONSTANT id: 24, constant_value: Float64_0.5, constant_value_type: Float64 ARGUMENTS - LIST id: 23, nodes: 2 - FUNCTION id: 24, function_name: quantiles, function_type: aggregate, result_type: Array(Float64) - PARAMETERS - LIST id: 25, nodes: 2 - CONSTANT id: 26, constant_value: Float64_0.5, constant_value_type: Float64 - CONSTANT id: 27, constant_value: Float64_0.9, constant_value_type: Float64 - ARGUMENTS - LIST id: 28, nodes: 1 - COLUMN id: 29, column_name: b, result_type: Int32, source_id: 30 - CONSTANT id: 31, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 32, function_name: arrayElement, function_type: ordinary, result_type: Float64 - ARGUMENTS - LIST id: 33, nodes: 2 - FUNCTION id: 24, function_name: quantiles, function_type: aggregate, result_type: Array(Float64) - PARAMETERS - LIST id: 25, nodes: 2 - CONSTANT id: 26, constant_value: Float64_0.5, constant_value_type: Float64 - CONSTANT id: 27, constant_value: Float64_0.9, constant_value_type: Float64 - ARGUMENTS - LIST id: 28, nodes: 1 - COLUMN id: 29, column_name: b, result_type: Int32, source_id: 30 - CONSTANT id: 34, constant_value: UInt64_2, constant_value_type: UInt8 + LIST id: 25, nodes: 1 + COLUMN id: 26, column_name: b, result_type: Int32, source_id: 27 JOIN TREE - TABLE id: 30, table_name: default.fuse_tbl + TABLE id: 27, table_name: default.fuse_tbl GROUP BY - LIST id: 35, nodes: 1 + LIST id: 28, nodes: 1 COLUMN id: 18, column_name: x, result_type: Float64, source_id: 19 diff --git a/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference b/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference index b2b15f92199..a2c441fa460 100644 --- a/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference +++ b/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference @@ -125,18 +125,13 @@ QUERY id: 0 QUERY id: 3, is_subquery: 1 PROJECTION COLUMNS a UInt8 - sum(b) UInt64 PROJECTION - LIST id: 4, nodes: 2 + LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 - FUNCTION id: 7, function_name: sum, function_type: aggregate, result_type: UInt64 - ARGUMENTS - LIST id: 8, nodes: 1 - COLUMN id: 9, column_name: b, result_type: UInt8, source_id: 6 JOIN TREE TABLE id: 6, table_name: default.test_rewrite_uniq_to_count GROUP BY - LIST id: 10, nodes: 1 + LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 SETTINGS allow_experimental_analyzer=1 6. test group by with subquery alias @@ -162,18 +157,13 @@ QUERY id: 0 QUERY id: 3, alias: t, is_subquery: 1 PROJECTION COLUMNS a UInt8 - sum(b) UInt64 PROJECTION - LIST id: 4, nodes: 2 + LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 - FUNCTION id: 7, function_name: sum, function_type: aggregate, result_type: UInt64 - ARGUMENTS - LIST id: 8, nodes: 1 - COLUMN id: 9, column_name: b, result_type: UInt8, source_id: 6 JOIN TREE TABLE id: 6, table_name: default.test_rewrite_uniq_to_count GROUP BY - LIST id: 10, nodes: 1 + LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 SETTINGS allow_experimental_analyzer=1 7. test group by with compound column name @@ -199,18 +189,13 @@ QUERY id: 0 QUERY id: 3, alias: t, is_subquery: 1 PROJECTION COLUMNS alias_of_a UInt8 - sum(b) UInt64 PROJECTION - LIST id: 4, nodes: 2 + LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 - FUNCTION id: 7, function_name: sum, function_type: aggregate, result_type: UInt64 - ARGUMENTS - LIST id: 8, nodes: 1 - COLUMN id: 9, column_name: b, result_type: UInt8, source_id: 6 JOIN TREE TABLE id: 6, table_name: default.test_rewrite_uniq_to_count GROUP BY - LIST id: 10, nodes: 1 + LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 SETTINGS allow_experimental_analyzer=1 8. test group by with select expression alias @@ -236,17 +221,12 @@ QUERY id: 0 QUERY id: 3, alias: t, is_subquery: 1 PROJECTION COLUMNS alias_of_a UInt8 - sum(b) UInt64 PROJECTION - LIST id: 4, nodes: 2 + LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 - FUNCTION id: 7, function_name: sum, function_type: aggregate, result_type: UInt64 - ARGUMENTS - LIST id: 8, nodes: 1 - COLUMN id: 9, column_name: b, result_type: UInt8, source_id: 6 JOIN TREE TABLE id: 6, table_name: default.test_rewrite_uniq_to_count GROUP BY - LIST id: 10, nodes: 1 + LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 SETTINGS allow_experimental_analyzer=1 From 1562e24232f17adab742978cfad6d821ab88bfdd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 9 Nov 2023 16:39:43 +0300 Subject: [PATCH 337/813] Fixed tests --- src/Planner/Utils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 9c46622f578..9a6ef6f5d83 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -436,8 +436,8 @@ QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTyp auto query_node = std::make_shared(std::move(context_copy)); - query_node->resolveProjectionColumns(projection_columns); query_node->getProjection().getNodes() = std::move(subquery_projection_nodes); + query_node->resolveProjectionColumns(projection_columns); query_node->getJoinTree() = table_expression; query_node->setIsSubquery(true); From 5f009e99f6d70bba7fe1ad7c985936dd666d07e8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 10 Nov 2023 16:29:19 +0300 Subject: [PATCH 338/813] Updated implementation --- src/Analyzer/IQueryTreeNode.h | 2 +- src/Analyzer/InDepthQueryTreeVisitor.h | 2 +- .../RemoveUnusedProjectionColumnsPass.cpp | 90 ++++++++++++++++--- src/Analyzer/QueryNode.cpp | 16 ---- src/Analyzer/Utils.cpp | 11 +++ src/Analyzer/Utils.h | 6 ++ tests/analyzer_tech_debt.txt | 4 - 7 files changed, 95 insertions(+), 36 deletions(-) diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 3f6816696b4..922eaabe75c 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -182,7 +182,7 @@ public: struct ConvertToASTOptions { - /// Add _CAST if constant litral type is different from column type + /// Add _CAST if constant literal type is different from column type bool add_cast_for_constants = true; /// Identifiers are fully qualified (`database.table.column`), otherwise names are just column names (`column`) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index dec329b5403..62ddc06659c 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -188,7 +188,7 @@ private: if (auto * table_function_node = parent->as()) { if (child != table_function_node->getArgumentsNode()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "TableFunctioNode is expected to have only one child node"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "TableFunctionNode is expected to have only one child node"); const auto & unresolved_indexes = table_function_node->getUnresolvedArgumentIndexes(); diff --git a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp index 223ccf67380..29626c97d68 100644 --- a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp +++ b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp @@ -1,11 +1,14 @@ #include +#include + #include #include #include #include #include -#include +#include +#include namespace DB { @@ -13,13 +16,15 @@ namespace DB namespace { -class CollectUsedColumnsVisitor : public InDepthQueryTreeVisitor +class CollectUsedColumnsVisitor : public InDepthQueryTreeVisitorWithContext { public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + bool needChildVisit(QueryTreeNodePtr &, QueryTreeNodePtr & child) { - auto node_type = child->getNodeType(); - if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + if (isQueryOrUnionNode(child)) { subqueries_nodes_to_visit.insert(child); return false; @@ -28,9 +33,21 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto node_type = node->getNodeType(); + + if (node_type == QueryTreeNodeType::QUERY) + { + auto & query_node = node->as(); + auto table_expressions = extractTableExpressions(query_node.getJoinTree()); + for (const auto & table_expression : table_expressions) + if (isQueryOrUnionNode(table_expression)) + query_or_union_node_to_used_columns.emplace(table_expression, std::unordered_set()); + + return; + } + if (node_type != QueryTreeNodeType::COLUMN) return; @@ -39,10 +56,7 @@ public: auto column_source_node_type = column_source_node->getNodeType(); if (column_source_node_type == QueryTreeNodeType::QUERY || column_source_node_type == QueryTreeNodeType::UNION) - { - auto * column_source_node_ptr = column_source_node.get(); - query_or_union_node_to_used_columns[column_source_node_ptr].insert(column_node.getColumnName()); - } + query_or_union_node_to_used_columns[column_source_node].insert(column_node.getColumnName()); } void reset() @@ -52,17 +66,59 @@ public: } std::unordered_set subqueries_nodes_to_visit; - std::unordered_map> query_or_union_node_to_used_columns; + std::unordered_map> query_or_union_node_to_used_columns; }; +std::unordered_set convertUsedColumnNamesToUsedProjectionIndexes(const QueryTreeNodePtr & query_or_union_node, const std::unordered_set & used_column_names) +{ + std::unordered_set result; + + auto * union_node = query_or_union_node->as(); + auto * query_node = query_or_union_node->as(); + + const auto & projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); + size_t projection_columns_size = projection_columns.size(); + + for (size_t i = 0; i < projection_columns_size; ++i) + { + const auto & projection_column = projection_columns[i]; + if (used_column_names.contains(projection_column.name)) + result.insert(i); + } + + return result; +} + +/// We cannot remove aggregate functions, if query does not contain GROUP BY or arrayJoin from subquery projection +void updateUsedProjectionIndexes(const QueryTreeNodePtr & query_or_union_node, std::unordered_set & used_projection_columns_indexes) +{ + if (auto * union_node = query_or_union_node->as()) + { + for (auto & query_node : union_node->getQueries().getNodes()) + updateUsedProjectionIndexes(query_node, used_projection_columns_indexes); + return; + } + + const auto & query_node = query_or_union_node->as(); + const auto & projection_nodes = query_node.getProjection().getNodes(); + size_t projection_nodes_size = projection_nodes.size(); + + for (size_t i = 0; i < projection_nodes_size; ++i) + { + const auto & projection_node = projection_nodes[i]; + if ((!query_node.hasGroupBy() && hasAggregateFunctionNodes(projection_node)) && hasFunctionNode(projection_node, "arrayJoin")) + used_projection_columns_indexes.insert(i); + } +} + } -void RemoveUnusedProjectionColumnsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +void RemoveUnusedProjectionColumnsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { std::vector nodes_to_visit; nodes_to_visit.push_back(query_tree_node); - CollectUsedColumnsVisitor visitor; + CollectUsedColumnsVisitor visitor(std::move(context)); while (!nodes_to_visit.empty()) { @@ -73,10 +129,16 @@ void RemoveUnusedProjectionColumnsPass::run(QueryTreeNodePtr query_tree_node, Co for (auto & [query_or_union_node, used_columns] : visitor.query_or_union_node_to_used_columns) { + auto used_projection_indexes = convertUsedColumnNamesToUsedProjectionIndexes(query_or_union_node, used_columns); + updateUsedProjectionIndexes(query_or_union_node, used_projection_indexes); + + /// Keep at least 1 column if used columns are empty + used_projection_indexes.insert(0); + if (auto * union_node = query_or_union_node->as()) - union_node->removeUnusedProjectionColumns(used_columns); + union_node->removeUnusedProjectionColumns(used_projection_indexes); else if (auto * query_node = query_or_union_node->as()) - query_node->removeUnusedProjectionColumns(used_columns); + query_node->removeUnusedProjectionColumns(used_projection_indexes); } for (const auto & subquery_node_to_visit : visitor.subqueries_nodes_to_visit) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 529631f045d..738b1ac62e8 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -57,14 +57,6 @@ void QueryNode::resolveProjectionColumns(NamesAndTypes projection_columns_value) void QueryNode::removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns) { auto & projection_nodes = getProjection().getNodes(); - - if (used_projection_columns.empty()) - { - /// Keep at least 1 column if used columns are empty - projection_nodes.erase(projection_nodes.begin() + 1, projection_nodes.end()); - projection_columns.erase(projection_columns.begin() + 1, projection_columns.end()); - } - size_t projection_columns_size = projection_columns.size(); size_t write_index = 0; @@ -85,14 +77,6 @@ void QueryNode::removeUnusedProjectionColumns(const std::unordered_set & used_projection_columns_indexes) { auto & projection_nodes = getProjection().getNodes(); - - if (used_projection_columns_indexes.empty()) - { - /// Keep at least 1 column if used columns are empty - projection_nodes.erase(projection_nodes.begin() + 1, projection_nodes.end()); - projection_columns.erase(projection_columns.begin() + 1, projection_columns.end()); - } - size_t projection_columns_size = projection_columns.size(); size_t write_index = 0; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 9b3a77e7e93..a3c9813f3d0 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -152,6 +152,17 @@ void makeUniqueColumnNamesInBlock(Block & block) } } +bool isQueryOrUnionNode(const IQueryTreeNode * node) +{ + auto node_type = node->getNodeType(); + return node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION; +} + +bool isQueryOrUnionNode(const QueryTreeNodePtr & node) +{ + return isQueryOrUnionNode(node.get()); +} + QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression, const DataTypePtr & type, const ContextPtr & context, diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 2bf12f01d8a..060dc7d8bc0 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -27,6 +27,12 @@ std::string getGlobalInFunctionNameForLocalInFunctionName(const std::string & fu /// Add unique suffix to names of duplicate columns in block void makeUniqueColumnNamesInBlock(Block & block); +/// Returns true, if node has type QUERY or UNION +bool isQueryOrUnionNode(const IQueryTreeNode * node); + +/// Returns true, if node has type QUERY or UNION +bool isQueryOrUnionNode(const QueryTreeNodePtr & node); + /** Build cast function that cast expression into type. * If resolve = true, then result cast function is resolved during build, otherwise * result cast function is not resolved during build. diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 833b04d5648..e5283f55d61 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -1,5 +1,4 @@ 00223_shard_distributed_aggregation_memory_efficient -00593_union_all_assert_columns_removed 00717_merge_and_distributed 00725_memory_tracking 01062_pm_all_join_with_block_continuation @@ -11,12 +10,9 @@ 01244_optimize_distributed_group_by_sharding_key 01268_mv_scalars 01268_shard_avgweighted -01287_max_execution_speed -01455_shard_leaf_max_rows_bytes_to_read 01495_subqueries_in_with_statement 01560_merge_distributed_join 01584_distributed_buffer_cannot_find_column -01586_columns_pruning 01624_soft_constraints 01656_test_query_log_factories_info 01739_index_hint From dc73819a38989462de554f4277e9d9434f23d52e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 11 Nov 2023 12:45:18 +0300 Subject: [PATCH 339/813] Updated implementation --- src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp index 29626c97d68..c4966a45074 100644 --- a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp +++ b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp @@ -132,8 +132,9 @@ void RemoveUnusedProjectionColumnsPass::run(QueryTreeNodePtr query_tree_node, Co auto used_projection_indexes = convertUsedColumnNamesToUsedProjectionIndexes(query_or_union_node, used_columns); updateUsedProjectionIndexes(query_or_union_node, used_projection_indexes); - /// Keep at least 1 column if used columns are empty - used_projection_indexes.insert(0); + /// Keep at least 1 column if used projection columns are empty + if (used_projection_indexes.empty()) + used_projection_indexes.insert(0); if (auto * union_node = query_or_union_node->as()) union_node->removeUnusedProjectionColumns(used_projection_indexes); From 13518633a8fb39aaaa5f1164c8a6f3df5b8b44ae Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 14 Nov 2023 14:39:33 +0300 Subject: [PATCH 340/813] Fixed tests --- .../RemoveUnusedProjectionColumnsPass.cpp | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp index c4966a45074..1d00f6e5dee 100644 --- a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp +++ b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp @@ -94,6 +94,22 @@ void updateUsedProjectionIndexes(const QueryTreeNodePtr & query_or_union_node, s { if (auto * union_node = query_or_union_node->as()) { + auto union_node_mode = union_node->getUnionMode(); + bool is_distinct = union_node_mode == SelectUnionMode::UNION_DISTINCT || + union_node_mode == SelectUnionMode::INTERSECT_DISTINCT || + union_node_mode == SelectUnionMode::EXCEPT_DISTINCT; + + if (is_distinct) + { + auto union_projection_columns = union_node->computeProjectionColumns(); + size_t union_projection_columns_size = union_projection_columns.size(); + + for (size_t i = 0; i < union_projection_columns_size; ++i) + used_projection_columns_indexes.insert(i); + + return; + } + for (auto & query_node : union_node->getQueries().getNodes()) updateUsedProjectionIndexes(query_node, used_projection_columns_indexes); return; @@ -106,7 +122,7 @@ void updateUsedProjectionIndexes(const QueryTreeNodePtr & query_or_union_node, s for (size_t i = 0; i < projection_nodes_size; ++i) { const auto & projection_node = projection_nodes[i]; - if ((!query_node.hasGroupBy() && hasAggregateFunctionNodes(projection_node)) && hasFunctionNode(projection_node, "arrayJoin")) + if ((!query_node.hasGroupBy() && hasAggregateFunctionNodes(projection_node)) || hasFunctionNode(projection_node, "arrayJoin")) used_projection_columns_indexes.insert(i); } } From 38f200d969c39e8f97ab658ac641142a7dd83a2d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 14 Nov 2023 18:54:55 +0000 Subject: [PATCH 341/813] Fix Date text parsing in optimistic path 1 --- src/IO/ReadHelpers.h | 22 ++++++++++++++-- .../02916_date_text_parsing.reference | 5 ++++ .../0_stateless/02916_date_text_parsing.sql | 25 +++++++++++++++++++ 3 files changed, 50 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02916_date_text_parsing.reference create mode 100644 tests/queries/0_stateless/02916_date_text_parsing.sql diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 5c55b36d3c3..17f3d3d4151 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -664,11 +664,20 @@ ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf); template inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) { + static constexpr bool throw_exception = std::is_same_v; + /// Optimistic path, when whole value is in buffer. if (!buf.eof() && buf.position() + 10 <= buf.buffer().end()) { char * pos = buf.position(); + auto error = [&] + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Cannot parse date here: {}", String(buf.position(), 10)); + return ReturnType(false); + }; + /// YYYY-MM-DD /// YYYY-MM-D /// YYYY-M-DD @@ -677,6 +686,9 @@ inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) /// The delimiters can be arbitrary characters, like YYYY/MM!DD, but obviously not digits. + if (!isNumericASCII(pos[0]) || !isNumericASCII(pos[1]) || !isNumericASCII(pos[2]) || !isNumericASCII(pos[3])) + return error(); + UInt16 year = (pos[0] - '0') * 1000 + (pos[1] - '0') * 100 + (pos[2] - '0') * 10 + (pos[3] - '0'); UInt8 month; UInt8 day; @@ -685,12 +697,18 @@ inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) if (isNumericASCII(pos[-1])) { /// YYYYMMDD + if (!isNumericASCII(pos[0]) || !isNumericASCII(pos[1]) || !isNumericASCII(pos[2])) + return error(); + month = (pos[-1] - '0') * 10 + (pos[0] - '0'); day = (pos[1] - '0') * 10 + (pos[2] - '0'); pos += 3; } else { + if (!isNumericASCII(pos[0])) + return error(); + month = pos[0] - '0'; if (isNumericASCII(pos[1])) { @@ -700,8 +718,8 @@ inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) else pos += 2; - if (isNumericASCII(pos[-1])) - return ReturnType(false); + if (isNumericASCII(pos[-1]) || !isNumericASCII(pos[0])) + return error(); day = pos[0] - '0'; if (isNumericASCII(pos[1])) diff --git a/tests/queries/0_stateless/02916_date_text_parsing.reference b/tests/queries/0_stateless/02916_date_text_parsing.reference new file mode 100644 index 00000000000..2ec123200d0 --- /dev/null +++ b/tests/queries/0_stateless/02916_date_text_parsing.reference @@ -0,0 +1,5 @@ +2020-01-02 SomeString +2020-01-02 SomeString +2020-01-02 SomeString +2020-01-02 SomeString +2020-01-02 SomeString diff --git a/tests/queries/0_stateless/02916_date_text_parsing.sql b/tests/queries/0_stateless/02916_date_text_parsing.sql new file mode 100644 index 00000000000..d895ccece19 --- /dev/null +++ b/tests/queries/0_stateless/02916_date_text_parsing.sql @@ -0,0 +1,25 @@ +select * from format(CSV, 'd Date, s String', 'abcdefgh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2bcdefgh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '20cdefgh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '202defgh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020efgh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '20200fgh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '202001gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020010h,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '20200102,SomeString'); +select * from format(CSV, 'd Date, s String', 'abcd-ef-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2bcd-ef-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '20cd-ef-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '202d-ef-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-ef-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-f-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-f-g,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-0f-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-01-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-01-h,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-1-gh,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-1-h,SomeString'); -- {serverError CANNOT_PARSE_DATE} +select * from format(CSV, 'd Date, s String', '2020-01-02,SomeString'); +select * from format(CSV, 'd Date, s String', '2020-01-2,SomeString'); +select * from format(CSV, 'd Date, s String', '2020-1-2,SomeString'); +select * from format(CSV, 'd Date, s String', '2020-1-02,SomeString'); From 64c2a696666d594783c1996c0910166cacba000f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 14 Nov 2023 20:28:37 +0100 Subject: [PATCH 342/813] check performance --- base/poco/Net/src/HTTPSession.cpp | 32 ++++++++++++++++--------------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 9ebbd7d04cd..97decded282 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -94,22 +94,24 @@ void HTTPSession::setTimeout(const Poco::Timespan& timeout) void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco::Timespan& sendTimeout, const Poco::Timespan& receiveTimeout) { _connectionTimeout = connectionTimeout; + _sendTimeout = sendTimeout; + _receiveTimeout = receiveTimeout; - if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) - { - _sendTimeout = sendTimeout; - - if (connected()) - _socket.setSendTimeout(_sendTimeout); - } - - if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) - { - _receiveTimeout = receiveTimeout; - - if (connected()) - _socket.setReceiveTimeout(_receiveTimeout); - } +// if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) +// { +// _sendTimeout = sendTimeout; +// +// if (connected()) +// _socket.setSendTimeout(_sendTimeout); +// } +// +// if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) +// { +// _receiveTimeout = receiveTimeout; +// +// if (connected()) +// _socket.setReceiveTimeout(_receiveTimeout); +// } } From cd909ffc48cc4fb6fb7bc23843659c9559c5921a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 14 Nov 2023 20:13:09 +0000 Subject: [PATCH 343/813] Test RabbitMQ with secure connection --- .../compose/docker_compose_rabbitmq.yml | 6 ++- .../integration/runner/misc/rabbitmq.conf | 8 --- .../runner/misc/rabbitmq/ca-cert.pem | 32 ++++++++++++ .../runner/misc/rabbitmq/generate_certs.sh | 10 ++++ .../runner/misc/rabbitmq/rabbitmq.conf | 15 ++++++ .../runner/misc/rabbitmq/server-cert.pem | 33 ++++++++++++ .../runner/misc/rabbitmq/server-key.pem | 52 +++++++++++++++++++ .../integration/test_storage_rabbitmq/test.py | 28 +++++++--- 8 files changed, 168 insertions(+), 16 deletions(-) delete mode 100644 docker/test/integration/runner/misc/rabbitmq.conf create mode 100644 docker/test/integration/runner/misc/rabbitmq/ca-cert.pem create mode 100755 docker/test/integration/runner/misc/rabbitmq/generate_certs.sh create mode 100644 docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf create mode 100644 docker/test/integration/runner/misc/rabbitmq/server-cert.pem create mode 100644 docker/test/integration/runner/misc/rabbitmq/server-key.pem diff --git a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml index 2db9fb589d2..61b21e0e3d9 100644 --- a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml +++ b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml @@ -6,9 +6,13 @@ services: hostname: rabbitmq1 expose: - ${RABBITMQ_PORT:-5672} + - ${RABBITMQ_SECURE_PORT:-5671} volumes: - type: ${RABBITMQ_LOGS_FS:-tmpfs} source: ${RABBITMQ_LOGS:-} target: /rabbitmq_logs/ - "${RABBITMQ_COOKIE_FILE}:/var/lib/rabbitmq/.erlang.cookie" - - /misc/rabbitmq.conf:/etc/rabbitmq/rabbitmq.conf \ No newline at end of file + - /misc/rabbitmq/rabbitmq.conf:/etc/rabbitmq/rabbitmq.conf + - /misc/rabbitmq/ca-cert.pem:/etc/rabbitmq/ca-cert.pem + - /misc/rabbitmq/server-cert.pem:/etc/rabbitmq/server-cert.pem + - /misc/rabbitmq/server-key.pem:/etc/rabbitmq/server-key.pem diff --git a/docker/test/integration/runner/misc/rabbitmq.conf b/docker/test/integration/runner/misc/rabbitmq.conf deleted file mode 100644 index 3527c83880b..00000000000 --- a/docker/test/integration/runner/misc/rabbitmq.conf +++ /dev/null @@ -1,8 +0,0 @@ -loopback_users.guest = false -listeners.tcp.default = 5672 -default_pass = clickhouse -default_user = root -management.tcp.port = 15672 - -log.file = /rabbitmq_logs/rabbit.log -log.file.level = debug diff --git a/docker/test/integration/runner/misc/rabbitmq/ca-cert.pem b/docker/test/integration/runner/misc/rabbitmq/ca-cert.pem new file mode 100644 index 00000000000..4a7b88f7936 --- /dev/null +++ b/docker/test/integration/runner/misc/rabbitmq/ca-cert.pem @@ -0,0 +1,32 @@ +-----BEGIN CERTIFICATE----- +MIIFhTCCA22gAwIBAgIUWhfjFfbwannH3KIqITDtgcvSItMwDQYJKoZIhvcNAQEL +BQAwUjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDELMAkGA1UEAwwCY2EwHhcNMjMxMTE0 +MTgyODI2WhcNMzMxMTExMTgyODI2WjBSMQswCQYDVQQGEwJSVTETMBEGA1UECAwK +U29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMQsw +CQYDVQQDDAJjYTCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAJfJegdC +gavNGYzSdva+5QMxGvqyLwZzjophMeyEzlW/Di4KFGPho+fVlVMB/EwaTRoBRLEu +SQusQwoFg71mGvUTOpgHzlsUz4vcVVFOiL4bJdzCWQKzdC8M8rUFoks9FMboVeSx +jhAnKAm/NpCLpm9VYnRjEq2KEbJp7VkPAHgZEXR7VABwCFvmDcztrfcWfmXxm6IH +o+AkF/nqdphLu7Q1yDQiF8Q8TuszuhqgQ7/1PrRcaSADrF15jJjQb05sILpGCT3e +lxJYId5RF0+fgTIqy03bAKB53+8V8cAkowI4rvPTmcFXhcG3rkDO6lyZixHhlpKi +PmXEzHh0kfsRjzkNBP0CKqPnu3D2iymROiPAH2cteaYe6jdD2HIjuVLk/TjX1ZFy +DlZCrJIwj0l8A2xAfLq8Gw5RSr0a9k5TiMD5nZtfd12Vd0K82vO32vmcjO2Igddc +VWccDDwUY/ZWV3uznkusOBrB8wba3ZsXA5hjJzs0KlTvQKPjX0y4lFMmZGbelwjt +pR5dRNLi5XTdMPzV0mAnvJhDTFEmME19Bh6AEsjuAz3gHUdwNTbSxUS3mF/hTL9k +v2wh5udUAOwqD1uEzqPJyG4JCJQozIDOEEZVixWqQ60b9wUHN8meqO4y9fxTdmHW +Vo5BAF1xEJhJJb0QY/O6GahPtWqb/Mr1rtPJAgMBAAGjUzBRMB0GA1UdDgQWBBSw +fQcOabXwX/v9F1hd2cmuIug56jAfBgNVHSMEGDAWgBSwfQcOabXwX/v9F1hd2cmu +Iug56jAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4ICAQAms8y6RVxl +mKSUbsU8JscYwOzcRUQJWETeIr4rtZvMHH+3vkdBU0yKxGpEm7U8J3+5oVTYPhbs +11ZAL+DvIZ6gT6pjDvECyVox1OkjNogz843fTMbNqjuuehjSKXwpMTy5/kmT2aLj +//nBi5UX1xo3RQ9vtmBwzZ3VFK99DFXraDOPS/yk43WV2uqdWsXCNvyEyCHmM1IB +9FQe2EFcO6s4/N+TarhIZ8Udhj5bl8d4eDd1yEckmTD4aHJBgMII2uEwrAxR5CT1 +tCqUKutvNrkXI5PIULvmy+Lwm7PJAC7grPtUHK6anSugpljd7bFj18fHH9APiC45 +Ou4OOK1BUZogCEo7rD36UlanxQO0GEzgDCVEoEdoe0WRdc6T9b4fM8vpQqwBdf9t +nkPB8oLCKerqqYwCiMuWm4BcRmExA7ypIkUCcluGO9/kTmdps3NqOvET9oLTjXuA +z5TPmaK5a3poKLoxBfv6WfRTgisOnMNTsjL1R8+xuhEn5hSlE2r3wAi8Cys9Z9PV +LhTj0SRTXILd2NW3lO8QfO0pGdjgk90GqkyUY9YjuiMVPvdUAFQsHm+0GEZEXjOD +Bw7tLSJQ4IKhfactg/Puxd15ahcWAxeelyED+w/zVGdHYblqbvfdtiGj370KVhoj +DL5HkdPa0IhTPqMBnmoVQ4C/WzKofXBjQQ== +-----END CERTIFICATE----- diff --git a/docker/test/integration/runner/misc/rabbitmq/generate_certs.sh b/docker/test/integration/runner/misc/rabbitmq/generate_certs.sh new file mode 100755 index 00000000000..442d2fe004f --- /dev/null +++ b/docker/test/integration/runner/misc/rabbitmq/generate_certs.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +# 1. Generate CA's private key and self-signed certificate +openssl req -newkey rsa:4096 -x509 -days 3650 -nodes -batch -keyout ca-key.pem -out ca-cert.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=ca" + +# 2. Generate server's private key and certificate signing request (CSR) +openssl req -newkey rsa:4096 -nodes -batch -keyout server-key.pem -out server-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=server" + +# 3. Use CA's private key to sign server's CSR and get back the signed certificate +openssl x509 -req -days 3650 -in server-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -extfile server-ext.cnf -out server-cert.pem diff --git a/docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf b/docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf new file mode 100644 index 00000000000..307871ba589 --- /dev/null +++ b/docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf @@ -0,0 +1,15 @@ +loopback_users.guest = false +listeners.tcp.default = 5672 +default_pass = clickhouse +default_user = root +management.tcp.port = 15672 + +log.file = /rabbitmq_logs/rabbit.log +log.file.level = debug + +listeners.ssl.default = 5671 +ssl_options.verify = verify_none +ssl_options.fail_if_no_peer_cert = false +ssl_options.cacertfile = /etc/rabbitmq/ca_cert.pem +ssl_options.certfile = /etc/rabbitmq/server_cert.pem +ssl_options.keyfile = /etc/rabbitmq/server_key.pem diff --git a/docker/test/integration/runner/misc/rabbitmq/server-cert.pem b/docker/test/integration/runner/misc/rabbitmq/server-cert.pem new file mode 100644 index 00000000000..338de91aa0f --- /dev/null +++ b/docker/test/integration/runner/misc/rabbitmq/server-cert.pem @@ -0,0 +1,33 @@ +-----BEGIN CERTIFICATE----- +MIIFpTCCA42gAwIBAgIUJvQslezZO09XgFGQCxOM6orIsWowDQYJKoZIhvcNAQEL +BQAwUjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDELMAkGA1UEAwwCY2EwHhcNMjMxMTE0 +MTgyODI5WhcNMzMxMTExMTgyODI5WjBWMQswCQYDVQQGEwJSVTETMBEGA1UECAwK +U29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMQ8w +DQYDVQQDDAZzZXJ2ZXIwggIiMA0GCSqGSIb3DQEBAQUAA4ICDwAwggIKAoICAQCe +o/K71WdKpVpdDvhaZy6wBVhFlu7j7DhfTSYvcPpAJfExmzO8JK3vh5/yGyAO1t79 +gAjqyXLMCZKw7ajM2rez9YnGYqaFi70BlTcU2KQ8LbFEYRc3cYNDmmWIKBpwpSri +We5SQrRLnDXqAn6T8FG5ejQ/t+1IUMrtZENB4lp8fBmEOJb5yr1TE++6EhiDBQho +cLDWWWP8b55kyZhqP/VgmId4lvboGMRKxbiRJ6/SPr/i/pteBD8jTYfbJr6ceXov +/p5yxIp61z5ry1anU7W3B8jTl/gj7SqtFdSnRajZ0DGJJAUKpiiJSCSlp5YB5Ub2 +eBBMHmdA5R1MuiU9TOA35nUW5wkhEOJXnBR/WCsYioVmn/+5dm6JPYiwp/TefYnr +x9iLbb/Tyx7MnXzeyvKg781SwmnvS6Blhtr0zhAW9szZz8cVHPBqFs6PzGs/5mwE +C+tM3Zp85aHd28nIT4NQLHdMDwVmGwmPdy4uavtYWMDhsuIyEU8hCZymiHhPnuHU +VbmfZ8GOTIzUgQAvZb0fL1Xow2Tf6XuARnvuU9weRttg9jSOqPuUENRsFXv0mU8M +EpQjrxry88Wfz7bBEjN5JHC16PB/Nu7zTGJ4/slThbxNv0bIONzvTBPbXrKnxw7Z +d9WhGJI+LQxRqLTynQe6yzDwIuW9LRdBNTp7CtQRwQIDAQABo28wbTArBgNVHREE +JDAigiBpbnRlZ3JhdGlvbi10ZXN0cy5jbGlja2hvdXNlLmNvbTAdBgNVHQ4EFgQU +54GvBUYWvMADpTz/zglwMlaJuskwHwYDVR0jBBgwFoAUsH0HDmm18F/7/RdYXdnJ +riLoOeowDQYJKoZIhvcNAQELBQADggIBADfNH6O6ay+xg0XmV6sR0n4j6PwL9Cnc +VjuCmHQbpFXfMvgCdfHvbtT0Y/pG7IoeKmrrm0JPvKa2E9Ht0j6ZnowQ2m9mJk8U +5Fd/PbC1I4KgVCw6HRSOcwqANJxOGe7RyN9PTZZ8fxzmzIR3FiQ2bXfr+LaotZOK +aVS8F8xCOzoMvL9LFls2YpEn20p/1EATIf2MFX3j9vKfcJVOyDJV4i5BMImStFLM +g3sdC96de/59yxt9khM0PNucU1ldNFs/kZVEcNSwGOAIgQEPwULJtDY+ZSWeROpX +EpWndN6zQsv1pdNvLtXsDXfi4YoH9QVaA/k4aFFJ08CjSZfMYmwyPOGsf/wqT65i +ADID2yb1A/FIIe/fM+d2gXHBVFBDmydJ1JCdCoYrEJgfWj1LO/0jLi34ZZ17Hu7F +D33fLARF9nlLzlUiWjcQlOjNoCM48AgG/3wHk4eiSfc/3PIJDuDGDa0NdtDeKKhH +XkP2ll4cMUH6EQ9KO1jHPmf5RokX4QJgH+ofO4U5XQFwc3lOyJzEQnED+wame7do +R7TE4F/OXhxLqA6DFkzXe89/kSCoAF9bjzmUn/ilrg8NXKKgprgHg4DJHgvCQVVC +34ab7Xj7msUm4D9vI+GAeUbUqnqCaWxDF6vCMT0Qq7iSVDxa/SV8TX8Vp2Zh+PSh +4m23Did+KjLq +-----END CERTIFICATE----- diff --git a/docker/test/integration/runner/misc/rabbitmq/server-key.pem b/docker/test/integration/runner/misc/rabbitmq/server-key.pem new file mode 100644 index 00000000000..92e93e8fba5 --- /dev/null +++ b/docker/test/integration/runner/misc/rabbitmq/server-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQgIBADANBgkqhkiG9w0BAQEFAASCCSwwggkoAgEAAoICAQCeo/K71WdKpVpd +DvhaZy6wBVhFlu7j7DhfTSYvcPpAJfExmzO8JK3vh5/yGyAO1t79gAjqyXLMCZKw +7ajM2rez9YnGYqaFi70BlTcU2KQ8LbFEYRc3cYNDmmWIKBpwpSriWe5SQrRLnDXq +An6T8FG5ejQ/t+1IUMrtZENB4lp8fBmEOJb5yr1TE++6EhiDBQhocLDWWWP8b55k +yZhqP/VgmId4lvboGMRKxbiRJ6/SPr/i/pteBD8jTYfbJr6ceXov/p5yxIp61z5r +y1anU7W3B8jTl/gj7SqtFdSnRajZ0DGJJAUKpiiJSCSlp5YB5Ub2eBBMHmdA5R1M +uiU9TOA35nUW5wkhEOJXnBR/WCsYioVmn/+5dm6JPYiwp/TefYnrx9iLbb/Tyx7M +nXzeyvKg781SwmnvS6Blhtr0zhAW9szZz8cVHPBqFs6PzGs/5mwEC+tM3Zp85aHd +28nIT4NQLHdMDwVmGwmPdy4uavtYWMDhsuIyEU8hCZymiHhPnuHUVbmfZ8GOTIzU +gQAvZb0fL1Xow2Tf6XuARnvuU9weRttg9jSOqPuUENRsFXv0mU8MEpQjrxry88Wf +z7bBEjN5JHC16PB/Nu7zTGJ4/slThbxNv0bIONzvTBPbXrKnxw7Zd9WhGJI+LQxR +qLTynQe6yzDwIuW9LRdBNTp7CtQRwQIDAQABAoICAA0lev0T3z5xW36wueYL/PN7 +TehebKeYsMc9BngR/bsJKea5fN0PkRZzf865brusFMifLp3+WbQM6wocd8uaKHUS +WPuGu1P/04bpDap9lYajJriK7ziaAI2+osFYyXAiT954I2bPvk8xv8oHsOOjm7Iq +LWBGZrSCdX6cu3IfRu5f/mFVqzVCFtRmp4wc6ckZxquZAx6QQ9fsjAzAJBBSAoyh +t0BICmgLfWDQ582no0tiBdbS0J9G7NCJIUQI/uzKqFSH3iuWm/84DSUzsZemOT3U +uFDInDil885qK7g87pQ2S5SY1o4eXOebgeX0cFrx3CKaqocUUewv0HDGUEW3NDFs +KhUvlJZIFgk6bMend16U6kfRCUsjLA22Rfxzanl53cGVywCeIMirnLYuEu0TsxyK +CblBvyhcpjrGi7FQskzR+J9LpZPnmtn6TAb7JCAALRVHcAGKhGeh613SjPUfkWb0 +KpDps08x8MWGEAALuHbOK0nMLFm+PuMt7+krqCeJET+XM44GT+6ZstrDv0RufxUN ++pkLW7AsVZoXcFvaOWjuyBvX/f6UHCSfueo0mB3H80WoftDIfdhM+AI7/oBTYCBx +Z8BtW+g7Eq3pOUg/Um7S7Z2bybBWE14kpi95gRf3upEYPqHJUpJPdu20lk24iAt9 +LCXF4AjZBIdAuyJrYOJBAoIBAQDd/Bm14WvmBOablGLn6hmohi6M75D+/eQanlg9 +eJhXJUVd8FzOTjKi70EHWvkqswenNDbe/WGtImqG+9G+N/ol2qhi5xVSQ2XQmcVQ +U+k15Bzm9xKM0OqsStFvRgP1Cy6Ms3/jxr5JEEwUepmjvWTDGTlhTQASA/D7Uh2q +5HpPiHEVm4g5eTAYWeAbI6cGwVS0L4y6xkFGde37Kh2P8ZodWB+d3fglVu4Ok9Nf +wE2f8MK2ewQ0SbF/Nj2WjlVomvOvOJG/2CDLuiH/vc4YUvLAm8pNwvsmgtSh1Okt +E/HfXegrlPPEgw6owqoQFt+aGUITgEhiwEVAcYS0pXzzkQX5AoIBAQC28wJ8ueKr +fINpJM2pSc7WRDFduP5yGsRreSLBXLKMbvOlIVb3PaWp11Cg3+X5O90bPXYJ9mBI +WGR0g14/VD8edxs2D5TUZcP4/vKXGHaWRY9Z4A3jVpjzAxAaviNDHJ08tLXEMXZQ +lbA7dX8z6lpoQfwnPzjBwB01mVegwXPeIwIIfT/FmAiGzvSnAMXBGSGWRRdzof0M +/vPFbgllcQmM4AnEGcErCgFRpwcssO87T2jnvf6QVE5JCcnUcGIli1ThxCU9TRZM +5s6R7Nvk3/UjwcpRcqMtnGpTT2QXSnRwvWUfM+bKTwaxz4PjqKpgIc11kwJAjlxk +4CxYf1mDGLwJAoIBAGFJRTNS8ejDKRXyOE6PaGNVOz2FGLTILJoF34JBQfKfYQFE +gEfiOYry9Dr3AdBW2fnLhmi//3jTZoB2CHwnKDhC1h1STSPaadq8KZ+ExuZZbNlE +WxrfzJlpyNPNiZpxJht/54K57Vc0D0PCX2dFb82ZVm5wQqGinJBocpwcugX1NCpW +GaOmmw9xBCigvWjWffriA/kvPhhVQtEaqg4Vwoctwd18FG645Gf7HV4Pd3WrHIrA +6xzHV0T7To6XHpNTpYybbDT50ZW3o4LjellqsPz8yfK+izdbizjJiM+6t/w+uauw +Ag2Tqm8HsWSPwbtVaoIFbLPqs+8EUTaieFp+qnECggEAVuaTdd9uFfrtCNKchh8z +CoAV2uj2pAim6E3//k0j2qURQozVnFdCC6zk9aWkvYB8BGZrXUwUbAjgnp+P8xD3 +cmctG77G+STls66WWMMcAUFFWHGe5y/JMxVvXuSWJ1i+L4m/FVRRWPHhZjznkSdu +jjtZpOLY+N9igIU4JHn/qbKDUrj7w8X1tuMzPuiVBqYDWDe1bg2x/6xS6qLb/71z +xeDdgrKhGOqFud1XARmCaW/M6tdKxg/lp7fokOpZFHBcf2kGL1ogj6LK2HHj+ZGQ +Bc4VZh7H9/BmaPA7IP0S1kKAeBPVOp/TFD737Pm/BC7KQ2DzHusAZEI/jkHfqO/k +0QKCAQEAuiYLn9iLgk4uQO9oaSBGWKrJsR2L2dqI7IWU0X9xJlsQrJKcEeWg4LXt +djLsz0HrxZV/c+Pnh79hmFlBoEmH+hz32D/xd+/qrwwAcMkHAwMbznJu0IIuW2O9 +Uzma++7SvVmr9H0DkUwXFP3jn1A2n3uuI4czqtQ8N7GiH0UAWR5CsIP7azHvZTSj +s4Fzf8rTE6pNqVgQXjrVbI9H/h0uPP4alJbhnPba9mgB1cGmfBEnPkKgYNqSZse+ +95G2TlcK74sKBUSdBKqYBZ4ZUeTXV974Nva9guE9vzDQt1Cj6k0HWISVPUshPzIh +qrdHdxcM6yhA0Z0Gu6zj+Zsy4lU8gA== +-----END PRIVATE KEY----- diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 983e52ca294..837263bd70f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -102,18 +102,32 @@ def rabbitmq_setup_teardown(): # Tests -def test_rabbitmq_select(rabbitmq_cluster): +@pytest.mark.parametrize( + "secure", + [ + pytest.param(0), + pytest.param(1), + ], +) +def test_rabbitmq_select(rabbitmq_cluster, secure): + port = 5672 + if secure: + port = 5671 + instance.query( """ CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = '{}:5672', + SETTINGS rabbitmq_host_port = '{}:{}', rabbitmq_exchange_name = 'select', rabbitmq_commit_on_select = 1, rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; + rabbitmq_row_delimiter = '\\n', + rabbitmq_secure = {}; """.format( - rabbitmq_cluster.rabbitmq_host + rabbitmq_cluster.rabbitmq_host, + port, + secure ) ) @@ -3442,18 +3456,18 @@ def test_rabbitmq_handle_error_mode_stream(rabbitmq_cluster): rabbitmq_row_delimiter = '\\n', rabbitmq_handle_error_mode = 'stream'; - + CREATE TABLE test.errors (error Nullable(String), broken_message Nullable(String)) ENGINE = MergeTree() ORDER BY tuple(); CREATE MATERIALIZED VIEW test.errors_view TO test.errors AS SELECT _error as error, _raw_message as broken_message FROM test.rabbit where not isNull(_error); - + CREATE TABLE test.data (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; - + CREATE MATERIALIZED VIEW test.view TO test.data AS SELECT key, value FROM test.rabbit; """.format( From 369aaef92fb280c4b7ae4e5e04d0da2930718e3b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 14 Nov 2023 20:26:32 +0000 Subject: [PATCH 344/813] Automatic style fix --- tests/integration/test_storage_rabbitmq/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 837263bd70f..adb7f59769a 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -125,9 +125,7 @@ def test_rabbitmq_select(rabbitmq_cluster, secure): rabbitmq_row_delimiter = '\\n', rabbitmq_secure = {}; """.format( - rabbitmq_cluster.rabbitmq_host, - port, - secure + rabbitmq_cluster.rabbitmq_host, port, secure ) ) From 07452b613a6b147c53530d6325fc9038ce58f675 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 14 Nov 2023 20:35:54 +0000 Subject: [PATCH 345/813] Fix certificate's file names --- docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf | 6 +++--- docker/test/integration/runner/misc/rabbitmq/server-ext.cnf | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) create mode 100644 docker/test/integration/runner/misc/rabbitmq/server-ext.cnf diff --git a/docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf b/docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf index 307871ba589..258a282907a 100644 --- a/docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf +++ b/docker/test/integration/runner/misc/rabbitmq/rabbitmq.conf @@ -10,6 +10,6 @@ log.file.level = debug listeners.ssl.default = 5671 ssl_options.verify = verify_none ssl_options.fail_if_no_peer_cert = false -ssl_options.cacertfile = /etc/rabbitmq/ca_cert.pem -ssl_options.certfile = /etc/rabbitmq/server_cert.pem -ssl_options.keyfile = /etc/rabbitmq/server_key.pem +ssl_options.cacertfile = /etc/rabbitmq/ca-cert.pem +ssl_options.certfile = /etc/rabbitmq/server-cert.pem +ssl_options.keyfile = /etc/rabbitmq/server-key.pem diff --git a/docker/test/integration/runner/misc/rabbitmq/server-ext.cnf b/docker/test/integration/runner/misc/rabbitmq/server-ext.cnf new file mode 100644 index 00000000000..49859873222 --- /dev/null +++ b/docker/test/integration/runner/misc/rabbitmq/server-ext.cnf @@ -0,0 +1 @@ +subjectAltName=DNS:integration-tests.clickhouse.com From 3d95ac5f2a3cadda0eb7571b0a4bf314f63ca139 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 14 Nov 2023 21:52:07 +0100 Subject: [PATCH 346/813] Fix flaky test_replicated_merge_tree_encryption_codec. --- .../test_replicated_merge_tree_encryption_codec/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_replicated_merge_tree_encryption_codec/test.py b/tests/integration/test_replicated_merge_tree_encryption_codec/test.py index a50f8341ee7..d2dbc7c5466 100644 --- a/tests/integration/test_replicated_merge_tree_encryption_codec/test.py +++ b/tests/integration/test_replicated_merge_tree_encryption_codec/test.py @@ -92,6 +92,8 @@ def test_different_keys(): create_table() insert_data() + node1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") + assert "BAD_DECRYPT" in node1.query_and_get_error("SELECT * FROM tbl") assert "BAD_DECRYPT" in node2.query_and_get_error("SELECT * FROM tbl") From f04db1ba4c26d679acf950ab239bf7bfd7506bb6 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 14 Nov 2023 21:06:49 +0000 Subject: [PATCH 347/813] Fix tests --- tests/queries/0_stateless/01933_invalid_date.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01933_invalid_date.sql b/tests/queries/0_stateless/01933_invalid_date.sql index aac09c99e60..b9ea9319aea 100644 --- a/tests/queries/0_stateless/01933_invalid_date.sql +++ b/tests/queries/0_stateless/01933_invalid_date.sql @@ -1,4 +1,4 @@ -SELECT toDate('07-08-2019'); -- { serverError 6 } +SELECT toDate('07-08-2019'); -- { serverError 38 } SELECT toDate('2019-0708'); -- { serverError 38 } SELECT toDate('201907-08'); -- { serverError 38 } SELECT toDate('2019^7^8'); @@ -6,5 +6,5 @@ SELECT toDate('2019^7^8'); CREATE TEMPORARY TABLE test (d Date); INSERT INTO test VALUES ('2018-01-01'); -SELECT * FROM test WHERE d >= '07-08-2019'; -- { serverError 53 } +SELECT * FROM test WHERE d >= '07-08-2019'; -- { serverError 38 } SELECT * FROM test WHERE d >= '2019-07-08'; From 58b9bde0faeb87bc3c616206fa550ab5ccb03195 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 14 Nov 2023 21:15:30 +0000 Subject: [PATCH 348/813] Better docs for virtual columns in Kafka/RabbitMQ/NATS/FileLog --- .../table-engines/integrations/kafka.md | 20 +++++++++---------- .../table-engines/integrations/nats.md | 8 ++++---- .../table-engines/integrations/rabbitmq.md | 18 ++++++++--------- .../engines/table-engines/special/filelog.md | 10 +++++----- 4 files changed, 28 insertions(+), 28 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index f16f9692bb6..de1a090d491 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -238,19 +238,19 @@ Example: ## Virtual Columns {#virtual-columns} -- `_topic` — Kafka topic. -- `_key` — Key of the message. -- `_offset` — Offset of the message. -- `_timestamp` — Timestamp of the message. -- `_timestamp_ms` — Timestamp in milliseconds of the message. -- `_partition` — Partition of Kafka topic. -- `_headers.name` — Array of message's headers keys. -- `_headers.value` — Array of message's headers values. +- `_topic` — Kafka topic. Data type: `LowCardinality(String)`. +- `_key` — Key of the message. Data type: `String`. +- `_offset` — Offset of the message. Data type: `UInt64`. +- `_timestamp` — Timestamp of the message Data type: `Nullable(DateTime)`. +- `_timestamp_ms` — Timestamp in milliseconds of the message. Data type: `Nullable(DateTime64(3))`. +- `_partition` — Partition of Kafka topic. Data type: `UInt64`. +- `_headers.name` — Array of message's headers keys. Data type: `Array(String)`. +- `_headers.value` — Array of message's headers values. Data type: `Array(String)`. Additional virtual columns when `kafka_handle_error_mode='stream'`: -- `_raw_message` - Raw message that couldn't be parsed successfully. -- `_error` - Exception message happened during failed parsing. +- `_raw_message` - Raw message that couldn't be parsed successfully. Data type: `String`. +- `_error` - Exception message happened during failed parsing. Data type: `String`. Note: `_raw_message` and `_error` virtual columns are filled only in case of exception during parsing, they are always empty when message was parsed successfully. diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index 5819a8e95c8..37a41159fab 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -163,14 +163,14 @@ If you want to change the target table by using `ALTER`, we recommend disabling ## Virtual Columns {#virtual-columns} -- `_subject` - NATS message subject. +- `_subject` - NATS message subject. Data type: `String`. Additional virtual columns when `kafka_handle_error_mode='stream'`: -- `_raw_message` - Raw message that couldn't be parsed successfully. -- `_error` - Exception message happened during failed parsing. +- `_raw_message` - Raw message that couldn't be parsed successfully. Data type: `Nullable(String)`. +- `_error` - Exception message happened during failed parsing. Data type: `Nullable(String)`. -Note: `_raw_message` and `_error` virtual columns are filled only in case of exception during parsing, they are always empty when message was parsed successfully. +Note: `_raw_message` and `_error` virtual columns are filled only in case of exception during parsing, they are always `NULL` when message was parsed successfully. ## Data formats support {#data-formats-support} diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 10e7146ff85..53c6e089a70 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -184,19 +184,19 @@ Example: ## Virtual Columns {#virtual-columns} -- `_exchange_name` - RabbitMQ exchange name. -- `_channel_id` - ChannelID, on which consumer, who received the message, was declared. -- `_delivery_tag` - DeliveryTag of the received message. Scoped per channel. -- `_redelivered` - `redelivered` flag of the message. -- `_message_id` - messageID of the received message; non-empty if was set, when message was published. -- `_timestamp` - timestamp of the received message; non-empty if was set, when message was published. +- `_exchange_name` - RabbitMQ exchange name. Data type: `String`. +- `_channel_id` - ChannelID, on which consumer, who received the message, was declared. Data type: `String`. +- `_delivery_tag` - DeliveryTag of the received message. Scoped per channel. Data type: `UInt64`. +- `_redelivered` - `redelivered` flag of the message. Data type: `UInt8`. +- `_message_id` - messageID of the received message; non-empty if was set, when message was published. Data type: `String`. +- `_timestamp` - timestamp of the received message; non-empty if was set, when message was published. Data type: `UInt64`. Additional virtual columns when `kafka_handle_error_mode='stream'`: -- `_raw_message` - Raw message that couldn't be parsed successfully. -- `_error` - Exception message happened during failed parsing. +- `_raw_message` - Raw message that couldn't be parsed successfully. Data type: `Nullable(String)`. +- `_error` - Exception message happened during failed parsing. Data type: `Nullable(String)`. -Note: `_raw_message` and `_error` virtual columns are filled only in case of exception during parsing, they are always empty when message was parsed successfully. +Note: `_raw_message` and `_error` virtual columns are filled only in case of exception during parsing, they are always `NULL` when message was parsed successfully. ## Data formats support {#data-formats-support} diff --git a/docs/en/engines/table-engines/special/filelog.md b/docs/en/engines/table-engines/special/filelog.md index 0c2a2601fc9..eef9a17444e 100644 --- a/docs/en/engines/table-engines/special/filelog.md +++ b/docs/en/engines/table-engines/special/filelog.md @@ -94,12 +94,12 @@ If you want to change the target table by using `ALTER`, we recommend disabling ## Virtual Columns {#virtual-columns} -- `_filename` - Name of the log file. -- `_offset` - Offset in the log file. +- `_filename` - Name of the log file. Data type: `LowCardinality(String)`. +- `_offset` - Offset in the log file. Data type: `UInt64`. Additional virtual columns when `kafka_handle_error_mode='stream'`: -- `_raw_record` - Raw record that couldn't be parsed successfully. -- `_error` - Exception message happened during failed parsing. +- `_raw_record` - Raw record that couldn't be parsed successfully. Data type: `Nullable(String)`. +- `_error` - Exception message happened during failed parsing. Data type: `Nullable(String)`. -Note: `_raw_record` and `_error` virtual columns are filled only in case of exception during parsing, they are always empty when message was parsed successfully. +Note: `_raw_record` and `_error` virtual columns are filled only in case of exception during parsing, they are always `NULL` when message was parsed successfully. From bd66ac483c9dcd770ea1182fd6f8d11f54ef04d2 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 14 Nov 2023 23:03:06 +0000 Subject: [PATCH 349/813] Load queries for dashboard page from new `system.dashboards` table --- programs/server/dashboard.html | 338 +++++++----------- .../System/StorageSystemDashboards.cpp | 226 ++++++++++++ src/Storages/System/StorageSystemDashboards.h | 28 ++ src/Storages/System/attachSystemTables.cpp | 2 + 4 files changed, 390 insertions(+), 204 deletions(-) create mode 100644 src/Storages/System/StorageSystemDashboards.cpp create mode 100644 src/Storages/System/StorageSystemDashboards.h diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index d4fa1626873..f3096d72e14 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -202,6 +202,10 @@ margin-right: 0.25rem; } + #chart-params .param { + width: 6%; + } + input { font-family: Liberation Sans, DejaVu Sans, sans-serif, Noto Color Emoji, Apple Color Emoji, Segoe UI Emoji; outline: none; @@ -240,14 +244,13 @@ font-weight: bold; user-select: none; cursor: pointer; - margin-bottom: 1rem; } #run:hover { filter: contrast(125%); } - #add, #reload, #edit { + #add, #reload, #edit, #search { padding: 0.25rem 0.5rem; text-align: center; font-weight: bold; @@ -264,10 +267,16 @@ height: 3ex; } - #add:hover, #reload:hover, #edit:hover { + #add:hover, #reload:hover, #edit:hover, #search:hover { background: var(--button-background-color); } + #search-query { + float: right; + width: 36%; + } + + #auth-error { align-self: center; width: 60%; @@ -445,6 +454,7 @@ +
@@ -501,151 +511,10 @@ const errorMessages = [ } ] -/// This is just a demo configuration of the dashboard. -let queries = [ - { - "title": "Queries/second", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_Query) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "CPU Usage (cores)", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSCPUVirtualTimeMicroseconds) / 1000000 -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Queries Running", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_Query) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Merges Running", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_Merge) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Selected Bytes/second", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_SelectedBytes) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "IO Wait", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSIOWaitMicroseconds) / 1000000 -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "CPU Wait", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSCPUWaitMicroseconds) / 1000000 -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "OS CPU Usage (Userspace)", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) -FROM system.asynchronous_metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -AND metric = 'OSUserTimeNormalized' -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "OS CPU Usage (Kernel)", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) -FROM system.asynchronous_metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -AND metric = 'OSSystemTimeNormalized' -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Read From Disk", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSReadBytes) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Read From Filesystem", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSReadChars) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Memory (tracked)", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_MemoryTracking) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Load Average (15 minutes)", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) -FROM system.asynchronous_metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -AND metric = 'LoadAverage15' -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Selected Rows/second", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_SelectedRows) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Inserted Rows/second", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_InsertedRows) -FROM system.metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Total MergeTree Parts", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) -FROM system.asynchronous_metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -AND metric = 'TotalPartsOfMergeTreeTables' -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - }, - { - "title": "Max Parts For Partition", - "query": `SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(value) -FROM system.asynchronous_metric_log -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -AND metric = 'MaxPartCountForPartition' -GROUP BY t -ORDER BY t WITH FILL STEP {rounding:UInt32}` - } -]; +/// Query to fill `queries` list for the dashboard +let search_query = `SELECT title, query FROM system.dashboards WHERE dashboard = 'overview'`; +let queries = []; /// Query parameters with predefined default values. /// All other parameters will be automatically found in the queries. @@ -683,7 +552,7 @@ function findParamsInQuery(query, new_params) { } function findParamsInQueries() { - let new_params = {} + let new_params = {}; queries.forEach(q => findParamsInQuery(q.query, new_params)); params = new_params; } @@ -958,7 +827,7 @@ function insertChart(i) { charts.appendChild(chart); return {chart: chart, textarea: query_editor_textarea}; -}; +} document.getElementById('add').addEventListener('click', e => { queries.push({ title: '', query: '' }); @@ -972,9 +841,12 @@ document.getElementById('add').addEventListener('click', e => { }); document.getElementById('reload').addEventListener('click', e => { - reloadAll(); + reloadAll(false); }); +document.getElementById('search').addEventListener('click', e => { + reloadAll(true); +}); let mass_editor_active = false; @@ -1085,13 +957,8 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend- }; } -async function draw(idx, chart, url_params, query) { - if (plots[idx]) { - plots[idx].destroy(); - plots[idx] = null; - } - - host = document.getElementById('url').value; +async function doFetch(query, url_params = '') { + host = document.getElementById('url').value || host; user = document.getElementById('user').value; password = document.getElementById('password').value; @@ -1135,6 +1002,17 @@ async function draw(idx, chart, url_params, query) { } } + return {data, error}; +} + +async function draw(idx, chart, url_params, query) { + if (plots[idx]) { + plots[idx].destroy(); + plots[idx] = null; + } + + let {data, error} = await doFetch(query, url_params); + if (!error) { if (!Array.isArray(data)) { error = "Query should return an array."; @@ -1151,7 +1029,7 @@ async function draw(idx, chart, url_params, query) { let title_div = chart.querySelector('.title'); if (error) { error_div.firstChild.data = error; - title_div.style.display = 'none'; + title_div.style.display = 'none'; error_div.style.display = 'block'; return false; } else { @@ -1194,7 +1072,7 @@ async function draw(idx, chart, url_params, query) { /// Set title const title = queries[idx] && queries[idx].title ? queries[idx].title.replaceAll(/\{(\w+)\}/g, (_, name) => params[name] ) : ''; chart.querySelector('.title').firstChild.data = title; - return true + return true; } function showAuthError(message) { @@ -1203,6 +1081,7 @@ function showAuthError(message) { charts.style.opacity = '0'; document.getElementById('add').style.display = 'none'; document.getElementById('edit').style.display = 'none'; + document.getElementById('search-span').style.display = 'none'; const authError = document.getElementById('auth-error'); authError.textContent = message; @@ -1239,20 +1118,20 @@ async function drawAll() { if (firstLoad) { firstLoad = false; } else { - enableReloadButton(); - enableRunButton(); + enableButtons(); } if (results.includes(true)) { const element = document.querySelector('.inputs'); element.classList.remove('unconnected'); document.getElementById('add').style.display = 'inline-block'; document.getElementById('edit').style.display = 'inline-block'; - } + document.getElementById('search-span').style.display = ''; + } else { const charts = document.getElementById('charts') charts.style.height = '0px'; } - }) + }); } function resize() { @@ -1266,58 +1145,98 @@ function resize() { new ResizeObserver(resize).observe(document.body); -function disableReloadButton() { - const reloadButton = document.getElementById('reload') - reloadButton.value = 'Reloading…' - reloadButton.disabled = true - reloadButton.classList.add('disabled') +function disableButtons() { + const reloadButton = document.getElementById('reload'); + reloadButton.value = 'Reloading…'; + reloadButton.disabled = true; + reloadButton.classList.add('disabled'); + + const runButton = document.getElementById('run'); + runButton.value = 'Reloading…'; + runButton.disabled = true; + runButton.classList.add('disabled'); + + const searchButton = document.getElementById('search'); + searchButton.value = '…'; + searchButton.disabled = true; + searchButton.classList.add('disabled'); } -function disableRunButton() { - const runButton = document.getElementById('run') - runButton.value = 'Reloading…' - runButton.disabled = true - runButton.classList.add('disabled') +function enableButtons() { + const reloadButton = document.getElementById('reload'); + reloadButton.value = 'Reload'; + reloadButton.disabled = false; + reloadButton.classList.remove('disabled'); + + const runButton = document.getElementById('run'); + runButton.value = 'Ok'; + runButton.disabled = false; + runButton.classList.remove('disabled'); + + const searchButton = document.getElementById('search'); + searchButton.value = '🔎'; + searchButton.disabled = false; + searchButton.classList.remove('disabled'); } -function enableReloadButton() { - const reloadButton = document.getElementById('reload') - reloadButton.value = 'Reload' - reloadButton.disabled = false - reloadButton.classList.remove('disabled') -} - -function enableRunButton() { - const runButton = document.getElementById('run') - runButton.value = 'Ok' - runButton.disabled = false - runButton.classList.remove('disabled') -} - -function reloadAll() { - updateParams(); - drawAll(); - saveState(); - disableReloadButton(); - disableRunButton(); +async function reloadAll(do_search) { + disableButtons(); + try { + if (do_search) { + await searchQueries(document.getElementById('search-query').value); + } + updateParams(); + await drawAll(); + saveState(); + } catch (e) { + showAuthError(e.toString()); + } + enableButtons(); } document.getElementById('params').onsubmit = function(event) { - reloadAll(); + let do_search = document.activeElement === document.getElementById('search-query'); + reloadAll(do_search); event.preventDefault(); } function saveState() { - const state = { host: host, user: user, queries: queries, params: params }; + const state = { host: host, user: user, queries: queries, params: params, search_query: search_query }; history.pushState(state, '', window.location.pathname + (window.location.search || '') + '#' + btoa(JSON.stringify(state))); } +async function searchQueries(query) { + search_query = query; + + let {data, error} = await doFetch(search_query); + if (error) { + throw new Error(error); + } + if (!Array.isArray(data)) { + throw new Error("Search query should return an array."); + } else if (data.length == 0) { + throw new Error("Search query returned empty result."); + } else if (data.length != 2) { + throw new Error("Search query should return exactly two columns: title and query."); + } else if (!Array.isArray(data[0]) || !Array.isArray(data[1]) || data[0].length != data[1].length) { + throw new Error("Wrong data format of the search query."); + } + + queries = []; + for (let i = 0; i < data[0].length; i++) { + queries.push({title: data[0][i], query: data[1][i]}); + } + + regenerate(); +} + function regenerate() { document.getElementById('url').value = host; document.getElementById('user').value = user; document.getElementById('password').value = password; + document.getElementById('search-query').value = search_query; findParamsInQueries(); buildParams(); @@ -1336,7 +1255,7 @@ function regenerate() { window.onpopstate = function(event) { if (!event.state) { return; } - ({host, user, queries, params} = event.state); + ({host, user, queries, params, search_query} = event.state); regenerate(); drawAll(); @@ -1344,19 +1263,30 @@ window.onpopstate = function(event) { if (window.location.hash) { try { - ({host, user, queries, params} = JSON.parse(atob(window.location.hash.substring(1)))); + ({host, user, queries, params, search_query} = JSON.parse(atob(window.location.hash.substring(1)))); } catch {} } -regenerate(); - -let new_theme = window.localStorage.getItem('theme'); -if (new_theme && new_theme != theme) { - setTheme(new_theme); -} else { - drawAll(); +async function start() { + try { + if (queries.length == 0) { + await searchQueries(search_query); + } else { + regenerate(); + } + let new_theme = window.localStorage.getItem('theme'); + if (new_theme && new_theme != theme) { + setTheme(new_theme); + } else { + drawAll(); + } + } catch (e) { + showAuthError(e.toString()); + } } +start(); + diff --git a/src/Storages/System/StorageSystemDashboards.cpp b/src/Storages/System/StorageSystemDashboards.cpp new file mode 100644 index 00000000000..3e22a6c0664 --- /dev/null +++ b/src/Storages/System/StorageSystemDashboards.cpp @@ -0,0 +1,226 @@ +#include +#include +#include + +namespace DB +{ + +NamesAndTypesList StorageSystemDashboards::getNamesAndTypes() +{ + return { + {"dashboard", std::make_shared()}, + {"title", std::make_shared()}, + {"query", std::make_shared()}, + }; +} + +String trim(const char * text) +{ + std::string_view view(text); + ::trim(view, '\n'); + return String(view); +} + +void StorageSystemDashboards::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +{ + static const std::vector> dashboards + { + { + { "dashboard", "overview" }, + { "title", "Queries/second" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_Query) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "CPU Usage (cores)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSCPUVirtualTimeMicroseconds) / 1000000 +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Queries Running" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_Query) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Merges Running" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_Merge) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Selected Bytes/second" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_SelectedBytes) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "IO Wait" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSIOWaitMicroseconds) / 1000000 +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "CPU Wait" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSCPUWaitMicroseconds) / 1000000 +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "OS CPU Usage (Userspace)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) +FROM system.asynchronous_metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSUserTimeNormalized' +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "OS CPU Usage (Kernel)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) +FROM system.asynchronous_metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSSystemTimeNormalized' +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Read From Disk" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSReadBytes) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Read From Filesystem" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSReadChars) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Memory (tracked)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_MemoryTracking) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Load Average (15 minutes)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) +FROM system.asynchronous_metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'LoadAverage15' +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Selected Rows/second" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_SelectedRows) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Inserted Rows/second" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_InsertedRows) +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Total MergeTree Parts" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) +FROM system.asynchronous_metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'TotalPartsOfMergeTreeTables' +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "overview" }, + { "title", "Max Parts For Partition" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(value) +FROM system.asynchronous_metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition' +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + } + }; + + for (const auto & row : dashboards) + { + size_t i = 0; + res_columns[i++]->insert(row.at("dashboard")); + res_columns[i++]->insert(row.at("title")); + res_columns[i++]->insert(row.at("query")); + } +} + +} diff --git a/src/Storages/System/StorageSystemDashboards.h b/src/Storages/System/StorageSystemDashboards.h new file mode 100644 index 00000000000..cbd7c5bbf57 --- /dev/null +++ b/src/Storages/System/StorageSystemDashboards.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class Context; + + +class StorageSystemDashboards final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemDashboards"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 6d875208fbb..1c4d1e16c92 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -85,6 +85,7 @@ #include #include #include +#include #if defined(__ELF__) && !defined(OS_FREEBSD) #include @@ -205,6 +206,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "user_processes"); attach(context, system_database, "jemalloc_bins"); attach(context, system_database, "s3queue"); + attach(context, system_database, "dashboards"); if (has_zookeeper) { From 4efef5b39f85af9128f61f8b47a7ebab8d935e6c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 15 Nov 2023 02:20:44 +0100 Subject: [PATCH 350/813] Disable test with Replicated database --- tests/queries/0_stateless/02911_row_policy_on_cluster.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02911_row_policy_on_cluster.sql b/tests/queries/0_stateless/02911_row_policy_on_cluster.sql index 6e94e301733..0c60bb5a6b9 100644 --- a/tests/queries/0_stateless/02911_row_policy_on_cluster.sql +++ b/tests/queries/0_stateless/02911_row_policy_on_cluster.sql @@ -1,4 +1,5 @@ --- Tags: no-parallel, zookeeper +-- Tags: no-parallel, zookeeper, no-replicated-database +-- Tag no-replicated-database: distributed_ddl_output_mode is none DROP ROW POLICY IF EXISTS 02911_rowpolicy ON default.* ON CLUSTER test_shard_localhost; DROP USER IF EXISTS 02911_user ON CLUSTER test_shard_localhost; From ed7d1dbb82db73892b7cb765cadc293874b3d29b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Nov 2023 07:43:55 +0100 Subject: [PATCH 351/813] Pick up linker's trace files --- utils/prepare-time-trace/prepare-time-trace.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/prepare-time-trace/prepare-time-trace.sh b/utils/prepare-time-trace/prepare-time-trace.sh index 7e585db2000..5f4aad4c0b9 100755 --- a/utils/prepare-time-trace/prepare-time-trace.sh +++ b/utils/prepare-time-trace/prepare-time-trace.sh @@ -44,9 +44,9 @@ ORDER BY (date, file, name, args_name); INPUT_DIR=$1 OUTPUT_DIR=$2 -find "$INPUT_DIR" -name '*.json' | grep -P '\.(c|cpp|cc|cxx)\.json$' | xargs -P $(nproc) -I{} bash -c " +find "$INPUT_DIR" -name '*.json' -or -name '*.time-trace' | grep -P '\.(c|cpp|cc|cxx)\.json|\.time-trace$' | xargs -P $(nproc) -I{} bash -c " - ORIGINAL_FILENAME=\$(echo '{}' | sed -r -e 's!\.json\$!!; s!/CMakeFiles/[^/]+\.dir!!') + ORIGINAL_FILENAME=\$(echo '{}' | sed -r -e 's!\.(json|time-trace)\$!!; s!/CMakeFiles/[^/]+\.dir!!') LIBRARY_NAME=\$(echo '{}' | sed -r -e 's!^.*/CMakeFiles/([^/]+)\.dir/.*\$!\1!') START_TIME=\$(jq '.beginningOfTime' '{}') From 5309dc05ef44da063ad095b91776dbfb3e849fb0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 15 Nov 2023 07:42:19 +0000 Subject: [PATCH 352/813] Fix test --- tests/integration/test_keeper_four_word_command/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 71501133ae7..84dd2a2fd93 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -287,7 +287,7 @@ def test_cmd_conf(started_cluster): assert result["quorum_reads"] == "false" assert result["force_sync"] == "true" - assert result["compress_logs"] == "true" + assert result["compress_logs"] == "false" assert result["compress_snapshots_with_zstd_format"] == "true" assert result["configuration_change_tries_count"] == "20" From aea43bdfadd74e8fa81a8f1f4a6cc4a3fd8704c2 Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Wed, 15 Nov 2023 10:06:00 +0200 Subject: [PATCH 353/813] fix typo in ClickHouseDictionarySource hostnmae -> hostname --- src/Dictionaries/ClickHouseDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 92fae2bc495..b788124d22c 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -222,7 +222,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) { validateNamedCollection( *named_collection, {}, ValidateKeysMultiset{ - "secure", "host", "hostnmae", "port", "user", "username", "password", "quota_key", "name", + "secure", "host", "hostname", "port", "user", "username", "password", "quota_key", "name", "db", "database", "table","query", "where", "invalidate_query", "update_field", "update_lag"}); const auto secure = named_collection->getOrDefault("secure", false); From 9d965368a2a337ef6cc2566462670e26fc9e2799 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 15 Nov 2023 08:36:24 +0000 Subject: [PATCH 354/813] Fix build --- src/Backups/RestoreCoordinationRemote.cpp | 2 +- src/Common/escapeForFileName.cpp | 5 ----- src/Common/escapeForFileName.h | 1 - 3 files changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index b54231afcf7..60a83c580f0 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -245,7 +245,7 @@ bool RestoreCoordinationRemote::acquireInsertingDataForKeeperMap(const String & with_retries.renewZooKeeper(zk); /// we need to remove leading '/' from root_zk_path - auto normalized_root_zk_path = std::string_view{root_zk_path}.substr(1); + auto normalized_root_zk_path = root_zk_path.substr(1); std::string restore_lock_path = fs::path(zookeeper_path) / "keeper_map_tables" / escapeForFileName(normalized_root_zk_path); zk->createAncestors(restore_lock_path); auto code = zk->tryCreate(restore_lock_path, table_unique_id, zkutil::CreateMode::Persistent); diff --git a/src/Common/escapeForFileName.cpp b/src/Common/escapeForFileName.cpp index 790d46a93ec..a1f9bff28d0 100644 --- a/src/Common/escapeForFileName.cpp +++ b/src/Common/escapeForFileName.cpp @@ -6,11 +6,6 @@ namespace DB { std::string escapeForFileName(const std::string & s) -{ - return escapeForFileName(std::string_view{s}); -} - -std::string escapeForFileName(std::string_view s) { std::string res; const char * pos = s.data(); diff --git a/src/Common/escapeForFileName.h b/src/Common/escapeForFileName.h index 279275f55d5..9ae29650804 100644 --- a/src/Common/escapeForFileName.h +++ b/src/Common/escapeForFileName.h @@ -11,7 +11,6 @@ namespace DB */ std::string escapeForFileName(const std::string & s); -std::string escapeForFileName(std::string_view s); std::string unescapeForFileName(const std::string & s); } From eb4add06bad171d09198df67bcbb97ef9d7549e2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 15 Nov 2023 10:35:39 +0100 Subject: [PATCH 355/813] Rewrite `repo.organization._url` to a proper repo._makeStringAttribute --- tests/ci/merge_pr.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 35b0614b01f..f35c07d84e7 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -230,8 +230,8 @@ def main(): # An ugly and not nice fix to patch the wrong organization URL, # see https://github.com/PyGithub/PyGithub/issues/2395#issuecomment-1378629710 # pylint: disable=protected-access - repo.organization._url.value = repo.organization.url.replace( # type: ignore - "/users/", "/orgs/", 1 + repo.organization._url = repo._makeStringAttribute( + repo.organization.url.replace("/users/", "/orgs/", 1) ) # pylint: enable=protected-access pr = repo.get_pull(args.pr) From ad3f422cff2b135fc70b77ac86b6838cb93e4414 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 15 Nov 2023 10:46:40 +0100 Subject: [PATCH 356/813] Get rid of `datetime` in calculation of since_updated --- tests/ci/cherry_pick.py | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 2ff502f52dc..9ee63a98f94 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -290,17 +290,18 @@ close it. self.cherrypick_pr.number, ) # The `updated_at` is Optional[datetime] - cherrypick_updated_at = self.cherrypick_pr.updated_at or datetime.now() - since_updated = datetime.now() - cherrypick_updated_at + cherrypick_updated_ts = ( + self.cherrypick_pr.updated_at or datetime.now() + ).timestamp() + since_updated = int(datetime.now().timestamp() - cherrypick_updated_ts) since_updated_str = ( - f"{since_updated.days}d{since_updated.seconds // 3600}" - f"h{since_updated.seconds // 60 % 60}m{since_updated.seconds % 60}s" + f"{since_updated // 86400}d{since_updated // 3600}" + f"h{since_updated // 60 % 60}m{since_updated % 60}s" ) - if since_updated < timedelta(days=1): + if since_updated < 86400: logging.info( - "The cherry-pick PR was updated at %s %s ago, " + "The cherry-pick PR was updated %s ago, " "waiting for the next running", - cherrypick_updated_at.isoformat(), since_updated_str, ) return From f859b90826f835816a7bb1b13abd936b03644a58 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 09:59:25 +0000 Subject: [PATCH 357/813] Undo duplicate test that also times out --- tests/queries/1_stateful/00178_gcd_codec.reference | 0 tests/queries/1_stateful/00178_gcd_codec.sql | 13 ------------- 2 files changed, 13 deletions(-) delete mode 100644 tests/queries/1_stateful/00178_gcd_codec.reference delete mode 100644 tests/queries/1_stateful/00178_gcd_codec.sql diff --git a/tests/queries/1_stateful/00178_gcd_codec.reference b/tests/queries/1_stateful/00178_gcd_codec.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/1_stateful/00178_gcd_codec.sql b/tests/queries/1_stateful/00178_gcd_codec.sql deleted file mode 100644 index 04715939d15..00000000000 --- a/tests/queries/1_stateful/00178_gcd_codec.sql +++ /dev/null @@ -1,13 +0,0 @@ -DROP TABLE IF EXISTS hits_gcd; - -CREATE TABLE hits_gcd (`WatchID` UInt64 CODEC (GCD,LZ4), `JavaEnable` UInt8 CODEC (GCD,LZ4), `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32 CODEC (GCD,LZ4), `ClientIP` UInt32 CODEC (GCD,LZ4), `ClientIP6` FixedString(16), `RegionID` UInt32 CODEC (GCD,LZ4), `UserID` UInt64 CODEC (GCD,LZ4), `CounterClass` Int8, `OS` UInt8 CODEC (GCD,LZ4), `UserAgent` UInt8 CODEC (GCD,LZ4), `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8 CODEC (GCD,LZ4), `IsRobot` UInt8 CODEC (GCD,LZ4), `RefererCategories` Array(UInt16) CODEC (GCD,LZ4), `URLCategories` Array(UInt16) CODEC (GCD,LZ4), `URLRegions` Array(UInt32) CODEC (GCD,LZ4), `RefererRegions` Array(UInt32) CODEC (GCD,LZ4), `ResolutionWidth` UInt16 CODEC (GCD,LZ4), `ResolutionHeight` UInt16 CODEC (GCD,LZ4), `ResolutionDepth` UInt8 CODEC (GCD,LZ4), `FlashMajor` UInt8 CODEC (GCD,LZ4), `FlashMinor` UInt8 CODEC (GCD,LZ4), `FlashMinor2` String, `NetMajor` UInt8 CODEC (GCD,LZ4), `NetMinor` UInt8 CODEC (GCD,LZ4), `UserAgentMajor` UInt16 CODEC (GCD,LZ4), `UserAgentMinor` FixedString(2), `CookieEnable` UInt8 CODEC (GCD,LZ4), `JavascriptEnable` UInt8 CODEC (GCD,LZ4), `IsMobile` UInt8 CODEC (GCD,LZ4), `MobilePhone` UInt8 CODEC (GCD,LZ4), `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32 CODEC (GCD,LZ4), `TraficSourceID` Int8, `SearchEngineID` UInt16 CODEC (GCD,LZ4), `SearchPhrase` String, `AdvEngineID` UInt8 CODEC (GCD,LZ4), `IsArtifical` UInt8 CODEC (GCD,LZ4), `WindowClientWidth` UInt16 CODEC (GCD,LZ4), `WindowClientHeight` UInt16 CODEC (GCD,LZ4), `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8 CODEC (GCD,LZ4), `SilverlightVersion2` UInt8 CODEC (GCD,LZ4), `SilverlightVersion3` UInt32 CODEC (GCD,LZ4), `SilverlightVersion4` UInt16 CODEC (GCD,LZ4), `PageCharset` String, `CodeVersion` UInt32 CODEC (GCD,LZ4), `IsLink` UInt8 CODEC (GCD,LZ4), `IsDownload` UInt8 CODEC (GCD,LZ4), `IsNotBounce` UInt8 CODEC (GCD,LZ4), `FUniqID` UInt64 CODEC (GCD,LZ4), `HID` UInt32 CODEC (GCD,LZ4), `IsOldCounter` UInt8 CODEC (GCD,LZ4), `IsEvent` UInt8 CODEC (GCD,LZ4), `IsParameter` UInt8 CODEC (GCD,LZ4), `DontCountHits` UInt8 CODEC (GCD,LZ4), `WithHash` UInt8 CODEC (GCD,LZ4), `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8 CODEC (GCD,LZ4), `Sex` UInt8 CODEC (GCD,LZ4), `Income` UInt8 CODEC (GCD,LZ4), `Interests` UInt16 CODEC (GCD,LZ4), `Robotness` UInt8 CODEC (GCD,LZ4), `GeneralInterests` Array(UInt16) CODEC (GCD,LZ4), `RemoteIP` UInt32 CODEC (GCD,LZ4), `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16 CODEC (GCD,LZ4), `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8 CODEC (GCD,LZ4), `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16 CODEC (GCD,LZ4), `GoalsReached` Array(UInt32) CODEC (GCD,LZ4), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8 CODEC (GCD,LZ4), `RefererHash` UInt64 CODEC (GCD,LZ4), `URLHash` UInt64 CODEC (GCD,LZ4), `CLID` UInt32 CODEC (GCD,LZ4), `YCLID` UInt64 CODEC (GCD,LZ4), `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32 CODEC (GCD,LZ4), `RequestTry` UInt8) - ENGINE = MergeTree() - PARTITION BY toYYYYMM(EventDate) - ORDER BY (CounterID, EventDate, intHash32(UserID)) - SAMPLE BY intHash32(UserID); - - -INSERT INTO hits_gcd SELECT * FROM test.hits; -SELECT * FROM hits_gcd FORMAT Null; - -DROP TABLE IF EXISTS hits_gcd; From 30c6cea8955f4727ee15abac49041847c1f5ebee Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 15 Nov 2023 11:09:13 +0100 Subject: [PATCH 358/813] Fix another place with offset-aware datetime --- tests/ci/merge_pr.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index f35c07d84e7..772821f4960 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -129,7 +129,7 @@ class Reviews: logging.info("The PR is changed at %s", last_changed.isoformat()) approved_at = max(review.submitted_at for review in approved.values()) - if approved_at == datetime.fromtimestamp(0): + if approved_at.timestamp() == 0: logging.info( "Unable to get `datetime.fromtimestamp(0)`, " "here's debug info about reviews: %s", @@ -138,7 +138,7 @@ class Reviews: else: logging.info("The PR is approved at %s", approved_at.isoformat()) - if approved_at < last_changed: + if approved_at.timestamp() < last_changed.timestamp(): logging.info( "There are changes done at %s after approval at %s", last_changed.isoformat(), From 308654195ca3ff5f078af8ca84115524acc48071 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 15 Nov 2023 10:58:51 +0100 Subject: [PATCH 359/813] Add config setting "wait_dictionaries_load_at_startup". --- programs/server/Server.cpp | 3 ++ programs/server/config.xml | 10 +++++ src/Interpreters/Context.cpp | 15 +++++++ src/Interpreters/Context.h | 1 + .../__init__.py | 0 .../configs/wait_for_dictionaries_load.xml | 3 ++ .../dictionaries/long_loading_dictionary.xml | 29 +++++++++++++ .../test_dictionaries_wait_for_load/test.py | 41 +++++++++++++++++++ 8 files changed, 102 insertions(+) create mode 100644 tests/integration/test_dictionaries_wait_for_load/__init__.py create mode 100644 tests/integration/test_dictionaries_wait_for_load/configs/wait_for_dictionaries_load.xml create mode 100644 tests/integration/test_dictionaries_wait_for_load/dictionaries/long_loading_dictionary.xml create mode 100644 tests/integration/test_dictionaries_wait_for_load/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 85ae6d7796c..9f59315ea45 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1816,6 +1816,9 @@ try try { global_context->loadOrReloadDictionaries(config()); + + if (config().getBool("wait_dictionaries_load_at_startup", false)) + global_context->waitForDictionariesLoad(); } catch (...) { diff --git a/programs/server/config.xml b/programs/server/config.xml index d7ad1545201..8ad04e6fcff 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1266,6 +1266,16 @@ --> *_dictionary.*ml + + 1 + + + 0 + *_function.*ml diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ac552a3969c..ea36dde26d9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2324,6 +2324,21 @@ void Context::loadOrReloadDictionaries(const Poco::Util::AbstractConfiguration & shared->dictionaries_xmls = external_dictionaries_loader.addConfigRepository(std::move(repository)); } +void Context::waitForDictionariesLoad() const +{ + LOG_TRACE(shared->log, "Waiting for dictionaries to be loaded"); + auto results = getExternalDictionariesLoader().tryLoadAll(); + for (const auto & result : results) + { + if ((result.status != ExternalLoaderStatus::LOADED) && (result.status != ExternalLoaderStatus::LOADED_AND_RELOADING)) + { + LOG_WARNING(shared->log, "Some dictionaries were not loaded ({}, {})", result.name, result.status); + return; + } + } + LOG_INFO(shared->log, "All dictionaries have been loaded"); +} + void Context::loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config) { auto patterns_values = getMultipleValuesFromConfig(config, "", "user_defined_executable_functions_config"); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e12a5c4b69b..b4a086c4b0e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -792,6 +792,7 @@ public: EmbeddedDictionaries & getEmbeddedDictionaries(); void tryCreateEmbeddedDictionaries(const Poco::Util::AbstractConfiguration & config) const; void loadOrReloadDictionaries(const Poco::Util::AbstractConfiguration & config); + void waitForDictionariesLoad() const; const ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoader() const; ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoader(); diff --git a/tests/integration/test_dictionaries_wait_for_load/__init__.py b/tests/integration/test_dictionaries_wait_for_load/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_dictionaries_wait_for_load/configs/wait_for_dictionaries_load.xml b/tests/integration/test_dictionaries_wait_for_load/configs/wait_for_dictionaries_load.xml new file mode 100644 index 00000000000..a446b730123 --- /dev/null +++ b/tests/integration/test_dictionaries_wait_for_load/configs/wait_for_dictionaries_load.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_dictionaries_wait_for_load/dictionaries/long_loading_dictionary.xml b/tests/integration/test_dictionaries_wait_for_load/dictionaries/long_loading_dictionary.xml new file mode 100644 index 00000000000..8b9d47833e4 --- /dev/null +++ b/tests/integration/test_dictionaries_wait_for_load/dictionaries/long_loading_dictionary.xml @@ -0,0 +1,29 @@ + + + long_loading_dictionary + + + + sleep 9 && echo "key,value" && echo "1,aa" && echo "2,bb" + CSVWithNames + + + + 600 + + + + + + + + key + + + value + String + + + + + diff --git a/tests/integration/test_dictionaries_wait_for_load/test.py b/tests/integration/test_dictionaries_wait_for_load/test.py new file mode 100644 index 00000000000..975e9ca3e56 --- /dev/null +++ b/tests/integration/test_dictionaries_wait_for_load/test.py @@ -0,0 +1,41 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + + +DICTIONARY_FILES = [ + "dictionaries/long_loading_dictionary.xml", +] + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=["configs/wait_for_dictionaries_load.xml"], + dictionaries=DICTIONARY_FILES, +) + +node0 = cluster.add_instance("node0", dictionaries=DICTIONARY_FILES) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def get_status(instance, dictionary_name): + return instance.query( + "SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'" + ).rstrip("\n") + + +def test_wait_for_dictionaries_load(): + assert get_status(node0, "long_loading_dictionary") == "NOT_LOADED" + + assert get_status(node1, "long_loading_dictionary") == "LOADED" + assert node1.query("SELECT * FROM dictionary(long_loading_dictionary)") == TSV( + [[1, "aa"], [2, "bb"]] + ) From 671a401ded9fa1ef18c5b38e5605dc51c65229dc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Nov 2023 12:26:14 +0100 Subject: [PATCH 360/813] Ask linker to remove garbage from external libraries --- CMakeLists.txt | 2 +- contrib/CMakeLists.txt | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 9e548c5a6d0..36fd3a00eba 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -164,7 +164,7 @@ if (OS_LINUX) # and whatever is poisoning it by LD_PRELOAD should not link to our symbols. # - The clickhouse-odbc-bridge and clickhouse-library-bridge binaries # should not expose their symbols to ODBC drivers and libraries. - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic -Wl,--gc-sections") endif () if (OS_DARWIN) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 390b0241e7d..0273a93b044 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -1,7 +1,7 @@ #"${folder}/CMakeLists.txt" Third-party libraries may have substandard code. -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w -ffunction-sections -fdata-sections") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w -ffunction-sections -fdata-sections") if (WITH_COVERAGE) set (WITHOUT_COVERAGE_LIST ${WITHOUT_COVERAGE}) From e6281147290ae249ba9573d2385a606f1e83a99a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 15 Nov 2023 12:17:29 +0100 Subject: [PATCH 361/813] Add test for avoided recursion --- tests/ci/test_digest.py | 16 ++++++++++------ tests/ci/tests/digests/dir1/subdir1_1/14 | 1 + 2 files changed, 11 insertions(+), 6 deletions(-) create mode 100644 tests/ci/tests/digests/dir1/subdir1_1/14 diff --git a/tests/ci/test_digest.py b/tests/ci/test_digest.py index 246a3226721..278b2712917 100644 --- a/tests/ci/test_digest.py +++ b/tests/ci/test_digest.py @@ -8,6 +8,7 @@ import digest_helper as dh _12 = b"12\n" _13 = b"13\n" +_14 = b"14\n" # pylint:disable=protected-access @@ -50,12 +51,13 @@ class TestDigests(unittest.TestCase): # dir1 hash_expected = md5() - hash_expected.update(_12) + hash_expected.update(_12 + _14) dh._digest_directory(self.tests_dir / "dir1", hash_tested) self.assertEqual(hash_expected.digest(), hash_tested.digest()) # dir2 contains 12 and 13 - hash_expected.update(_13) + hash_expected = md5() + hash_expected.update(_12 + _13) hash_tested = md5() dh._digest_directory(self.tests_dir / "dir2", hash_tested) self.assertEqual(hash_expected.digest(), hash_tested.digest()) @@ -78,7 +80,7 @@ class TestDigests(unittest.TestCase): ) # Test directory works fine hash_expected = md5() - hash_expected.update(_12) + hash_expected.update(_12 + _14) self.assertEqual( hash_expected.digest(), dh.digest_path(self.tests_dir / "dir1").digest() ) @@ -86,20 +88,22 @@ class TestDigests(unittest.TestCase): hash_tested = hash_expected.copy() dh.digest_path(self.tests_dir / "dir3", hash_tested) hash_expected = md5() - hash_expected.update(_12 + _12 + _13) + hash_expected.update(_12 + _14 + _12 + _13) self.assertEqual(hash_expected.digest(), hash_tested.digest()) # Test the full content of the following structure # tests/digests # ├── 12 # ├── dir1 - # │   └── 12 + # │   ├── 12 + # │   └── subdir1_1 + # │   └── 14 # ├── dir2 # │   ├── 12 # │   └── 13 # ├── dir3 -> dir2 # └── symlink-12 -> 12 hash_expected = md5() - hash_expected.update(_12 * 3 + (_13 + _12) * 2) + hash_expected.update(_12 * 2 + _14 + (_12 + _13) * 2 + _12) self.assertEqual( hash_expected.digest(), dh.digest_path(self.tests_dir).digest() ) diff --git a/tests/ci/tests/digests/dir1/subdir1_1/14 b/tests/ci/tests/digests/dir1/subdir1_1/14 new file mode 100644 index 00000000000..8351c19397f --- /dev/null +++ b/tests/ci/tests/digests/dir1/subdir1_1/14 @@ -0,0 +1 @@ +14 From 5ee3bed87f99cd1070baf55ca94511a2bf91e0d5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 15 Nov 2023 12:38:03 +0100 Subject: [PATCH 362/813] Improve logs after review. --- src/Interpreters/Context.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ea36dde26d9..368545e0101 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2328,15 +2328,19 @@ void Context::waitForDictionariesLoad() const { LOG_TRACE(shared->log, "Waiting for dictionaries to be loaded"); auto results = getExternalDictionariesLoader().tryLoadAll(); + bool all_dictionaries_loaded = true; for (const auto & result : results) { if ((result.status != ExternalLoaderStatus::LOADED) && (result.status != ExternalLoaderStatus::LOADED_AND_RELOADING)) { - LOG_WARNING(shared->log, "Some dictionaries were not loaded ({}, {})", result.name, result.status); - return; + LOG_WARNING(shared->log, "Dictionary {} was not loaded ({})", result.name, result.status); + all_dictionaries_loaded = false; } } - LOG_INFO(shared->log, "All dictionaries have been loaded"); + if (all_dictionaries_loaded) + LOG_INFO(shared->log, "All dictionaries have been loaded"); + else + LOG_INFO(shared->log, "Some dictionaries were not loaded"); } void Context::loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config) From 9376ac56037107f25ae92df6b79bf2cc29690630 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 15 Nov 2023 11:42:52 +0000 Subject: [PATCH 363/813] added implementation for negative nulls --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index 8b40cd12282..b31dfe2eda5 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -37,7 +37,11 @@ float convertFloat16ToFloat32(uint16_t float16_value) uint16_t fraction = float16_value & 0x3FF; if (exponent == 0 && fraction == 0) - return float(sign << 31); + { + if (sign) + return float(-0.0); + return float(0.0); + } // Handling special cases for exponent if (exponent == 0x1F) From a8284109930439bdd5ce9ecd62d427c4b42d740c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 15 Nov 2023 12:14:48 +0000 Subject: [PATCH 364/813] Fix randomization of Keeper configs in stress tests --- docker/test/stateless/stress_tests.lib | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index fc35bff5e40..edcf2bc7bee 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -63,16 +63,16 @@ function configure() if [[ -n "$RANDOMIZE_KEEPER_FEATURE_FLAGS" ]] && [[ "$RANDOMIZE_KEEPER_FEATURE_FLAGS" -eq 1 ]]; then # Randomize all Keeper feature flags - randomize_config_boolean_value filtered_list keeper_port - randomize_config_boolean_value multi_read keeper_port - randomize_config_boolean_value check_not_exists keeper_port - randomize_config_boolean_value create_if_not_exists keeper_port + randomize_keeper_config_boolean_value filtered_list keeper_port + randomize_keeper_config_boolean_value multi_read keeper_port + randomize_keeper_config_boolean_value check_not_exists keeper_port + randomize_keeper_config_boolean_value create_if_not_exists keeper_port fi sudo chown clickhouse /etc/clickhouse-server/config.d/keeper_port.xml sudo chgrp clickhouse /etc/clickhouse-server/config.d/keeper_port.xml - randomize_config_boolean_value use_compression zookeeper + randomize_keeper_config_boolean_value use_compression zookeeper # for clickhouse-server (via service) echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment From 4e48df6bfc5c872e956b77bf0d14d7a387372fb0 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 15 Nov 2023 12:15:21 +0000 Subject: [PATCH 365/813] improved nulls --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index b31dfe2eda5..1edf2438ab6 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -38,9 +38,8 @@ float convertFloat16ToFloat32(uint16_t float16_value) if (exponent == 0 && fraction == 0) { - if (sign) - return float(-0.0); - return float(0.0); + uint32_t float32_value = sign << 31; + return std::bit_cast(float32_value); } // Handling special cases for exponent From 178d23b9511d953e84a2b0b7923699cfcb841cc6 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 15 Nov 2023 13:53:52 +0100 Subject: [PATCH 366/813] Concat with arbitrary types + tests --- src/Functions/concat.cpp | 64 ++++++++++--------- src/Functions/concatWithSeparator.cpp | 19 +++--- src/Functions/formatString.h | 13 ++-- .../0_stateless/00727_concat.reference | 51 ++++++++++++++- tests/queries/0_stateless/00727_concat.sql | 59 ++++++++++++++++- ...75_show_columns_called_from_clickhouse.sql | 2 +- 6 files changed, 157 insertions(+), 51 deletions(-) diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 350cbee58a3..37311e6c09b 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -1,16 +1,14 @@ #include #include -#include #include #include +#include #include #include -#include #include #include #include #include -#include #include "formatString.h" @@ -18,9 +16,9 @@ namespace DB { namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ILLEGAL_COLUMN; } using namespace GatherUtils; @@ -33,7 +31,7 @@ class ConcatImpl : public IFunction { public: static constexpr auto name = Name::name; - explicit ConcatImpl(ContextPtr context_) : context(context_) {} + explicit ConcatImpl(ContextPtr context_) : context(context_) { } static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } String getName() const override { return name; } @@ -68,8 +66,7 @@ public: /// For 3 and more arguments FormatStringImpl is much faster (up to 50-60%). if (arguments.size() == 2) return executeBinary(arguments, input_rows_count); - else - return executeFormatImpl(arguments, input_rows_count); + return executeFormatImpl(arguments, input_rows_count); } private: @@ -113,6 +110,7 @@ private: std::vector offsets(num_arguments); std::vector fixed_string_sizes(num_arguments); std::vector> constant_strings(num_arguments); + std::vector converted_col_ptrs(num_arguments); bool has_column_string = false; bool has_column_fixed_string = false; for (size_t i = 0; i < num_arguments; ++i) @@ -136,16 +134,27 @@ private: } else { - // An arbitrary type argument: converting it to a StringColumn as if `toString` was called - ColumnsWithTypeAndName args; - args.emplace_back(column, arguments[i].type, "tmp"); - const ColumnPtr converted_col_ptr = ConvertImplGenericToString::execute( - args, std::make_shared(), column->size()); - const ColumnString * converted_col_str = assert_cast(converted_col_ptr.get()); + // An arbitrary type argument: converting it to a StringColumn first + const auto serialization = arguments[i].type->getDefaultSerialization(); + ColumnString::MutablePtr converted_col_str = ColumnString::create(); + static FormatSettings format_settings; + + ColumnStringHelpers::WriteHelper write_helper(*converted_col_str, column->size()); + auto & write_buffer = write_helper.getWriteBuffer(); + for (size_t j = 0; j < column->size(); ++j) + { + serialization->serializeText(*column, j, write_buffer, format_settings); + write_helper.rowWritten(); + } + write_helper.finalize(); + // Same as the normal `ColumnString` branch has_column_string = true; data[i] = &converted_col_str->getChars(); offsets[i] = &converted_col_str->getOffsets(); + + // keep the refcounted-pointer around (to be able to use data/offsets later) + converted_col_ptrs[i] = std::move(converted_col_str); } } @@ -193,7 +202,7 @@ public: static constexpr auto name = "concat"; static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } - explicit ConcatOverloadResolver(ContextPtr context_) : context(context_) {} + explicit ConcatOverloadResolver(ContextPtr context_) : context(context_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -202,28 +211,25 @@ public: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { if (isArray(arguments.at(0).type)) - { return FunctionFactory::instance().getImpl("arrayConcat", context)->build(arguments); - } - else if (isMap(arguments.at(0).type)) - { + if (isMap(arguments.at(0).type)) return FunctionFactory::instance().getImpl("mapConcat", context)->build(arguments); - } - else if (isTuple(arguments.at(0).type)) - { + if (isTuple(arguments.at(0).type)) return FunctionFactory::instance().getImpl("tupleConcat", context)->build(arguments); - } - else - return std::make_unique( - FunctionConcat::create(context), collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); + return std::make_unique( + FunctionConcat::create(context), + collections::map(arguments, [](const auto & elem) { return elem.type; }), + return_type); } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be at least 2.", - getName(), arguments.size()); + getName(), + arguments.size()); /// We always return Strings from concat, even if arguments were fixed strings. return std::make_shared(); diff --git a/src/Functions/concatWithSeparator.cpp b/src/Functions/concatWithSeparator.cpp index f0a7afbbaa7..f295d86943f 100644 --- a/src/Functions/concatWithSeparator.cpp +++ b/src/Functions/concatWithSeparator.cpp @@ -1,5 +1,5 @@ -#include #include +#include #include #include #include @@ -14,9 +14,9 @@ namespace DB { namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ILLEGAL_COLUMN; } namespace @@ -26,7 +26,7 @@ class ConcatWithSeparatorImpl : public IFunction { public: static constexpr auto name = Name::name; - explicit ConcatWithSeparatorImpl(ContextPtr context_) : context(context_) {} + explicit ConcatWithSeparatorImpl(ContextPtr context_) : context(context_) { } static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } @@ -113,8 +113,7 @@ public: else if (const ColumnConst * const_col = checkAndGetColumnConstStringOrFixedString(column.get())) constant_strings[2 * i] = const_col->getValue(); else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of argument of function {}", column->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", column->getName(), getName()); } String pattern; @@ -156,14 +155,14 @@ using FunctionConcatWithSeparatorAssumeInjective = ConcatWithSeparatorImpl(FunctionDocumentation{ - .description=R"( + .description = R"( Returns the concatenation strings separated by string separator. Syntax: concatWithSeparator(sep, expr1, expr2, expr3...) )", .examples{{"concatWithSeparator", "SELECT concatWithSeparator('a', '1', '2', '3')", ""}}, .categories{"String"}}); factory.registerFunction(FunctionDocumentation{ - .description=R"( + .description = R"( Same as concatWithSeparator, the difference is that you need to ensure that concatWithSeparator(sep, expr1, expr2, expr3...) → result is injective, it will be used for optimization of GROUP BY. The function is named “injective” if it always returns different result for different values of arguments. In other words: different arguments never yield identical result. @@ -171,7 +170,7 @@ The function is named “injective” if it always returns different result for .examples{{"concatWithSeparatorAssumeInjective", "SELECT concatWithSeparatorAssumeInjective('a', '1', '2', '3')", ""}}, .categories{"String"}}); - /// Compatibility with Spark: + /// Compatibility with Spark and MySQL: factory.registerAlias("concat_ws", "concatWithSeparator", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/formatString.h b/src/Functions/formatString.h index 30149e9a5b0..4bdb672caf4 100644 --- a/src/Functions/formatString.h +++ b/src/Functions/formatString.h @@ -1,18 +1,13 @@ #pragma once -#include -#include -#include -#include -#include -#include - - #include #include #include -#include #include +#include +#include +#include +#include namespace DB diff --git a/tests/queries/0_stateless/00727_concat.reference b/tests/queries/0_stateless/00727_concat.reference index af5626b4a11..4785f67bdd9 100644 --- a/tests/queries/0_stateless/00727_concat.reference +++ b/tests/queries/0_stateless/00727_concat.reference @@ -1 +1,50 @@ -Hello, world! +-- Const string + non-const arbitrary type +With 42 +With 43 +With 44 +With 45 +With 46 +With 47 +With 48 +With 49 +With 50 +With 51 +With 52 +With 53 +With 42.42 +With 43.43 +With 44 +With true +With false +With foo +With bar +With foo +With bar +With foo +With bar +With foo +With bar +With fae310ca-d52a-4923-9e9b-02bf67f4b009 +With 2023-11-14 +With 2123-11-14 +With 2023-11-14 05:50:12 +With 2023-11-14 05:50:12.123 +With hallo +With [\'foo\',\'bar\'] +With {"foo":"bar"} +With (42,\'foo\') +With {42:\'foo\'} +With 122.233.64.201 +With 2001:1:130f:2:3:9c0:876a:130b +With (42,43) +With [(0,0),(10,0),(10,10),(0,10)] +With [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] +With [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] +\N +-- Miscellaneous tests +Non-const strings +Three arguments test +3 arguments test with int type +Testing the alias +\N +\N diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index 800ebd5ec53..3119bd76c0c 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -1 +1,58 @@ -SELECT CONCAT('Hello', ', ', 'world!'); +-- Tags: no-fasttest +-- no-fasttest: json type needs rapidjson library, geo types need s2 geometry + +-- not tested here: (Simple)AggregateFunction, Nested + +SET allow_experimental_object_type = 1; + +SELECT '-- Const string + non-const arbitrary type'; +SELECT concat('With ', materialize(42 :: Int8)); +SELECT concat('With ', materialize(43 :: Int16)); +SELECT concat('With ', materialize(44 :: Int32)); +SELECT concat('With ', materialize(45 :: Int64)); +SELECT concat('With ', materialize(46 :: Int128)); +SELECT concat('With ', materialize(47 :: Int256)); +SELECT concat('With ', materialize(48 :: UInt8)); +SELECT concat('With ', materialize(49 :: UInt16)); +SELECT concat('With ', materialize(50 :: UInt32)); +SELECT concat('With ', materialize(51 :: UInt64)); +SELECT concat('With ', materialize(52 :: UInt128)); +SELECT concat('With ', materialize(53 :: UInt256)); +SELECT concat('With ', materialize(42.42 :: Float32)); +SELECT concat('With ', materialize(43.43 :: Float64)); +SELECT concat('With ', materialize(44.44 :: Decimal(2))); +SELECT concat('With ', materialize(true :: Bool)); +SELECT concat('With ', materialize(false :: Bool)); +SELECT concat('With ', materialize('foo' :: String)); +SELECT concat('With ', materialize('bar' :: FixedString(3))); +SELECT concat('With ', materialize('foo' :: Nullable(String))); +SELECT concat('With ', materialize('bar' :: Nullable(FixedString(3)))); +SELECT concat('With ', materialize('foo' :: LowCardinality(String))); +SELECT concat('With ', materialize('bar' :: LowCardinality(FixedString(3)))); +SELECT concat('With ', materialize('foo' :: LowCardinality(Nullable(String)))); +SELECT concat('With ', materialize('bar' :: LowCardinality(Nullable(FixedString(3))))); +SELECT concat('With ', materialize('fae310ca-d52a-4923-9e9b-02bf67f4b009' :: UUID)); +SELECT concat('With ', materialize('2023-11-14' :: Date)); +SELECT concat('With ', materialize('2123-11-14' :: Date32)); +SELECT concat('With ', materialize('2023-11-14 05:50:12' :: DateTime)); +SELECT concat('With ', materialize('2023-11-14 05:50:12.123' :: DateTime64(3))); +SELECT concat('With ', materialize('hallo' :: Enum('hallo' = 1))); +SELECT concat('With ', materialize(['foo', 'bar'] :: Array(String))); +SELECT concat('With ', materialize('{"foo": "bar"}' :: JSON)); +SELECT concat('With ', materialize((42, 'foo') :: Tuple(Int32, String))); +SELECT concat('With ', materialize(map(42, 'foo') :: Map(Int32, String))); +SELECT concat('With ', materialize('122.233.64.201' :: IPv4)); +SELECT concat('With ', materialize('2001:0001:130F:0002:0003:09C0:876A:130B' :: IPv6)); +SELECT concat('With ', materialize((42, 43) :: Point)); +SELECT concat('With ', materialize([(0,0),(10,0),(10,10),(0,10)] :: Ring)); +SELECT concat('With ', materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]] :: Polygon)); +SELECT concat('With ', materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]] :: MultiPolygon)); +SELECT concat('With ', materialize(NULL :: Nullable(UInt64))); + +SELECT '-- Miscellaneous tests'; +SELECT concat(materialize('Non-const'), materialize(' strings')); +SELECT concat('Three ', 'arguments', ' test'); +SELECT concat(materialize(3 :: Int64), ' arguments test', ' with int type'); +SELECT CONCAT('Testing the ', 'alias'); +SELECT concat(materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); +SELECT concat(42, materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); diff --git a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql index 89073bd2943..752367517af 100644 --- a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql +++ b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql @@ -5,7 +5,7 @@ -- Tests the output of SHOW COLUMNS when called through the ClickHouse protocol. -- ----------------------------------------------------------------------------------- --- Please keep this test in-sync with 02775_show_columns_called_through_mysql.sql +-- Please keep this test in-sync with 02775_show_columns_called_from_mysql.expect -- ----------------------------------------------------------------------------------- DROP TABLE IF EXISTS tab; From d62048031facb33d8199325437cd2be344f88946 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 15 Nov 2023 13:57:28 +0100 Subject: [PATCH 367/813] Add documentation. --- .../settings.md | 26 +++++++++++++++++-- .../settings.md | 24 +++++++++++++++-- programs/server/config.xml | 11 ++++---- 3 files changed, 52 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 98636a653fb..cfc5a939a0e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -961,9 +961,13 @@ See also “[Executable User Defined Functions](../../sql-reference/functions/in Lazy loading of dictionaries. -If `true`, then each dictionary is created on first use. If dictionary creation failed, the function that was using the dictionary throws an exception. +If `true`, then each dictionary is loaded on the first use. If the loading is failed, the function that was using the dictionary throws an exception. -If `false`, all dictionaries are created when the server starts, if the dictionary or dictionaries are created too long or are created with errors, then the server boots without of these dictionaries and continues to try to create these dictionaries. +If `false`, then the server starts loading all dictionaries at startup. +Dictionaries are loaded in background. +The server doesn't wait at startup until all the dictionaries finish their loading +(exception: if `wait_dictionaries_load_at_startup` is set to `true` - see below). +When a dictionary is used in a query for the first time then the query waits until the dictionary is loaded if it's not loaded yet. The default is `true`. @@ -2391,6 +2395,24 @@ Path to the file that contains: users.xml ``` +## wait_dictionaries_load_at_startup {#wait_dictionaries_load_at_startup} + +If `false`, then the server will not wait at startup until all the dictionaries finish their loading. +This allows to start ClickHouse faster. + +If `true`, then the server will wait at startup until all the dictionaries finish their loading (successfully or not) +before listening to any connections. +This can make ClickHouse start slowly, however after that some queries can be executed faster +(because they won't have to wait for the used dictionaries to be load). + +The default is `false`. + +**Example** + +``` xml +false +``` + ## zookeeper {#server-settings_zookeeper} Contains settings that allow ClickHouse to interact with a [ZooKeeper](http://zookeeper.apache.org/) cluster. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 2c7f0b773e8..99ea7894ef8 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -277,8 +277,10 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Если `true`, то каждый словарь создаётся при первом использовании. Если словарь не удалось создать, то вызов функции, использующей словарь, сгенерирует исключение. -Если `false`, то все словари создаются при старте сервера, если словарь или словари создаются слишком долго или создаются с ошибкой, то сервер загружается без -этих словарей и продолжает попытки создать эти словари. +Если `false`, сервер начнет загрузку всех словарей на старте сервера. +Словари загружаются в фоне. Сервер не ждет на старте, пока словари закончат загружаться +(исключение: если `wait_dictionaries_load_at_startup` установлена в `true` - см. ниже). +Когда словарь используется в запросе первый раз, этот запрос будет ждать окончания загрузки словаря, если он еще не загрузился. По умолчанию - `true`. @@ -1718,6 +1720,24 @@ TCP порт для защищённого обмена данными с кли users.xml ``` +## wait_dictionaries_load_at_startup {#wait_dictionaries_load_at_startup} + +Если `false`, то сервер не будет ждать на старте, пока словари закончат загружаться. +Это позволяет ClickHouse стартовать быстрее. + +Если `true`, то ClickHouse будет ждать на старте до окончания загрузки всех словарей (успешно или нет) +перед тем, как начать принимать соединения. +Это может привести к медленному старту ClickHouse, однако после этого некоторые запросы могут выполняться быстрее +(потому что им не придется ждать окончания загрузки используемых словарей). + +По умолчанию - `false`. + +**Пример** + +``` xml +false +``` + ## zookeeper {#server-settings_zookeeper} Содержит параметры, позволяющие ClickHouse взаимодействовать с кластером [ZooKeeper](http://zookeeper.apache.org/). diff --git a/programs/server/config.xml b/programs/server/config.xml index 8ad04e6fcff..2af0384645c 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1267,14 +1267,15 @@ *_dictionary.*ml - 1 + true - - 0 + false *_function.*ml From 49e6bad2bd707fa0839c0ee5144e01a48223df44 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 15 Nov 2023 16:14:33 +0300 Subject: [PATCH 368/813] Fixed tests --- .../02476_fuse_sum_count.reference | 23 +++++++++--- .../02477_fuse_quantiles.reference | 36 ++++++++++++++----- 2 files changed, 45 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02476_fuse_sum_count.reference b/tests/queries/0_stateless/02476_fuse_sum_count.reference index c62583e8c88..43a39e8b7e5 100644 --- a/tests/queries/0_stateless/02476_fuse_sum_count.reference +++ b/tests/queries/0_stateless/02476_fuse_sum_count.reference @@ -256,14 +256,27 @@ QUERY id: 0 QUERY id: 14, is_subquery: 1 PROJECTION COLUMNS x Int64 + count(b) UInt64 PROJECTION - LIST id: 15, nodes: 1 - FUNCTION id: 16, function_name: sum, function_type: aggregate, result_type: Int64 + LIST id: 15, nodes: 2 + FUNCTION id: 16, function_name: tupleElement, function_type: ordinary, result_type: Int64 ARGUMENTS - LIST id: 17, nodes: 1 - COLUMN id: 18, column_name: b, result_type: Int8, source_id: 19 + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: sumCount, function_type: aggregate, result_type: Tuple(Int64, UInt64) + ARGUMENTS + LIST id: 19, nodes: 1 + COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21 + CONSTANT id: 22, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 23, function_name: tupleElement, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 24, nodes: 2 + FUNCTION id: 18, function_name: sumCount, function_type: aggregate, result_type: Tuple(Int64, UInt64) + ARGUMENTS + LIST id: 19, nodes: 1 + COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21 + CONSTANT id: 25, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE id: 19, table_name: default.fuse_tbl + TABLE id: 21, table_name: default.fuse_tbl 0 0 nan 0 0 nan 45 10 4.5 Decimal(38, 0) UInt64 Float64 diff --git a/tests/queries/0_stateless/02477_fuse_quantiles.reference b/tests/queries/0_stateless/02477_fuse_quantiles.reference index 8384df9f04a..7c7d581f7fb 100644 --- a/tests/queries/0_stateless/02477_fuse_quantiles.reference +++ b/tests/queries/0_stateless/02477_fuse_quantiles.reference @@ -48,17 +48,35 @@ QUERY id: 0 QUERY id: 19, is_subquery: 1 PROJECTION COLUMNS x Float64 + quantile(0.9)(b) Float64 PROJECTION - LIST id: 21, nodes: 1 - FUNCTION id: 22, function_name: quantile, function_type: aggregate, result_type: Float64 - PARAMETERS - LIST id: 23, nodes: 1 - CONSTANT id: 24, constant_value: Float64_0.5, constant_value_type: Float64 + LIST id: 21, nodes: 2 + FUNCTION id: 22, function_name: arrayElement, function_type: ordinary, result_type: Float64 ARGUMENTS - LIST id: 25, nodes: 1 - COLUMN id: 26, column_name: b, result_type: Int32, source_id: 27 + LIST id: 23, nodes: 2 + FUNCTION id: 24, function_name: quantiles, function_type: aggregate, result_type: Array(Float64) + PARAMETERS + LIST id: 25, nodes: 2 + CONSTANT id: 26, constant_value: Float64_0.5, constant_value_type: Float64 + CONSTANT id: 27, constant_value: Float64_0.9, constant_value_type: Float64 + ARGUMENTS + LIST id: 28, nodes: 1 + COLUMN id: 29, column_name: b, result_type: Int32, source_id: 30 + CONSTANT id: 31, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 32, function_name: arrayElement, function_type: ordinary, result_type: Float64 + ARGUMENTS + LIST id: 33, nodes: 2 + FUNCTION id: 24, function_name: quantiles, function_type: aggregate, result_type: Array(Float64) + PARAMETERS + LIST id: 25, nodes: 2 + CONSTANT id: 26, constant_value: Float64_0.5, constant_value_type: Float64 + CONSTANT id: 27, constant_value: Float64_0.9, constant_value_type: Float64 + ARGUMENTS + LIST id: 28, nodes: 1 + COLUMN id: 29, column_name: b, result_type: Int32, source_id: 30 + CONSTANT id: 34, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE id: 27, table_name: default.fuse_tbl + TABLE id: 30, table_name: default.fuse_tbl GROUP BY - LIST id: 28, nodes: 1 + LIST id: 35, nodes: 1 COLUMN id: 18, column_name: x, result_type: Float64, source_id: 19 From 0aaea6e51ddbd13ad5becafd010e07dd652ab3c7 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 15 Nov 2023 14:42:38 +0100 Subject: [PATCH 369/813] Fix ColumnConst serialization issues, more tests --- src/Functions/concat.cpp | 9 +++---- .../0_stateless/00727_concat.reference | 17 ++++++++++--- tests/queries/0_stateless/00727_concat.sql | 24 +++++++++++++++---- 3 files changed, 39 insertions(+), 11 deletions(-) diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 37311e6c09b..346f96e4f03 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -135,6 +135,7 @@ private: else { // An arbitrary type argument: converting it to a StringColumn first + const auto full_column = column->convertToFullIfNeeded(); const auto serialization = arguments[i].type->getDefaultSerialization(); ColumnString::MutablePtr converted_col_str = ColumnString::create(); static FormatSettings format_settings; @@ -143,7 +144,7 @@ private: auto & write_buffer = write_helper.getWriteBuffer(); for (size_t j = 0; j < column->size(); ++j) { - serialization->serializeText(*column, j, write_buffer, format_settings); + serialization->serializeText(*full_column, j, write_buffer, format_settings); write_helper.rowWritten(); } write_helper.finalize(); @@ -210,11 +211,11 @@ public: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { - if (isArray(arguments.at(0).type)) + if (std::ranges::all_of(arguments, [](const auto & elem) { return isArray(elem.type); })) return FunctionFactory::instance().getImpl("arrayConcat", context)->build(arguments); - if (isMap(arguments.at(0).type)) + if (std::ranges::all_of(arguments, [](const auto & elem) { return isMap(elem.type); })) return FunctionFactory::instance().getImpl("mapConcat", context)->build(arguments); - if (isTuple(arguments.at(0).type)) + if (std::ranges::all_of(arguments, [](const auto & elem) { return isTuple(elem.type); })) return FunctionFactory::instance().getImpl("tupleConcat", context)->build(arguments); return std::make_unique( FunctionConcat::create(context), diff --git a/tests/queries/0_stateless/00727_concat.reference b/tests/queries/0_stateless/00727_concat.reference index 4785f67bdd9..9b6a8b3857b 100644 --- a/tests/queries/0_stateless/00727_concat.reference +++ b/tests/queries/0_stateless/00727_concat.reference @@ -24,6 +24,7 @@ With foo With bar With foo With bar +With 42 With fae310ca-d52a-4923-9e9b-02bf67f4b009 With 2023-11-14 With 2123-11-14 @@ -40,11 +41,21 @@ With (42,43) With [(0,0),(10,0),(10,10),(0,10)] With [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] With [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] +-- NULL arguments \N --- Miscellaneous tests +\N +\N +\N +\N +\N +\N +-- Various arguments tests Non-const strings +Two arguments test Three arguments test 3 arguments test with int type +42144 +42144255 +42144 +42144255 Testing the alias -\N -\N diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index 3119bd76c0c..ba76ff53884 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -4,6 +4,7 @@ -- not tested here: (Simple)AggregateFunction, Nested SET allow_experimental_object_type = 1; +SET allow_suspicious_low_cardinality_types=1; SELECT '-- Const string + non-const arbitrary type'; SELECT concat('With ', materialize(42 :: Int8)); @@ -31,6 +32,7 @@ SELECT concat('With ', materialize('foo' :: LowCardinality(String))); SELECT concat('With ', materialize('bar' :: LowCardinality(FixedString(3)))); SELECT concat('With ', materialize('foo' :: LowCardinality(Nullable(String)))); SELECT concat('With ', materialize('bar' :: LowCardinality(Nullable(FixedString(3))))); +SELECT concat('With ', materialize(42 :: LowCardinality(Nullable(UInt32)))); SELECT concat('With ', materialize('fae310ca-d52a-4923-9e9b-02bf67f4b009' :: UUID)); SELECT concat('With ', materialize('2023-11-14' :: Date)); SELECT concat('With ', materialize('2123-11-14' :: Date32)); @@ -47,12 +49,26 @@ SELECT concat('With ', materialize((42, 43) :: Point)); SELECT concat('With ', materialize([(0,0),(10,0),(10,10),(0,10)] :: Ring)); SELECT concat('With ', materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]] :: Polygon)); SELECT concat('With ', materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]] :: MultiPolygon)); -SELECT concat('With ', materialize(NULL :: Nullable(UInt64))); -SELECT '-- Miscellaneous tests'; +SELECT '-- NULL arguments'; +SELECT concat(NULL, NULL); +SELECT concat(NULL, materialize(NULL :: Nullable(UInt64))); +SELECT concat(materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); + +SELECT concat(42, materialize(NULL :: Nullable(UInt64))); +SELECT concat('42', materialize(NULL :: Nullable(UInt64))); + +SELECT concat(42, materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); +SELECT concat('42', materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); + +SELECT '-- Various arguments tests'; SELECT concat(materialize('Non-const'), materialize(' strings')); +SELECT concat('Two arguments ', 'test'); SELECT concat('Three ', 'arguments', ' test'); SELECT concat(materialize(3 :: Int64), ' arguments test', ' with int type'); +SELECT concat(materialize(42 :: Int32), materialize(144 :: UInt64)); +SELECT concat(materialize(42 :: Int32), materialize(144 :: UInt64), materialize(255 :: UInt32)); +SELECT concat(42, 144); +SELECT concat(42, 144, 255); + SELECT CONCAT('Testing the ', 'alias'); -SELECT concat(materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); -SELECT concat(42, materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); From 62378b421928268c6285cc1cd65b446a799291fb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 15 Nov 2023 13:56:51 +0000 Subject: [PATCH 370/813] Fixing style. --- src/Storages/StorageInput.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index 9ac082a9add..82ddda4cdd1 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -10,7 +10,7 @@ namespace DB class StorageInput final : public IStorage { - friend class ReadFromInput; + friend class ReadFromInput; public: StorageInput(const StorageID & table_id, const ColumnsDescription & columns_); From c7cd4fa972893ad6dead0a6f3fe9038c2c61ebad Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 15 Nov 2023 14:57:34 +0100 Subject: [PATCH 371/813] Update concat docs --- .../sql-reference/functions/string-functions.md | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 4df987b5e2a..dc324e8e331 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -439,7 +439,7 @@ concat(s1, s2, ...) **Arguments** -Values of type String or FixedString. +Values of arbitrary types. If an argument is not a String or FixedString, it is converted to the String type using the default serialization. **Returned values** @@ -461,6 +461,20 @@ Result: └─────────────────────────────┘ ``` +**Example** + +```sql +SELECT concat(42, 144); +``` + +Result: + +```result +┌─concat(42, 144)─┐ +│ 42144 │ +└─────────────────┘ +``` + ## concatAssumeInjective Like [concat](#concat) but assumes that `concat(s1, s2, ...) → sn` is injective. Can be used for optimization of GROUP BY. From 1ea74cee3a5b664687fc8c059da0fed144fd9ea9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Nov 2023 15:04:07 +0100 Subject: [PATCH 372/813] Early disconnect if there is authentication failure with interserver secret --- src/Server/TCPHandler.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f929d0f5ff9..e7c40092077 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -588,6 +587,10 @@ void TCPHandler::runImpl() } catch (const Exception & e) { + /// Authentication failure with interserver secret. + if (e.code() == ErrorCodes::AUTHENTICATION_FAILED) + throw; + state.io.onException(); exception.reset(e.clone()); @@ -1717,7 +1720,18 @@ void TCPHandler::receiveQuery() { client_info.interface = ClientInfo::Interface::TCP_INTERSERVER; #if USE_SSL - String cluster_secret = server.context()->getCluster(cluster)->getSecret(); + + String cluster_secret; + try + { + cluster_secret = server.context()->getCluster(cluster)->getSecret(); + } + catch (const Exception & e) + { + auto exception = Exception::createRuntime(ErrorCodes::AUTHENTICATION_FAILED, e.message()); + session->onAuthenticationFailure(/* user_name= */ std::nullopt, socket().peerAddress(), exception); + throw exception; /// NOLINT + } if (salt.empty() || cluster_secret.empty()) { From 993c3e6dba20306c711f900e2a4aa06d6564f5cc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 15 Nov 2023 15:03:16 +0100 Subject: [PATCH 373/813] Update outdated script in troubleshooting --- docs/en/operations/_troubleshooting.md | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/en/operations/_troubleshooting.md b/docs/en/operations/_troubleshooting.md index dbb0dad7976..b3846643e7a 100644 --- a/docs/en/operations/_troubleshooting.md +++ b/docs/en/operations/_troubleshooting.md @@ -17,12 +17,8 @@ - The issue may be happened when the GPG key is changed. -Please use the following scripts to resolve the issue: +Please use the manual from the [setup](../getting-started/install.md#setup-the-debian-repository) page to update the repository configuration. -```bash -sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 -sudo apt-get update -``` ### You Get Different Warnings with `apt-get update` {#you-get-different-warnings-with-apt-get-update} From 20cfe91ff967733bdbc34244759eb26d379c8869 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 15 Nov 2023 15:21:19 +0100 Subject: [PATCH 374/813] Remove unused error codes --- src/Functions/concat.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 346f96e4f03..081096b745e 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -16,9 +16,7 @@ namespace DB { namespace ErrorCodes { -extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int ILLEGAL_COLUMN; } using namespace GatherUtils; From e3842ebc893fe36941a1151547b886d86acbf899 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 15 Nov 2023 15:45:04 +0100 Subject: [PATCH 375/813] Remove substitution attrs from target node if source already has a value --- src/Common/Config/ConfigProcessor.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 3d64eac6fa6..9eb2e5ac033 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -326,15 +326,19 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, NodePtr new_node = config->importNode(with_node, true); config_root->replaceChild(new_node, config_node); } - else if (with_element.hasChildNodes() && with_element.firstChild()->nodeType() == Node::TEXT_NODE) - { - NodePtr new_node = config->importNode(with_node, true); - config_root->replaceChild(new_node, config_node); - } else { Element & config_element = dynamic_cast(*config_node); + /// Remove substitution attributes from the merge target node if source node already has a value + bool source_has_value = with_element.hasChildNodes(); + if (source_has_value) + for (const auto & attr_name: SUBSTITUTION_ATTRS) + { + if (config_element.hasAttribute(attr_name)) + config_element.removeAttribute(attr_name); + } + mergeAttributes(config_element, with_element); mergeRecursive(config, config_node, with_node); } From 9edcdf41d5d370fa21da0f751ab8f773abbd4495 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 15 Nov 2023 15:45:46 +0100 Subject: [PATCH 376/813] Add exceptions during substitution processing if node already has a value --- src/Common/Config/ConfigProcessor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 9eb2e5ac033..2e7372fa9a2 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -522,6 +522,9 @@ void ConfigProcessor::doIncludesRecursive( if (attr_nodes["from_zk"]) /// we have zookeeper subst { + if (node->hasChildNodes()) /// only allow substitution for nodes with no value + throw Poco::Exception("Element <" + node->nodeName() + "> has value, can't process from_zk substitution"); + contributing_zk_paths.insert(attr_nodes["from_zk"]->getNodeValue()); if (zk_node_cache) @@ -544,6 +547,9 @@ void ConfigProcessor::doIncludesRecursive( if (attr_nodes["from_env"]) /// we have env subst { + if (node->hasChildNodes()) /// only allow substitution for nodes with no value + throw Poco::Exception("Element <" + node->nodeName() + "> has value, can't process from_env substitution"); + XMLDocumentPtr env_document; auto get_env_node = [&](const std::string & name) -> const Node * { From a089e6181024a8f48e2e415de8033cca2fa98f50 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Nov 2023 15:55:56 +0100 Subject: [PATCH 377/813] Fix crash in FPC codec --- src/Compression/CompressionCodecFPC.cpp | 168 +++++++++--------- .../0_stateless/02915_fpc_overflow.reference | 2 + .../queries/0_stateless/02915_fpc_overflow.sh | 11 ++ 3 files changed, 101 insertions(+), 80 deletions(-) create mode 100644 tests/queries/0_stateless/02915_fpc_overflow.reference create mode 100755 tests/queries/0_stateless/02915_fpc_overflow.sh diff --git a/src/Compression/CompressionCodecFPC.cpp b/src/Compression/CompressionCodecFPC.cpp index 506093bbe49..ec8efa0fb38 100644 --- a/src/Compression/CompressionCodecFPC.cpp +++ b/src/Compression/CompressionCodecFPC.cpp @@ -153,23 +153,23 @@ void registerCodecFPC(CompressionCodecFactory & factory) namespace { -template -requires (sizeof(TUint) >= 4) +template +requires (sizeof(TUInt) >= 4) class DfcmPredictor { public: - explicit DfcmPredictor(std::size_t table_size) + explicit DfcmPredictor(size_t table_size) : table(table_size, 0), prev_value{0}, hash{0} { } [[nodiscard]] - TUint predict() const noexcept + TUInt predict() const noexcept { return table[hash] + prev_value; } - void add(TUint value) noexcept + void add(TUInt value) noexcept { table[hash] = value - prev_value; recalculateHash(); @@ -180,38 +180,38 @@ private: void recalculateHash() noexcept { auto value = table[hash]; - if constexpr (sizeof(TUint) >= 8) + if constexpr (sizeof(TUInt) >= 8) { - hash = ((hash << 2) ^ static_cast(value >> 40)) & (table.size() - 1); + hash = ((hash << 2) ^ static_cast(value >> 40)) & (table.size() - 1); } else { - hash = ((hash << 4) ^ static_cast(value >> 23)) & (table.size() - 1); + hash = ((hash << 4) ^ static_cast(value >> 23)) & (table.size() - 1); } } - std::vector table; - TUint prev_value; - std::size_t hash; + std::vector table; + TUInt prev_value; + size_t hash; }; -template -requires (sizeof(TUint) >= 4) +template +requires (sizeof(TUInt) >= 4) class FcmPredictor { public: - explicit FcmPredictor(std::size_t table_size) + explicit FcmPredictor(size_t table_size) : table(table_size, 0), hash{0} { } [[nodiscard]] - TUint predict() const noexcept + TUInt predict() const noexcept { return table[hash]; } - void add(TUint value) noexcept + void add(TUInt value) noexcept { table[hash] = value; recalculateHash(); @@ -221,31 +221,31 @@ private: void recalculateHash() noexcept { auto value = table[hash]; - if constexpr (sizeof(TUint) >= 8) + if constexpr (sizeof(TUInt) >= 8) { - hash = ((hash << 6) ^ static_cast(value >> 48)) & (table.size() - 1); + hash = ((hash << 6) ^ static_cast(value >> 48)) & (table.size() - 1); } else { - hash = ((hash << 1) ^ static_cast(value >> 22)) & (table.size() - 1); + hash = ((hash << 1) ^ static_cast(value >> 22)) & (table.size() - 1); } } - std::vector table; - std::size_t hash; + std::vector table; + size_t hash; }; -template +template class FPCOperation { - static constexpr auto VALUE_SIZE = sizeof(TUint); + static constexpr size_t VALUE_SIZE = sizeof(TUInt); static constexpr std::byte FCM_BIT{0}; static constexpr std::byte DFCM_BIT{1u << 3}; - static constexpr auto DFCM_BIT_1 = DFCM_BIT << 4; - static constexpr auto DFCM_BIT_2 = DFCM_BIT; - static constexpr unsigned MAX_ZERO_BYTE_COUNT = 0b111u; + static constexpr std::byte DFCM_BIT_1 = DFCM_BIT << 4; + static constexpr std::byte DFCM_BIT_2 = DFCM_BIT; + static constexpr UInt32 MAX_ZERO_BYTE_COUNT = 0b111u; static constexpr std::endian ENDIAN = std::endian::little; - static constexpr std::size_t CHUNK_SIZE = 64; + static constexpr size_t CHUNK_SIZE = 64; public: FPCOperation(std::span destination, UInt8 compression_level) @@ -253,12 +253,12 @@ public: { } - std::size_t encode(std::span data) && + size_t encode(std::span data) && { auto initial_size = result.size(); std::span chunk_view(chunk); - for (std::size_t i = 0; i < data.size(); i += chunk_view.size_bytes()) + for (size_t i = 0; i < data.size(); i += chunk_view.size_bytes()) { auto written_values_count = importChunk(data.subspan(i), chunk_view); encodeChunk(chunk_view.subspan(0, written_values_count)); @@ -267,12 +267,12 @@ public: return initial_size - result.size(); } - void decode(std::span values, std::size_t decoded_size) && + void decode(std::span values, size_t decoded_size) && { - std::size_t read_bytes = 0; + size_t read_bytes = 0; - std::span chunk_view(chunk); - for (std::size_t i = 0; i < decoded_size; i += chunk_view.size_bytes()) + std::span chunk_view(chunk); + for (size_t i = 0; i < decoded_size; i += chunk_view.size_bytes()) { if (i + chunk_view.size_bytes() > decoded_size) chunk_view = chunk_view.first(ceilBytesToEvenValues(decoded_size - i)); @@ -282,50 +282,50 @@ public: } private: - static std::size_t ceilBytesToEvenValues(std::size_t bytes_count) + static size_t ceilBytesToEvenValues(size_t bytes_count) { - auto values_count = (bytes_count + VALUE_SIZE - 1) / VALUE_SIZE; + size_t values_count = (bytes_count + VALUE_SIZE - 1) / VALUE_SIZE; return values_count % 2 == 0 ? values_count : values_count + 1; } - std::size_t importChunk(std::span values, std::span chnk) + size_t importChunk(std::span values, std::span current_chunk) { - if (auto chunk_view = std::as_writable_bytes(chnk); chunk_view.size() <= values.size()) + if (auto chunk_view = std::as_writable_bytes(current_chunk); chunk_view.size() <= values.size()) { - std::memcpy(chunk_view.data(), values.data(), chunk_view.size()); + memcpy(chunk_view.data(), values.data(), chunk_view.size()); return chunk_view.size() / VALUE_SIZE; } else { - std::memset(chunk_view.data(), 0, chunk_view.size()); - std::memcpy(chunk_view.data(), values.data(), values.size()); + memset(chunk_view.data(), 0, chunk_view.size()); + memcpy(chunk_view.data(), values.data(), values.size()); return ceilBytesToEvenValues(values.size()); } } - void exportChunk(std::span chnk) + void exportChunk(std::span current_chunk) { - auto chunk_view = std::as_bytes(chnk).first(std::min(result.size(), chnk.size_bytes())); - std::memcpy(result.data(), chunk_view.data(), chunk_view.size()); + auto chunk_view = std::as_bytes(current_chunk).first(std::min(result.size(), current_chunk.size_bytes())); + memcpy(result.data(), chunk_view.data(), chunk_view.size()); result = result.subspan(chunk_view.size()); } - void encodeChunk(std::span seq) + void encodeChunk(std::span sequence) { - for (std::size_t i = 0; i < seq.size(); i += 2) + for (size_t i = 0; i < sequence.size(); i += 2) { - encodePair(seq[i], seq[i + 1]); + encodePair(sequence[i], sequence[i + 1]); } } struct CompressedValue { - TUint value; - unsigned compressed_size; + TUInt value; + UInt32 compressed_size; std::byte predictor; }; - unsigned encodeCompressedZeroByteCount(unsigned compressed) + UInt32 encodeCompressedZeroByteCount(UInt32 compressed) { if constexpr (VALUE_SIZE == MAX_ZERO_BYTE_COUNT + 1) { @@ -335,7 +335,7 @@ private: return std::min(compressed, MAX_ZERO_BYTE_COUNT); } - unsigned decodeCompressedZeroByteCount(unsigned encoded_size) + UInt32 decodeCompressedZeroByteCount(UInt32 encoded_size) { if constexpr (VALUE_SIZE == MAX_ZERO_BYTE_COUNT + 1) { @@ -345,22 +345,22 @@ private: return encoded_size; } - CompressedValue compressValue(TUint value) noexcept + CompressedValue compressValue(TUInt value) noexcept { static constexpr auto BITS_PER_BYTE = std::numeric_limits::digits; - TUint compressed_dfcm = dfcm_predictor.predict() ^ value; - TUint compressed_fcm = fcm_predictor.predict() ^ value; + TUInt compressed_dfcm = dfcm_predictor.predict() ^ value; + TUInt compressed_fcm = fcm_predictor.predict() ^ value; dfcm_predictor.add(value); fcm_predictor.add(value); auto zeroes_dfcm = std::countl_zero(compressed_dfcm); auto zeroes_fcm = std::countl_zero(compressed_fcm); if (zeroes_dfcm > zeroes_fcm) - return {compressed_dfcm, encodeCompressedZeroByteCount(static_cast(zeroes_dfcm) / BITS_PER_BYTE), DFCM_BIT}; - return {compressed_fcm, encodeCompressedZeroByteCount(static_cast(zeroes_fcm) / BITS_PER_BYTE), FCM_BIT}; + return {compressed_dfcm, encodeCompressedZeroByteCount(static_cast(zeroes_dfcm) / BITS_PER_BYTE), DFCM_BIT}; + return {compressed_fcm, encodeCompressedZeroByteCount(static_cast(zeroes_fcm) / BITS_PER_BYTE), FCM_BIT}; } - void encodePair(TUint first, TUint second) + void encodePair(TUInt first, TUInt second) { auto [compressed_value1, zero_byte_count1, predictor1] = compressValue(first); auto [compressed_value2, zero_byte_count2, predictor2] = compressValue(second); @@ -374,24 +374,24 @@ private: auto tail_size1 = VALUE_SIZE - zero_byte_count1; auto tail_size2 = VALUE_SIZE - zero_byte_count2; - std::memcpy(result.data() + 1, valueTail(compressed_value1, zero_byte_count1), tail_size1); - std::memcpy(result.data() + 1 + tail_size1, valueTail(compressed_value2, zero_byte_count2), tail_size2); + memcpy(result.data() + 1, valueTail(compressed_value1, zero_byte_count1), tail_size1); + memcpy(result.data() + 1 + tail_size1, valueTail(compressed_value2, zero_byte_count2), tail_size2); result = result.subspan(1 + tail_size1 + tail_size2); } - std::size_t decodeChunk(std::span values, std::span seq) + size_t decodeChunk(std::span values, std::span sequence) { - std::size_t read_bytes = 0; - for (std::size_t i = 0; i < seq.size(); i += 2) + size_t read_bytes = 0; + for (size_t i = 0; i < sequence.size(); i += 2) { - read_bytes += decodePair(values.subspan(read_bytes), seq[i], seq[i + 1]); + read_bytes += decodePair(values.subspan(read_bytes), sequence[i], sequence[i + 1]); } return read_bytes; } - TUint decompressValue(TUint value, bool isDfcmPredictor) + TUInt decompressValue(TUInt value, bool isDfcmPredictor) { - TUint decompressed; + TUInt decompressed; if (isDfcmPredictor) { decompressed = dfcm_predictor.predict() ^ value; @@ -405,27 +405,35 @@ private: return decompressed; } - std::size_t decodePair(std::span bytes, TUint& first, TUint& second) + size_t decodePair(std::span bytes, TUInt & first, TUInt & second) { if (bytes.empty()) [[unlikely]] throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Unexpected end of encoded sequence"); - auto zero_byte_count1 = decodeCompressedZeroByteCount( - std::to_integer(bytes.front() >> 4) & MAX_ZERO_BYTE_COUNT); - auto zero_byte_count2 = decodeCompressedZeroByteCount( - std::to_integer(bytes.front()) & MAX_ZERO_BYTE_COUNT); + UInt32 zero_byte_count1 = decodeCompressedZeroByteCount( + std::to_integer(bytes.front() >> 4) & MAX_ZERO_BYTE_COUNT); + UInt32 zero_byte_count2 = decodeCompressedZeroByteCount( + std::to_integer(bytes.front()) & MAX_ZERO_BYTE_COUNT); - auto tail_size1 = VALUE_SIZE - zero_byte_count1; - auto tail_size2 = VALUE_SIZE - zero_byte_count2; + if (zero_byte_count1 > VALUE_SIZE || zero_byte_count2 > VALUE_SIZE) [[unlikely]] + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Invalid compressed data"); - if (bytes.size() < 1 + tail_size1 + tail_size2) [[unlikely]] + size_t tail_size1 = VALUE_SIZE - zero_byte_count1; + size_t tail_size2 = VALUE_SIZE - zero_byte_count2; + + size_t expected_size = 0; + if (__builtin_add_overflow(tail_size1, tail_size2, &expected_size) + || __builtin_add_overflow(expected_size, 1, &expected_size)) [[unlikely]] + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Invalid compressed data"); + + if (bytes.size() < expected_size) [[unlikely]] throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Unexpected end of encoded sequence"); - TUint value1 = 0; - TUint value2 = 0; + TUInt value1 = 0; + TUInt value2 = 0; - std::memcpy(valueTail(value1, zero_byte_count1), bytes.data() + 1, tail_size1); - std::memcpy(valueTail(value2, zero_byte_count2), bytes.data() + 1 + tail_size1, tail_size2); + memcpy(valueTail(value1, zero_byte_count1), bytes.data() + 1, tail_size1); + memcpy(valueTail(value2, zero_byte_count2), bytes.data() + 1 + tail_size1, tail_size2); auto is_dfcm_predictor1 = std::to_integer(bytes.front() & DFCM_BIT_1) != 0; auto is_dfcm_predictor2 = std::to_integer(bytes.front() & DFCM_BIT_2) != 0; @@ -435,7 +443,7 @@ private: return 1 + tail_size1 + tail_size2; } - static void* valueTail(TUint& value, unsigned compressed_size) + static void* valueTail(TUInt& value, UInt32 compressed_size) { if constexpr (ENDIAN == std::endian::little) { @@ -447,11 +455,11 @@ private: } } - DfcmPredictor dfcm_predictor; - FcmPredictor fcm_predictor; + DfcmPredictor dfcm_predictor; + FcmPredictor fcm_predictor; // memcpy the input into this buffer to align reads, this improves performance compared to unaligned reads (bit_cast) by ~10% - std::array chunk{}; + std::array chunk{}; std::span result{}; }; diff --git a/tests/queries/0_stateless/02915_fpc_overflow.reference b/tests/queries/0_stateless/02915_fpc_overflow.reference new file mode 100644 index 00000000000..73011ecb641 --- /dev/null +++ b/tests/queries/0_stateless/02915_fpc_overflow.reference @@ -0,0 +1,2 @@ +Exc +Exc diff --git a/tests/queries/0_stateless/02915_fpc_overflow.sh b/tests/queries/0_stateless/02915_fpc_overflow.sh new file mode 100755 index 00000000000..a10543ed8c4 --- /dev/null +++ b/tests/queries/0_stateless/02915_fpc_overflow.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo -ne 'checksumchecksum\x98\x90\x00\x00\x00\x11\x11\x11\x11\x04\x0f\x51 ' | + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&decompress=1&http_native_compression_disable_checksumming_on_decompress=1" --data-binary @- 2>&1 | grep -oF 'Exc' + +echo -ne 'checksumchecksum\x98\x90\x00\x00\x00\x11\x11\x11\x11\x04\x0f\x16 ' | + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&decompress=1&http_native_compression_disable_checksumming_on_decompress=1" --data-binary @- 2>&1 | grep -oF 'Exc' From a7543e3c7c36eee293c169c408a7ec2607fd86ad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Nov 2023 16:05:30 +0100 Subject: [PATCH 378/813] Fix test --- .../0_stateless/01555_system_distribution_queue_mask.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql index 3a90765226a..7ade1d24c59 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -17,7 +17,7 @@ system stop distributed sends dist_01555; insert into dist_01555 values (1)(2); -- since test_cluster_with_incorrect_pw contains incorrect password ignore error -system flush distributed dist_01555; -- { serverError 516 } +system flush distributed dist_01555; -- { clientError ATTEMPT_TO_READ_AFTER_EOF } select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; drop table dist_01555; @@ -30,7 +30,7 @@ create table dist_01555 (key Int) Engine=Distributed(test_cluster_with_incorrect insert into dist_01555 values (1)(2); -- since test_cluster_with_incorrect_pw contains incorrect password ignore error -system flush distributed dist_01555; -- { serverError 516 } +system flush distributed dist_01555; -- { clientError ATTEMPT_TO_READ_AFTER_EOF } select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; drop table dist_01555; From 5c4e58d27dc311c5115148e82b6cc690c0c7c972 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 15 Nov 2023 13:08:57 +0000 Subject: [PATCH 379/813] Try fix No user --- src/Functions/currentProfiles.cpp | 51 ++++++++++-------- src/Functions/currentRoles.cpp | 52 +++++++++++-------- .../System/StorageSystemFunctions.cpp | 2 - 3 files changed, 60 insertions(+), 45 deletions(-) diff --git a/src/Functions/currentProfiles.cpp b/src/Functions/currentProfiles.cpp index 71b0eda5f26..77c8a20ccee 100644 --- a/src/Functions/currentProfiles.cpp +++ b/src/Functions/currentProfiles.cpp @@ -45,8 +45,34 @@ namespace return toString(kind); } - explicit FunctionProfiles(const ContextPtr & context, Kind kind_) + explicit FunctionProfiles(const ContextPtr & context_, Kind kind_) : kind(kind_) + , context(context_) + {} + + size_t getNumberOfArguments() const override { return 0; } + bool isDeterministic() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(std::make_shared()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + std::call_once(initialized_flag, [&]{ initialize(); }); + + auto col_res = ColumnArray::create(ColumnString::create()); + ColumnString & res_strings = typeid_cast(col_res->getData()); + ColumnArray::Offsets & res_offsets = col_res->getOffsets(); + for (const String & profile_name : profile_names) + res_strings.insertData(profile_name.data(), profile_name.length()); + res_offsets.push_back(res_strings.size()); + return ColumnConst::create(std::move(col_res), input_rows_count); + } + + private: + void initialize() const { const auto & manager = context->getAccessControl(); @@ -62,28 +88,11 @@ namespace profile_names = manager.tryReadNames(profile_ids); } - size_t getNumberOfArguments() const override { return 0; } - bool isDeterministic() const override { return false; } + mutable std::once_flag initialized_flag; - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(std::make_shared()); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override - { - auto col_res = ColumnArray::create(ColumnString::create()); - ColumnString & res_strings = typeid_cast(col_res->getData()); - ColumnArray::Offsets & res_offsets = col_res->getOffsets(); - for (const String & profile_name : profile_names) - res_strings.insertData(profile_name.data(), profile_name.length()); - res_offsets.push_back(res_strings.size()); - return ColumnConst::create(std::move(col_res), input_rows_count); - } - - private: Kind kind; - Strings profile_names; + ContextPtr context; + mutable Strings profile_names; }; } diff --git a/src/Functions/currentRoles.cpp b/src/Functions/currentRoles.cpp index 45d2000d088..e577bd5377a 100644 --- a/src/Functions/currentRoles.cpp +++ b/src/Functions/currentRoles.cpp @@ -35,7 +35,33 @@ namespace String getName() const override { return name; } - explicit FunctionCurrentRoles(const ContextPtr & context) + explicit FunctionCurrentRoles(const ContextPtr & context_) + : context(context_) + {} + + size_t getNumberOfArguments() const override { return 0; } + bool isDeterministic() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(std::make_shared()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + std::call_once(initialized_flag, [&]{ initialize(); }); + + auto col_res = ColumnArray::create(ColumnString::create()); + ColumnString & res_strings = typeid_cast(col_res->getData()); + ColumnArray::Offsets & res_offsets = col_res->getOffsets(); + for (const String & role_name : role_names) + res_strings.insertData(role_name.data(), role_name.length()); + res_offsets.push_back(res_strings.size()); + return ColumnConst::create(std::move(col_res), input_rows_count); + } + + private: + void initialize() const { if constexpr (kind == Kind::CURRENT_ROLES) { @@ -57,27 +83,9 @@ namespace ::sort(role_names.begin(), role_names.end()); } - size_t getNumberOfArguments() const override { return 0; } - bool isDeterministic() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(std::make_shared()); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override - { - auto col_res = ColumnArray::create(ColumnString::create()); - ColumnString & res_strings = typeid_cast(col_res->getData()); - ColumnArray::Offsets & res_offsets = col_res->getOffsets(); - for (const String & role_name : role_names) - res_strings.insertData(role_name.data(), role_name.length()); - res_offsets.push_back(res_strings.size()); - return ColumnConst::create(std::move(col_res), input_rows_count); - } - - private: - Strings role_names; + mutable std::once_flag initialized_flag; + ContextPtr context; + mutable Strings role_names; }; } diff --git a/src/Storages/System/StorageSystemFunctions.cpp b/src/Storages/System/StorageSystemFunctions.cpp index 107f0c31ee0..9809b9435f2 100644 --- a/src/Storages/System/StorageSystemFunctions.cpp +++ b/src/Storages/System/StorageSystemFunctions.cpp @@ -20,7 +20,6 @@ namespace ErrorCodes { extern const int DICTIONARIES_WAS_NOT_LOADED; extern const int FUNCTION_NOT_ALLOWED; - extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int SUPPORT_IS_DISABLED; extern const int ACCESS_DENIED; @@ -150,7 +149,6 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c /// Some functions throw because they need special configuration or setup before use. if (e.code() == ErrorCodes::DICTIONARIES_WAS_NOT_LOADED || e.code() == ErrorCodes::FUNCTION_NOT_ALLOWED - || e.code() == ErrorCodes::LOGICAL_ERROR || e.code() == ErrorCodes::NOT_IMPLEMENTED || e.code() == ErrorCodes::SUPPORT_IS_DISABLED || e.code() == ErrorCodes::ACCESS_DENIED) From 3e47a54a94e32c293ad8f98166f829453e238320 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 30 Oct 2023 21:13:38 +0100 Subject: [PATCH 380/813] Added a shortcut for no-wildcard globs in file storage --- src/Storages/StorageFile.cpp | 101 ++++++++++++++++++++--------------- 1 file changed, 58 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 58c9a973575..fcac7673b16 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -106,34 +106,44 @@ namespace ErrorCodes namespace { -/// Forward-declare to use in expandSelector() -void listFilesWithRegexpMatchingImpl( - const std::string & path_for_ls, - const std::string & for_match, - size_t & total_bytes_to_read, - std::vector & result, - bool recursive = false); - /// Process {a,b,c...} globs separately: don't match it against regex, but generate a,b,c strings instead. -void expandSelector(const std::string & path_for_ls, - const std::string & for_match, - size_t & total_bytes_to_read, - std::vector & result, - bool recursive) +void expandSelector(const std::string & path, Strings & for_match_paths_expanded) { + /// regexp for {expr1,expr2,expr3}, expr.. should be without "{", "}", "*" and "," + static const re2::RE2 selector_regex(R"({([^{}*,]+,[^{}*]*[^{}*,])})"); + + std::string_view path_view(path); + std::string_view matched; + + if (RE2::FindAndConsume(&path_view, selector_regex, &matched)) + std::string buffer(matched); + else + { + for_match_paths_expanded.push_back(path); + return; + } + + Strings expanded_paths; + std::vector anchor_positions = {}; bool opened = false, closed = false; - for (std::string::const_iterator it = for_match.begin(); it != for_match.end(); it++) + for (std::string::const_iterator it = path.begin(); it != path.end(); it++) { if (*it == '{') { - anchor_positions.push_back(std::distance(for_match.begin(), it)); + if (opened) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unexpected '{{' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); + anchor_positions.push_back(std::distance(path.begin(), it)); opened = true; } else if (*it == '}') { - anchor_positions.push_back(std::distance(for_match.begin(), it)); + if (!opened) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unexpected '}}' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); + anchor_positions.push_back(std::distance(path.begin(), it)); closed = true; break; } @@ -141,22 +151,22 @@ void expandSelector(const std::string & path_for_ls, { if (!opened) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected ''' found in path '{}' at position {}.", for_match, std::distance(for_match.begin(), it)); - anchor_positions.push_back(std::distance(for_match.begin(), it)); + "Unexpected ',' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); + anchor_positions.push_back(std::distance(path.begin(), it)); } } if (!opened || !closed) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Invalid {{}} glob in path {}.", for_match); + "Invalid {{}} glob in path {}.", path); - std::string common_prefix = for_match.substr(0, anchor_positions[0]); - std::string common_suffix = for_match.substr(anchor_positions[anchor_positions.size()-1] + 1); + std::string common_prefix = path.substr(0, anchor_positions[0]); + std::string common_suffix = path.substr(anchor_positions[anchor_positions.size()-1] + 1); for (size_t i = 1; i < anchor_positions.size(); ++i) { std::string expanded_matcher = common_prefix - + for_match.substr(anchor_positions[i-1] + 1, (anchor_positions[i] - anchor_positions[i-1] - 1)) + + path.substr(anchor_positions[i-1] + 1, (anchor_positions[i] - anchor_positions[i-1] - 1)) + common_suffix; - listFilesWithRegexpMatchingImpl(path_for_ls, expanded_matcher, total_bytes_to_read, result, recursive); + expandSelector(expanded_matcher, for_match_paths_expanded); } } @@ -168,25 +178,25 @@ void listFilesWithRegexpMatchingImpl( const std::string & for_match, size_t & total_bytes_to_read, std::vector & result, - bool recursive) + bool recursive = false) { - /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without "{", "}", "*" and "," - static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); - - std::string_view for_match_view(for_match); - std::string_view matched; - if (RE2::FindAndConsume(&for_match_view, enum_or_range, &matched)) - { - std::string buffer(matched); - if (buffer.find(',') != std::string::npos) - { - expandSelector(path_for_ls, for_match, total_bytes_to_read, result, recursive); - return; - } - } - const size_t first_glob_pos = for_match.find_first_of("*?{"); + if (first_glob_pos == std::string::npos) + { + try + { + fs::path path = fs::canonical(path_for_ls + for_match); + result.push_back(path.string()); + } + catch (const std::exception &) + { + /// There is no such file, but we just ignore this. +// throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", for_match); + } + return; + } + const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); const std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' @@ -201,7 +211,7 @@ void listFilesWithRegexpMatchingImpl( throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); - bool skip_regex = current_glob == "/*" ? true : false; + bool skip_regex = current_glob == "/*"; if (!recursive) recursive = current_glob == "/**" ; @@ -245,12 +255,17 @@ void listFilesWithRegexpMatchingImpl( } std::vector listFilesWithRegexpMatching( - const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read) { std::vector result; - listFilesWithRegexpMatchingImpl(path_for_ls, for_match, total_bytes_to_read, result); + + Strings for_match_paths_expanded; + expandSelector(for_match, for_match_paths_expanded); + + for (const auto & for_match_expanded : for_match_paths_expanded) + listFilesWithRegexpMatchingImpl("/", for_match_expanded, total_bytes_to_read, result); + return result; } @@ -415,7 +430,7 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user else { /// We list only non-directory files. - paths = listFilesWithRegexpMatching("/", path, total_bytes_to_read); + paths = listFilesWithRegexpMatching(path, total_bytes_to_read); can_be_directory = false; } From 70aa6e267234f39467a72166467a96d9cdc5abfd Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 2 Nov 2023 09:58:15 +0100 Subject: [PATCH 381/813] Added a shortcut for no-wildcard globs in HDFS --- src/Common/parseGlobs.cpp | 68 +++++++++++++++++++++++ src/Common/parseGlobs.h | 6 +- src/Storages/HDFS/StorageHDFS.cpp | 92 +++++++++---------------------- src/Storages/StorageFile.cpp | 66 +--------------------- 4 files changed, 100 insertions(+), 132 deletions(-) diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index e9460c03d0a..07f78730591 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -17,6 +17,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + /* Transforms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp for using re2 library for matching * with such steps: * 1) search intervals like {0..9} and enums like {abc,xyz,qwe} in {}, replace them by regexp with pipe (expr1|expr2|expr3), @@ -116,4 +121,67 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob } return buf_final_processing.str(); } + +void expandSelector(const std::string & path, Strings & for_match_paths_expanded) +{ + /// regexp for {expr1,expr2,expr3}, expr.. should be without "{", "}", "*" and "," + static const re2::RE2 selector_regex(R"({([^{}*,]+,[^{}*]*[^{}*,])})"); + + std::string_view path_view(path); + std::string_view matched; + + if (RE2::FindAndConsume(&path_view, selector_regex, &matched)) + std::string buffer(matched); + else + { + for_match_paths_expanded.push_back(path); + return; + } + + Strings expanded_paths; + + std::vector anchor_positions = {}; + bool opened = false, closed = false; + + for (std::string::const_iterator it = path.begin(); it != path.end(); it++) + { + if (*it == '{') + { + if (opened) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unexpected '{{' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); + anchor_positions.push_back(std::distance(path.begin(), it)); + opened = true; + } + else if (*it == '}') + { + if (!opened) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unexpected '}}' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); + anchor_positions.push_back(std::distance(path.begin(), it)); + closed = true; + break; + } + else if (*it == ',') + { + if (!opened) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unexpected ',' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); + anchor_positions.push_back(std::distance(path.begin(), it)); + } + } + if (!opened || !closed) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Invalid {{}} glob in path {}.", path); + + std::string common_prefix = path.substr(0, anchor_positions[0]); + std::string common_suffix = path.substr(anchor_positions[anchor_positions.size()-1] + 1); + for (size_t i = 1; i < anchor_positions.size(); ++i) + { + std::string expanded_matcher = common_prefix + + path.substr(anchor_positions[i-1] + 1, (anchor_positions[i] - anchor_positions[i-1] - 1)) + + common_suffix; + expandSelector(expanded_matcher, for_match_paths_expanded); + } +} } diff --git a/src/Common/parseGlobs.h b/src/Common/parseGlobs.h index 043a87884cf..1397d84c8a4 100644 --- a/src/Common/parseGlobs.h +++ b/src/Common/parseGlobs.h @@ -6,5 +6,9 @@ namespace DB { /* Parse globs in string and make a regexp for it. */ -std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs); + std::string makeRegexpPatternFromGlobs(const std::string &initial_str_with_globs); + + +/// Process {a,b,c...} globs separately: don't match it against regex, but generate a,b,c strings instead. + void expandSelector(const std::string &path, std::vector &for_match_paths_expanded); } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index d827353ad8e..55aad03b7f7 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -75,59 +75,6 @@ namespace ErrorCodes } namespace { - /// Forward-declare to use in expandSelector() - std::vector LSWithRegexpMatching(const String & path_for_ls, - const HDFSFSPtr & fs, - const String & for_match); - - /// Process {a,b,c...} globs separately: don't match it against regex, but generate a,b,c strings instead. - std::vector expandSelector(const String & path_for_ls, - const HDFSFSPtr & fs, - const String & for_match) - { - std::vector anchor_positions = {}; - bool opened = false, closed = false; - - for (std::string::const_iterator it = for_match.begin(); it != for_match.end(); it++) - { - if (*it == '{') - { - anchor_positions.push_back(std::distance(for_match.begin(), it)); - opened = true; - } - else if (*it == '}') - { - anchor_positions.push_back(std::distance(for_match.begin(), it)); - closed = true; - break; - } - else if (*it == ',') - { - if (!opened) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected ''' found in path '{}' at position {}.", for_match, std::distance(for_match.begin(), it)); - anchor_positions.push_back(std::distance(for_match.begin(), it)); - } - } - if (!opened || !closed) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Invalid {{}} glob in path {}.", for_match); - - std::vector ret = {}; - - std::string common_prefix = for_match.substr(0, anchor_positions[0]); - std::string common_suffix = for_match.substr(anchor_positions[anchor_positions.size()-1] + 1); - for (size_t i = 1; i < anchor_positions.size(); ++i) - { - std::string expanded_matcher = common_prefix - + for_match.substr(anchor_positions[i-1] + 1, (anchor_positions[i] - anchor_positions[i-1] - 1)) - + common_suffix; - std::vector result_part = LSWithRegexpMatching(path_for_ls, fs, expanded_matcher); - ret.insert(ret.end(), result_part.begin(), result_part.end()); - } - return ret; - } - /* Recursive directory listing with matched paths as a result. * Have the same method in StorageFile. */ @@ -136,20 +83,24 @@ namespace const HDFSFSPtr & fs, const String & for_match) { - /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without "{", "}", "*" and "," - static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); - - std::string_view for_match_view(for_match); - std::string_view matched; - if (RE2::FindAndConsume(&for_match_view, enum_or_range, &matched)) - { - std::string buffer(matched); - if (buffer.find(',') != std::string::npos) - return expandSelector(path_for_ls, fs, for_match); - } + std::vector result; const size_t first_glob_pos = for_match.find_first_of("*?{"); + if (first_glob_pos == std::string::npos) + { + const String path = fs::path(path_for_ls + for_match.substr(1)).lexically_normal(); + HDFSFileInfo ls; + ls.file_info = hdfsGetPathInfo(fs.get(), path.c_str()); + if (ls.file_info != nullptr) // NOLINT + { + result.push_back(StorageHDFS::PathWithInfo{ + String(path), + StorageHDFS::PathInfo{ls.file_info->mLastMod, static_cast(ls.file_info->mSize)}}); + } + return result; + } + const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' @@ -171,7 +122,7 @@ namespace throw Exception( ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", prefix_without_globs, String(hdfsGetLastError())); } - std::vector result; + if (!ls.file_info && ls.length > 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); for (int i = 0; i < ls.length; ++i) @@ -222,7 +173,16 @@ namespace HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - auto res = LSWithRegexpMatching("/", fs, path_from_uri); + Strings paths; + expandSelector(path_from_uri, paths); + + std::vector res; + + for (const auto & path : paths) + { + auto part_of_res = LSWithRegexpMatching("/", fs, path); + res.insert(res.end(), part_of_res.begin(), part_of_res.end()); + } return res; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index fcac7673b16..4c982473255 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -106,70 +106,6 @@ namespace ErrorCodes namespace { -/// Process {a,b,c...} globs separately: don't match it against regex, but generate a,b,c strings instead. -void expandSelector(const std::string & path, Strings & for_match_paths_expanded) -{ - /// regexp for {expr1,expr2,expr3}, expr.. should be without "{", "}", "*" and "," - static const re2::RE2 selector_regex(R"({([^{}*,]+,[^{}*]*[^{}*,])})"); - - std::string_view path_view(path); - std::string_view matched; - - if (RE2::FindAndConsume(&path_view, selector_regex, &matched)) - std::string buffer(matched); - else - { - for_match_paths_expanded.push_back(path); - return; - } - - Strings expanded_paths; - - std::vector anchor_positions = {}; - bool opened = false, closed = false; - - for (std::string::const_iterator it = path.begin(); it != path.end(); it++) - { - if (*it == '{') - { - if (opened) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected '{{' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); - anchor_positions.push_back(std::distance(path.begin(), it)); - opened = true; - } - else if (*it == '}') - { - if (!opened) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected '}}' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); - anchor_positions.push_back(std::distance(path.begin(), it)); - closed = true; - break; - } - else if (*it == ',') - { - if (!opened) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected ',' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); - anchor_positions.push_back(std::distance(path.begin(), it)); - } - } - if (!opened || !closed) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Invalid {{}} glob in path {}.", path); - - std::string common_prefix = path.substr(0, anchor_positions[0]); - std::string common_suffix = path.substr(anchor_positions[anchor_positions.size()-1] + 1); - for (size_t i = 1; i < anchor_positions.size(); ++i) - { - std::string expanded_matcher = common_prefix - + path.substr(anchor_positions[i-1] + 1, (anchor_positions[i] - anchor_positions[i-1] - 1)) - + common_suffix; - expandSelector(expanded_matcher, for_match_paths_expanded); - } -} - /* Recursive directory listing with matched paths as a result. * Have the same method in StorageHDFS. */ @@ -189,7 +125,7 @@ void listFilesWithRegexpMatchingImpl( fs::path path = fs::canonical(path_for_ls + for_match); result.push_back(path.string()); } - catch (const std::exception &) + catch (const std::exception &) // NOLINT { /// There is no such file, but we just ignore this. // throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", for_match); From eb08bba6077a41f974843739ce8a13f5b463b746 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 13:32:31 +0000 Subject: [PATCH 382/813] Update docs --- docs/en/sql-reference/table-functions/file.md | 73 +++++++++---------- .../sql-reference/table-functions/remote.md | 45 +++++++----- src/Common/parseGlobs.h | 10 +-- 3 files changed, 64 insertions(+), 64 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 7e869af82ef..a871bdaafa9 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -1,4 +1,4 @@ ---- + -- slug: /en/sql-reference/table-functions/file sidebar_position: 60 sidebar_label: file @@ -6,7 +6,7 @@ sidebar_label: file # file -Provides a table-like interface to SELECT from and INSERT to files. This table function is similar to the [s3](/docs/en/sql-reference/table-functions/url.md) table function. Use file() when working with local files, and s3() when working with buckets in S3, GCS, or MinIO. +A table engine which provides a table-like interface to SELECT from and INSERT into files, similar to the [s3](/docs/en/sql-reference/table-functions/url.md) table function. Use `file()` when working with local files, and `s3()` when working with buckets in object storage such as S3, GCS, or MinIO. The `file` function can be used in `SELECT` and `INSERT` queries to read from or write to files. @@ -18,18 +18,18 @@ 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). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. -- `path_to_archive` - The relative path to zip/tar/7z archive. Path to archive support the same globs as `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). 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, ...'`. -- `compression` — The existing compression type when used in a `SELECT` query, or the desired compression type when used in an `INSERT` query. The supported compression types are `gz`, `br`, `xz`, `zst`, `lz4`, and `bz2`. +- `compression` — The existing compression type when used in a `SELECT` query, or the desired compression type when used in an `INSERT` query. Supported compression types are `gz`, `br`, `xz`, `zst`, `lz4`, and `bz2`. **Returned value** -A table with the specified structure for reading or writing data in the specified file. +A table for reading or writing data in a file. -## File Write Examples +## Examples for Writing to a File ### Write to a TSV file @@ -48,9 +48,9 @@ As a result, the data is written into the file `test.tsv`: 1 3 2 ``` -### Partitioned Write to multiple TSV files +### Partitioned write to multiple TSV files -If you specify `PARTITION BY` expression when inserting data into a file() function, a separate file is created for each partition value. Splitting the data into separate files helps to improve reading operations efficiency. +If you specify a `PARTITION BY` expression when inserting data into a table function of type `file()`, then a separate file is created for each partition. Splitting the data into separate files helps to improve performance of read operations. ```sql INSERT INTO TABLE FUNCTION @@ -72,11 +72,11 @@ As a result, the data is written into three files: `test_1.tsv`, `test_2.tsv`, a 1 2 3 ``` -## File Read Examples +## Examples for Reading from a File ### SELECT from a CSV file -Setting `user_files_path` and the contents of the file `test.csv`: +First, set `user_files_path` in the server configuration and prepare a file `test.csv`: ``` bash $ grep user_files_path /etc/clickhouse-server/config.xml @@ -88,7 +88,7 @@ $ cat /var/lib/clickhouse/user_files/test.csv 78,43,45 ``` -Getting data from a table in `test.csv` and selecting the first two rows from it: +Then, read data from `test.csv` into a table and select its first two rows: ``` sql SELECT * FROM @@ -103,14 +103,6 @@ LIMIT 2; └─────────┴─────────┴─────────┘ ``` -Getting the first 10 lines of a table that contains 3 columns of [UInt32](/docs/en/sql-reference/data-types/int-uint.md) type from a CSV file: - -``` sql -SELECT * FROM -file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') -LIMIT 10; -``` - ### Inserting data from a file into a table: ``` sql @@ -130,41 +122,42 @@ file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32'); └─────────┴─────────┴─────────┘ ``` -Getting data from table in table.csv, located in archive1.zip or/and archive2.zip +Reading data from `table.csv`, located in `archive1.zip` or/and `archive2.zip`: + ``` sql SELECT * FROM file('user_files/archives/archive{1..2}.zip :: table.csv'); ``` -## Globs in Path {#globs_in_path} +## Globbing {#globs_in_path} -Multiple path components can have globs. For being processed file must exist and match to the whole path pattern (not only suffix or prefix). +Paths may use globbing. Files must match the whole path pattern, not only the suffix or prefix. -- `*` — Substitutes any number of any characters except `/` including empty string. -- `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. The strings can contain the `/` symbol. -- `{N..M}` — Substitutes any number in range from N to M including both borders. -- `**` - Fetches all files inside the folder recursively. +- `*` — Represents arbitrarily many characters except `/` but including the empty string. +- `?` — Represents an arbitrary single character. +- `{some_string,another_string,yet_another_one}` — Represents any of alternative strings `'some_string', 'another_string', 'yet_another_one'`. The strings may contain `/`. +- `{N..M}` — Represents any number `>= N` and `<= M`. +- `**` - Represents all files inside a folder recursively. Constructions with `{}` are similar to the [remote](remote.md) table function. **Example** -Suppose we have several files with the following relative paths: +Suppose there are these files with the following relative paths: -- 'some_dir/some_file_1' -- 'some_dir/some_file_2' -- 'some_dir/some_file_3' -- 'another_dir/some_file_1' -- 'another_dir/some_file_2' -- 'another_dir/some_file_3' +- `some_dir/some_file_1` +- `some_dir/some_file_2` +- `some_dir/some_file_3` +- `another_dir/some_file_1` +- `another_dir/some_file_2` +- `another_dir/some_file_3` -Query the number of rows in these files: +Query the total number of rows in all files: ``` sql SELECT count(*) FROM file('{some,another}_dir/some_file_{1..3}', 'TSV', 'name String, value UInt32'); ``` -Query the number of rows in all files of these two directories: +An alternative path expression which achieves the same: ``` sql SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32'); @@ -176,7 +169,7 @@ If your listing of files contains number ranges with leading zeros, use the cons **Example** -Query the data from files named `file000`, `file001`, … , `file999`: +Query the total number of rows in files named `file000`, `file001`, … , `file999`: ``` sql SELECT count(*) FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, value UInt32'); @@ -184,7 +177,7 @@ SELECT count(*) FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, **Example** -Query the data from all files inside `big_dir` directory recursively: +Query the total number of rows from all files inside directory `big_dir/` recursively: ``` sql SELECT count(*) FROM file('big_dir/**', 'CSV', 'name String, value UInt32'); @@ -192,7 +185,7 @@ SELECT count(*) FROM file('big_dir/**', 'CSV', 'name String, value UInt32'); **Example** -Query the data from all `file002` files from any folder inside `big_dir` directory recursively: +Query the total number of rows from all files `file002` inside any folder in directory `big_dir/` recursively: ``` sql SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt32'); diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index 59ed4bf1985..3ca177050d3 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -6,7 +6,7 @@ sidebar_label: remote # remote, remoteSecure -Allows accessing remote servers, including migration of data, without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. `remoteSecure` - same as `remote` but with a secured connection. +Table function `remote` allows to access remote servers on-the-fly, i.e. without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. Table function `remoteSecure` is same as `remote` but over a secure connection. Both functions can be used in `SELECT` and `INSERT` queries. @@ -21,36 +21,36 @@ remoteSecure('addresses_expr', [db.table, 'user'[, 'password'], sharding_key]) ## Parameters -- `addresses_expr` — An expression that generates addresses of remote servers. This may be just one server address. The server address is `host:port`, or just `host`. +- `addresses_expr` — A remote server address or an expression that generates multiple addresses of remote servers. Format: `host` or `host:port`. - The host can be specified as the server name, or as the IPv4 or IPv6 address. An IPv6 address is specified in square brackets. + The `host` can be specified as a server name, or as a IPv4 or IPv6 address. An IPv6 address must be specified in square brackets. - The port is the TCP port on the remote server. If the port is omitted, it uses [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) from the server’s config file in `remote` (by default, 9000) and [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) in `remoteSecure` (by default, 9440). + The `port` is the TCP port on the remote server. If the port is omitted, it uses [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) from the server config file for table function `remote` (by default, 9000) and [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) for table function `remoteSecure` (by default, 9440). - The port is required for an IPv6 address. + For IPv6 addresses, a port is required. - If only specify this parameter, `db` and `table` will use `system.one` by default. + If only parameter `addresses_expr` is specified, `db` and `table` will use `system.one` by default. Type: [String](../../sql-reference/data-types/string.md). - `db` — Database name. Type: [String](../../sql-reference/data-types/string.md). - `table` — Table name. Type: [String](../../sql-reference/data-types/string.md). -- `user` — User name. If the user is not specified, `default` is used. Type: [String](../../sql-reference/data-types/string.md). -- `password` — User password. If the password is not specified, an empty password is used. Type: [String](../../sql-reference/data-types/string.md). +- `user` — User name. If not specified, `default` is used. Type: [String](../../sql-reference/data-types/string.md). +- `password` — User password. If not specified, an empty password is used. Type: [String](../../sql-reference/data-types/string.md). - `sharding_key` — Sharding key to support distributing data across nodes. For example: `insert into remote('127.0.0.1:9000,127.0.0.2', db, table, 'default', rand())`. Type: [UInt32](../../sql-reference/data-types/int-uint.md). ## Returned value -The dataset from remote servers. +A table located on a remote server. ## Usage -Unless you are migrating data from one system to another, using the `remote` table function is less optimal than creating a `Distributed` table because in this case the server connection is re-established for every request. Also, if hostnames are set, the names are resolved, and errors are not counted when working with various replicas. When processing a large number of queries, always create the `Distributed` table ahead of time, and do not use the `remote` table function. +As table functions `remote` and `remoteSecure` re-establish the connection for each request, it is recommended to use a `Distributed` table instead. Also, if hostnames are set, the names are resolved, and errors are not counted when working with various replicas. When processing a large number of queries, always create the `Distributed` table ahead of time, and do not use the `remote` table function. The `remote` table function can be useful in the following cases: -- Migrating data from one system to another -- Accessing a specific server for data comparison, debugging, and testing. +- One-time data migration from one system to another +- Accessing a specific server for data comparison, debugging, and testing, i.e. ad-hoc connections. - Queries between various ClickHouse clusters for research purposes. - Infrequent distributed requests that are made manually. - Distributed requests where the set of servers is re-defined each time. @@ -68,7 +68,7 @@ localhost [2a02:6b8:0:1111::11]:9000 ``` -Multiple addresses can be comma-separated. In this case, ClickHouse will use distributed processing, so it will send the query to all specified addresses (like shards with different data). Example: +Multiple addresses can be comma-separated. In this case, ClickHouse will use distributed processing and send the query to all specified addresses (like shards with different data). Example: ``` text example01-01-1,example01-02-1 @@ -91,10 +91,13 @@ SELECT * FROM remote_table; ``` ### Migration of tables from one system to another: + This example uses one table from a sample dataset. The database is `imdb`, and the table is `actors`. #### On the source ClickHouse system (the system that currently hosts the data) + - Verify the source database and table name (`imdb.actors`) + ```sql show databases ``` @@ -104,6 +107,7 @@ This example uses one table from a sample dataset. The database is `imdb`, and ``` - Get the CREATE TABLE statement from the source: + ``` select create_table_query from system.tables @@ -111,6 +115,7 @@ This example uses one table from a sample dataset. The database is `imdb`, and ``` Response + ```sql CREATE TABLE imdb.actors (`id` UInt32, `first_name` String, @@ -123,11 +128,13 @@ This example uses one table from a sample dataset. The database is `imdb`, and #### On the destination ClickHouse system: - Create the destination database: + ```sql CREATE DATABASE imdb ``` - Using the CREATE TABLE statement from the source, create the destination: + ```sql CREATE TABLE imdb.actors (`id` UInt32, `first_name` String, @@ -140,21 +147,23 @@ This example uses one table from a sample dataset. The database is `imdb`, and #### Back on the source deployment: Insert into the new database and table created on the remote system. You will need the host, port, username, password, destination database, and destination table. + ```sql INSERT INTO FUNCTION remoteSecure('remote.clickhouse.cloud:9440', 'imdb.actors', 'USER', 'PASSWORD') SELECT * from imdb.actors ``` -## Globs in Addresses {#globs-in-addresses} +## Globbing {#globs-in-addresses} Patterns in curly brackets `{ }` are used to generate a set of shards and to specify replicas. If there are multiple pairs of curly brackets, then the direct product of the corresponding sets is generated. + The following pattern types are supported. -- {*a*,*b*} - Any number of variants separated by a comma. The pattern is replaced with *a* in the first shard address and it is replaced with *b* in the second shard address and so on. For instance, `example0{1,2}-1` generates addresses `example01-1` and `example02-1`. -- {*n*..*m*} - A range of numbers. This pattern generates shard addresses with incrementing indices from *n* to *m*. `example0{1..2}-1` generates `example01-1` and `example02-1`. -- {*0n*..*0m*} - A range of numbers with leading zeroes. This modification preserves leading zeroes in indices. The pattern `example{01..03}-1` generates `example01-1`, `example02-1` and `example03-1`. -- {*a*|*b*} - Any number of variants separated by a `|`. The pattern specifies replicas. For instance, `example01-{1|2}` generates replicas `example01-1` and `example01-2`. +- `{a,b,c}` - Represents any of alternative strings `a`, `b` or `c`. The pattern is replaced with `a` in the first shard address and replaced with `b` in the second shard address and so on. For instance, `example0{1,2}-1` generates addresses `example01-1` and `example02-1`. +- `{N..M}` - A range of numbers. This pattern generates shard addresses with incrementing indices from `N` to (and including) `M`. For instance, `example0{1..2}-1` generates `example01-1` and `example02-1`. +- `{0n..0m}` - A range of numbers with leading zeroes. This pattern preserves leading zeroes in indices. For instance, `example{01..03}-1` generates `example01-1`, `example02-1` and `example03-1`. +- `{a|b}` - Any number of variants separated by a `|`. The pattern specifies replicas. For instance, `example01-{1|2}` generates replicas `example01-1` and `example01-2`. The query will be sent to the first healthy replica. However, for `remote` the replicas are iterated in the order currently set in the [load_balancing](../../operations/settings/settings.md#settings-load_balancing) setting. The number of generated addresses is limited by [table_function_remote_max_addresses](../../operations/settings/settings.md#table_function_remote_max_addresses) setting. diff --git a/src/Common/parseGlobs.h b/src/Common/parseGlobs.h index 1397d84c8a4..d109b146b4d 100644 --- a/src/Common/parseGlobs.h +++ b/src/Common/parseGlobs.h @@ -4,11 +4,9 @@ namespace DB { -/* Parse globs in string and make a regexp for it. - */ - std::string makeRegexpPatternFromGlobs(const std::string &initial_str_with_globs); + /// Parse globs in string and make a regexp for it. + std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs); - -/// Process {a,b,c...} globs separately: don't match it against regex, but generate a,b,c strings instead. - void expandSelector(const std::string &path, std::vector &for_match_paths_expanded); + /// Process {a,b,c...} globs separately: don't match it against regex, but generate a,b,c strings instead. + void expandSelector(const std::string & path, std::vector & for_match_paths_expanded); } From 20163e6aba4a2ec6ecc34ad7643af33dfbffb186 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 15 Nov 2023 15:34:52 +0100 Subject: [PATCH 383/813] add globbing to aspell-ignore --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index aedb267b3fb..0d3d4fef3dc 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1542,6 +1542,7 @@ github glibc globalIn globalNotIn +globbing glushkovds golang googletest From a8aa206cfdca23c395f292f3d525fec7e7ed16c1 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 15 Nov 2023 16:35:25 +0100 Subject: [PATCH 384/813] Update after review expandSelector is now returning vector directly minor changes --- src/Common/parseGlobs.cpp | 23 ++++++++++++++++------- src/Common/parseGlobs.h | 6 ++++-- src/Storages/HDFS/StorageHDFS.cpp | 3 +-- src/Storages/StorageFile.cpp | 3 +-- 4 files changed, 22 insertions(+), 13 deletions(-) diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index 07f78730591..e19393478b4 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -122,17 +122,18 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob return buf_final_processing.str(); } -void expandSelector(const std::string & path, Strings & for_match_paths_expanded) +namespace { - /// regexp for {expr1,expr2,expr3}, expr.. should be without "{", "}", "*" and "," +void expandSelectorGlobImpl(const std::string & path, std::vector & for_match_paths_expanded) +{ + /// regexp for {expr1,expr2,....}; + /// expr1, expr2,... cannot contain any of these: '{', '}', ',' static const re2::RE2 selector_regex(R"({([^{}*,]+,[^{}*]*[^{}*,])})"); std::string_view path_view(path); std::string_view matched; - if (RE2::FindAndConsume(&path_view, selector_regex, &matched)) - std::string buffer(matched); - else + if (!RE2::FindAndConsume(&path_view, selector_regex, &matched)) { for_match_paths_expanded.push_back(path); return; @@ -140,7 +141,7 @@ void expandSelector(const std::string & path, Strings & for_match_paths_expanded Strings expanded_paths; - std::vector anchor_positions = {}; + std::vector anchor_positions; bool opened = false, closed = false; for (std::string::const_iterator it = path.begin(); it != path.end(); it++) @@ -181,7 +182,15 @@ void expandSelector(const std::string & path, Strings & for_match_paths_expanded std::string expanded_matcher = common_prefix + path.substr(anchor_positions[i-1] + 1, (anchor_positions[i] - anchor_positions[i-1] - 1)) + common_suffix; - expandSelector(expanded_matcher, for_match_paths_expanded); + expandSelectorGlobImpl(expanded_matcher, for_match_paths_expanded); } } } + +std::vector expandSelectionGlob(const std::string & path) +{ + std::vector result; + expandSelectorGlobImpl(path, result); + return result; +} +} diff --git a/src/Common/parseGlobs.h b/src/Common/parseGlobs.h index d109b146b4d..83607766223 100644 --- a/src/Common/parseGlobs.h +++ b/src/Common/parseGlobs.h @@ -7,6 +7,8 @@ namespace DB /// Parse globs in string and make a regexp for it. std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs); - /// Process {a,b,c...} globs separately: don't match it against regex, but generate a,b,c strings instead. - void expandSelector(const std::string & path, std::vector & for_match_paths_expanded); + /// Process {a,b,c...} globs: + /// Don't match it against regex, but generate a,b,c strings instead and process each of them separately. + /// E.g. for a string like `file{1,2,3}.csv` return vector of strings: {`file1.csv`,`file2.csv`,`file3.csv`} + std::vector expandSelectionGlob(const std::string & path); } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 55aad03b7f7..7d845dac57d 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -173,8 +173,7 @@ namespace HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - Strings paths; - expandSelector(path_from_uri, paths); + Strings paths = expandSelectionGlob(path_from_uri); std::vector res; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 4c982473255..2dffdfa8be9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -196,8 +196,7 @@ std::vector listFilesWithRegexpMatching( { std::vector result; - Strings for_match_paths_expanded; - expandSelector(for_match, for_match_paths_expanded); + Strings for_match_paths_expanded = expandSelectionGlob(for_match); for (const auto & for_match_expanded : for_match_paths_expanded) listFilesWithRegexpMatchingImpl("/", for_match_expanded, total_bytes_to_read, result); From f21dd37d1838c2041a4dcd7671d616a3dca817a8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 15:42:00 +0000 Subject: [PATCH 385/813] Some fixups --- .../en/sql-reference/functions/string-functions.md | 12 +++++++++--- src/Columns/ColumnStringHelpers.h | 2 +- src/Functions/concat.cpp | 14 ++++++-------- src/Functions/formatString.h | 10 +++++----- .../02775_show_columns_called_from_clickhouse.sql | 2 +- .../02775_show_columns_called_from_mysql.expect | 2 +- 6 files changed, 23 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index dc324e8e331..4b6e0356301 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -429,7 +429,7 @@ SELECT format('{} {}', 'Hello', 'World') ## concat -Concatenates the strings listed in the arguments without separator. +Concatenates the given arguments. **Syntax** @@ -439,7 +439,9 @@ concat(s1, s2, ...) **Arguments** -Values of arbitrary types. If an argument is not a String or FixedString, it is converted to the String type using the default serialization. +At least two values of arbitrary type. + +Arguments which are not of types [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md) are converted to strings using their default serialization. As this decreases performance, it is not recommended to use non-String/FixedString arguments. **Returned values** @@ -449,6 +451,8 @@ If any of arguments is `NULL`, the function returns `NULL`. **Example** +Query: + ``` sql SELECT concat('Hello, ', 'World!'); ``` @@ -461,7 +465,7 @@ Result: └─────────────────────────────┘ ``` -**Example** +Query: ```sql SELECT concat(42, 144); @@ -540,6 +544,8 @@ Concatenates the given strings with a given separator. concatWithSeparator(sep, expr1, expr2, expr3...) ``` +Alias: `concat_ws` + **Arguments** - sep — separator. Const [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). diff --git a/src/Columns/ColumnStringHelpers.h b/src/Columns/ColumnStringHelpers.h index 851486e490a..97b52506ae0 100644 --- a/src/Columns/ColumnStringHelpers.h +++ b/src/Columns/ColumnStringHelpers.h @@ -62,7 +62,7 @@ public: return buffer; } - inline void rowWritten() + void rowWritten() { if constexpr (std::is_same_v) { diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 081096b745e..9aa6de5d219 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -91,7 +91,6 @@ private: else { /// Fallback: use generic implementation for not very important cases. - /// Concat of arbitrary types also goes here. return executeFormatImpl(arguments, input_rows_count); } @@ -108,7 +107,7 @@ private: std::vector offsets(num_arguments); std::vector fixed_string_sizes(num_arguments); std::vector> constant_strings(num_arguments); - std::vector converted_col_ptrs(num_arguments); + std::vector converted_col_ptrs(num_arguments); bool has_column_string = false; bool has_column_fixed_string = false; for (size_t i = 0; i < num_arguments; ++i) @@ -132,14 +131,13 @@ private: } else { - // An arbitrary type argument: converting it to a StringColumn first + /// A non-String/non-FixedString-type argument: use the default serialization to convert it to String const auto full_column = column->convertToFullIfNeeded(); const auto serialization = arguments[i].type->getDefaultSerialization(); - ColumnString::MutablePtr converted_col_str = ColumnString::create(); - static FormatSettings format_settings; - + auto converted_col_str = ColumnString::create(); ColumnStringHelpers::WriteHelper write_helper(*converted_col_str, column->size()); auto & write_buffer = write_helper.getWriteBuffer(); + FormatSettings format_settings; for (size_t j = 0; j < column->size(); ++j) { serialization->serializeText(*full_column, j, write_buffer, format_settings); @@ -147,12 +145,12 @@ private: } write_helper.finalize(); - // Same as the normal `ColumnString` branch + /// Same as the normal `ColumnString` branch has_column_string = true; data[i] = &converted_col_str->getChars(); offsets[i] = &converted_col_str->getOffsets(); - // keep the refcounted-pointer around (to be able to use data/offsets later) + /// Keep the refcounted-pointer alive converted_col_ptrs[i] = std::move(converted_col_str); } } diff --git a/src/Functions/formatString.h b/src/Functions/formatString.h index 4bdb672caf4..315e5c06227 100644 --- a/src/Functions/formatString.h +++ b/src/Functions/formatString.h @@ -1,14 +1,14 @@ #pragma once +#include +#include +#include +#include + #include #include #include #include -#include -#include -#include -#include - namespace DB { diff --git a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql index 752367517af..3bbcbb1a535 100644 --- a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql +++ b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql @@ -5,7 +5,7 @@ -- Tests the output of SHOW COLUMNS when called through the ClickHouse protocol. -- ----------------------------------------------------------------------------------- --- Please keep this test in-sync with 02775_show_columns_called_from_mysql.expect +-- Please keep this test in-sync with 02775_show_columns_called_from_clickhouse.expect -- ----------------------------------------------------------------------------------- DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/02775_show_columns_called_from_mysql.expect b/tests/queries/0_stateless/02775_show_columns_called_from_mysql.expect index bef5bd10ff3..8ba5774820e 100755 --- a/tests/queries/0_stateless/02775_show_columns_called_from_mysql.expect +++ b/tests/queries/0_stateless/02775_show_columns_called_from_mysql.expect @@ -6,7 +6,7 @@ # Tests the output of SHOW COLUMNS when called through the MySQL protocol. # ----------------------------------------------------------------------------------- -# Please keep this test in-sync with 02775_show_columns_called_through_clickhouse.sql +# Please keep this test in-sync with 02775_show_columns_called_from_clickhouse.sql # ----------------------------------------------------------------------------------- set basedir [file dirname $argv0] From 604b10238bee73fa64465373ad7859320c69cd3f Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Wed, 15 Nov 2023 17:45:28 +0200 Subject: [PATCH 386/813] SensitiveDataMasker: reduce the scope of mutex --- src/Common/SensitiveDataMasker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index fd7f45b11c6..c4d6952f31d 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -99,11 +99,11 @@ std::mutex SensitiveDataMasker::instance_mutex; void SensitiveDataMasker::setInstance(std::unique_ptr sensitive_data_masker_) { - std::lock_guard lock(instance_mutex); if (!sensitive_data_masker_) throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: the 'sensitive_data_masker' is not set"); + std::lock_guard lock(instance_mutex); if (sensitive_data_masker_->rulesCount() > 0) { sensitive_data_masker = std::move(sensitive_data_masker_); From 08d0a160aedba69a04d258850947a434f67c9fac Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Wed, 15 Nov 2023 17:48:24 +0200 Subject: [PATCH 387/813] test_reload_query_masking_rules: don't rely on num of events Due to the fact that integration tests are run in parallel and there might be no ClickHouse restart between several tests, relying on `system.events` in the test is fragile as it can be affected by previous tests. This commit removes any assumptions regarding `system.events` from the test and tries to keep it robust. --- .../test_reload_query_masking_rules/test.py | 23 +++---------------- 1 file changed, 3 insertions(+), 20 deletions(-) diff --git a/tests/integration/test_reload_query_masking_rules/test.py b/tests/integration/test_reload_query_masking_rules/test.py index 0f29bd0825e..f269aefbacb 100644 --- a/tests/integration/test_reload_query_masking_rules/test.py +++ b/tests/integration/test_reload_query_masking_rules/test.py @@ -37,15 +37,8 @@ def test_reload_query_masking_rules(): # event will not be registered node.query("SELECT 'TOPSECRET.TOPSECRET'") assert_logs_contain_with_retry(node, "SELECT 'TOPSECRET.TOPSECRET'") - - # If there were no 'QueryMaskingRulesMatch' events, the query below returns - # 0 rows - assert ( - node.query( - "SELECT count(value) FROM system.events WHERE name = 'QueryMaskingRulesMatch'" - ) - == "0\n" - ) + assert not node.contains_in_log(r"SELECT '\[hidden\]'") + node.rotate_logs() node.copy_file_to_container( os.path.join(SCRIPT_DIR, "configs/changed_settings.xml"), @@ -58,17 +51,7 @@ def test_reload_query_masking_rules(): # will be incremented node.query("SELECT 'TOPSECRET.TOPSECRET'") - assert_eq_with_retry( - node, - "SELECT count(value) FROM system.events WHERE name = 'QueryMaskingRulesMatch'", - "1", - ) assert_logs_contain_with_retry(node, r"SELECT '\[hidden\]'") - assert ( - node.query( - "SELECT value FROM system.events WHERE name = 'QueryMaskingRulesMatch'" - ) - == "1\n" - ) + assert not node.contains_in_log("SELECT 'TOPSECRET.TOPSECRET'") node.rotate_logs() From fbaa6a4e54d8bc1435d78cf031525093cb945684 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 9 Nov 2023 16:39:41 +0000 Subject: [PATCH 388/813] Optimize equality with is null check in JOIN ON section --- .../Passes/LogicalExpressionOptimizerPass.cpp | 187 ++++++++++++++++++ .../Passes/LogicalExpressionOptimizerPass.h | 11 ++ ...11_join_on_nullsafe_optimization.reference | 25 +++ .../02911_join_on_nullsafe_optimization.sql | 27 +++ 4 files changed, 250 insertions(+) create mode 100644 tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference create mode 100644 tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 46056aeaf6f..372c760a20a 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -5,11 +5,17 @@ #include #include #include +#include #include namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class LogicalExpressionOptimizerVisitor : public InDepthQueryTreeVisitorWithContext { public: @@ -21,6 +27,15 @@ public: void enterImpl(QueryTreeNodePtr & node) { + if (auto * join_node = node->as()) + { + join_stack.push_back(join_node); + return; + } + + if (!join_stack.empty() && join_stack.back()->getJoinExpression().get() == node.get()) + is_inside_on_section = true; + auto * function_node = node->as(); if (!function_node) @@ -29,6 +44,10 @@ public: if (function_node->getFunctionName() == "or") { tryReplaceOrEqualsChainWithIn(node); + + /// Operator <=> is not supported outside of JOIN ON section + if (is_inside_on_section) + tryOptimizeIsNotDistinctOrIsNull(node); return; } @@ -38,6 +57,20 @@ public: return; } } + + void leaveImpl(QueryTreeNodePtr & node) + { + if (!join_stack.empty() && join_stack.back()->getJoinExpression().get() == node.get()) + is_inside_on_section = false; + + if (auto * join_node = node->as()) + { + assert(join_stack.back() == join_node); + join_stack.pop_back(); + return; + } + } + private: void tryReplaceAndEqualsChainsWithConstant(QueryTreeNodePtr & node) { @@ -231,6 +264,160 @@ private: function_node.getArguments().getNodes() = std::move(or_operands); function_node.resolveAsFunction(or_function_resolver); } + + void tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node) + { + auto & function_node = node->as(); + assert(function_node.getFunctionName() == "or"); + + QueryTreeNodes or_operands; + + /// Indices of `equals` or `isNotDistinctFrom` functions in the vector above + std::vector equals_functions_indices; + + /** Map from `isNull` argument to indices of operands that contains that `isNull` functions + * `a = b OR (a IS NULL AND b IS NULL) OR (a IS NULL AND c IS NULL)` + * will be mapped to + * { + * a => [(a IS NULL AND b IS NULL), (a IS NULL AND c IS NULL)] + * b => [(a IS NULL AND b IS NULL)] + * c => [(a IS NULL AND c IS NULL)] + * } + * Then for each a <=> b we can find all operands that contains both a IS NULL and b IS NULL + */ + QueryTreeNodePtrWithHashMap> is_null_argument_to_indices; + + for (const auto & argument : function_node.getArguments()) + { + or_operands.push_back(argument); + + auto * argument_function = argument->as(); + if (!argument_function) + continue; + + const auto & func_name = argument_function->getFunctionName(); + if (func_name == "equals" || func_name == "isNotDistinctFrom") + equals_functions_indices.push_back(or_operands.size() - 1); + + if (func_name == "and") + { + for (const auto & and_argument : argument_function->getArguments().getNodes()) + { + auto * and_argument_function = and_argument->as(); + if (and_argument_function && and_argument_function->getFunctionName() == "isNull") + { + const auto & is_null_argument = and_argument_function->getArguments().getNodes()[0]; + is_null_argument_to_indices[is_null_argument].push_back(or_operands.size() - 1); + } + } + } + } + + /// OR operands that are changed to and needs to be re-resolved + std::unordered_set arguments_to_reresolve; + + for (size_t equals_function_idx : equals_functions_indices) + { + auto * equals_function = or_operands[equals_function_idx]->as(); + + /// For a <=> b we are looking for expressions containing both `a IS NULL` and `b IS NULL` combined with AND + const auto & argument_nodes = equals_function->getArguments().getNodes(); + const auto & lhs_is_null_parents = is_null_argument_to_indices[argument_nodes[0]]; + const auto & rhs_is_null_parents = is_null_argument_to_indices[argument_nodes[1]]; + std::unordered_set operands_to_optimize; + std::set_intersection(lhs_is_null_parents.begin(), lhs_is_null_parents.end(), + rhs_is_null_parents.begin(), rhs_is_null_parents.end(), + std::inserter(operands_to_optimize, operands_to_optimize.begin())); + + /// If we have `a = b OR (a IS NULL AND b IS NULL)` we can optimize it to `a <=> b` + if (!operands_to_optimize.empty() && equals_function->getFunctionName() == "equals") + arguments_to_reresolve.insert(equals_function_idx); + + for (size_t to_optimize_idx : operands_to_optimize) + { + /// We are looking for operand `a IS NULL AND b IS NULL AND ...` + auto * operand_to_optimize = or_operands[to_optimize_idx]->as(); + + /// Remove `a IS NULL` and `b IS NULL` arguments from AND + QueryTreeNodes new_arguments; + for (const auto & and_argument : operand_to_optimize->getArguments().getNodes()) + { + bool to_eliminate = false; + + const auto * and_argument_function = and_argument->as(); + if (and_argument_function && and_argument_function->getFunctionName() == "isNull") + { + const auto & is_null_argument = and_argument_function->getArguments().getNodes()[0]; + to_eliminate = (is_null_argument->isEqual(*argument_nodes[0]) || is_null_argument->isEqual(*argument_nodes[1])); + } + + if (to_eliminate) + arguments_to_reresolve.insert(to_optimize_idx); + else + new_arguments.emplace_back(and_argument); + } + /// If less than two arguments left, we will remove or replace the whole AND below + operand_to_optimize->getArguments().getNodes() = std::move(new_arguments); + } + } + + + if (arguments_to_reresolve.empty()) + /// Nothing have been changed + return; + + auto and_function_resolver = FunctionFactory::instance().get("and", getContext()); + auto strict_equals_function_resolver = FunctionFactory::instance().get("isNotDistinctFrom", getContext()); + QueryTreeNodes new_or_operands; + for (size_t i = 0; i < or_operands.size(); ++i) + { + if (arguments_to_reresolve.contains(i)) + { + auto * function = or_operands[i]->as(); + if (function->getFunctionName() == "equals") + { + /// Because we removed checks for IS NULL, we should replace `a = b` with `a <=> b` + function->resolveAsFunction(strict_equals_function_resolver); + new_or_operands.emplace_back(std::move(or_operands[i])); + } + else if (function->getFunctionName() == "and") + { + const auto & and_arguments = function->getArguments().getNodes(); + if (and_arguments.size() > 1) + { + function->resolveAsFunction(and_function_resolver); + new_or_operands.emplace_back(std::move(or_operands[i])); + } + else if (and_arguments.size() == 1) + { + /// Replace AND with a single argument with the argument itself + new_or_operands.emplace_back(std::move(and_arguments[0])); + } + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function name: '{}'", function->getFunctionName()); + } + else + { + new_or_operands.emplace_back(std::move(or_operands[i])); + } + } + + if (new_or_operands.size() == 1) + { + node = std::move(new_or_operands[0]); + return; + } + + /// Rebuild OR function + auto or_function_resolver = FunctionFactory::instance().get("or", getContext()); + function_node.getArguments().getNodes() = std::move(new_or_operands); + function_node.resolveAsFunction(or_function_resolver); + } + +private: + bool is_inside_on_section = false; + std::deque join_stack; }; void LogicalExpressionOptimizerPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h index 05c10ddc685..80062f38eac 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h @@ -67,6 +67,17 @@ namespace DB * FROM TABLE * WHERE a = 1 AND b = 'test'; * ------------------------------- + * + * 5. Remove unnecessary IS NULL checks in JOIN ON clause + * - equality check with explicit IS NULL check replaced with <=> operator + * ------------------------------- + * SELECT * FROM t1 JOIN t2 ON a = b OR (a IS NULL AND b IS NULL) + * SELECT * FROM t1 JOIN t2 ON a <=> b OR (a IS NULL AND b IS NULL) + * + * will be transformed into + * + * SELECT * FROM t1 JOIN t2 ON a <=> b + * ------------------------------- */ class LogicalExpressionOptimizerPass final : public IQueryTreePass diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference new file mode 100644 index 00000000000..976c1503b02 --- /dev/null +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference @@ -0,0 +1,25 @@ +-- { echoOn } +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) ORDER BY t1.x NULLS LAST; +2 2 2 2 +3 3 3 33 +\N \N \N \N +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND t2.x IS NULL) ORDER BY t1.x NULLS LAST; +1 42 4 42 +2 2 2 2 +3 3 3 33 +\N \N \N \N +SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) AND t1.y <=> t2.y ORDER BY t1.x NULLS LAST; +2 2 2 2 +\N \N \N \N +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND t1.y IS NULL AND t2.x IS NULL AND t2.y IS NULL)) ORDER BY t1.x NULLS LAST; +1 42 4 42 +2 2 2 2 +3 3 3 33 +\N \N \N \N +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) AND (t1.y == t2.y OR (t1.y IS NULL AND t2.y IS NULL)) AND COALESCE(t1.x, 0) != 2 ORDER BY t1.x NULLS LAST; +\N \N \N \N +SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; +0 +1 +1 +1 diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql new file mode 100644 index 00000000000..6a98a7bb57b --- /dev/null +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -0,0 +1,27 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (x Nullable(Int64), y Nullable(UInt64)) ENGINE = TinyLog; +CREATE TABLE t2 (x Nullable(Int64), y Nullable(UInt64)) ENGINE = TinyLog; + +INSERT INTO t1 VALUES (1,42), (2,2), (3,3), (NULL,NULL); +INSERT INTO t2 VALUES (NULL,NULL), (2,2), (3,33), (4,42); + +SET allow_experimental_analyzer = 1; + +-- { echoOn } +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) ORDER BY t1.x NULLS LAST; + +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND t2.x IS NULL) ORDER BY t1.x NULLS LAST; + +SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) AND t1.y <=> t2.y ORDER BY t1.x NULLS LAST; + +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND t1.y IS NULL AND t2.x IS NULL AND t2.y IS NULL)) ORDER BY t1.x NULLS LAST; + +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) AND (t1.y == t2.y OR (t1.y IS NULL AND t2.y IS NULL)) AND COALESCE(t1.x, 0) != 2 ORDER BY t1.x NULLS LAST; + +SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; +-- { echoOff } + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; From 05163be79c3aa7b106c93f084103be6af403ae80 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 10 Nov 2023 15:50:21 +0000 Subject: [PATCH 389/813] fix clang tidy --- src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 372c760a20a..9602ef8a743 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -391,7 +391,7 @@ private: else if (and_arguments.size() == 1) { /// Replace AND with a single argument with the argument itself - new_or_operands.emplace_back(std::move(and_arguments[0])); + new_or_operands.emplace_back(and_arguments[0]); } } else @@ -415,7 +415,6 @@ private: function_node.resolveAsFunction(or_function_resolver); } -private: bool is_inside_on_section = false; std::deque join_stack; }; From 9e3ba550701f68295774b7992e6261cd94da2167 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 15 Nov 2023 16:53:38 +0100 Subject: [PATCH 390/813] Fix incorrect nullable columns reporting in MySQL binary protocol --- .../MySQLJavaClientTest.java | 106 +++++++------- src/Core/MySQL/MySQLUtils.cpp | 11 +- src/Core/MySQL/MySQLUtils.h | 5 - src/Core/MySQL/PacketsProtocolBinary.cpp | 35 ++--- src/Core/MySQL/PacketsProtocolText.cpp | 7 +- .../test_mysql_protocol/java_client.reference | 133 ++++++++++++++++++ .../java_client_binary.reference | 129 ----------------- .../test_mysql_protocol/java_client_test.sql | 3 +- .../java_client_text.reference | 129 ----------------- tests/integration/test_mysql_protocol/test.py | 4 +- 10 files changed, 213 insertions(+), 349 deletions(-) create mode 100644 tests/integration/test_mysql_protocol/java_client.reference delete mode 100644 tests/integration/test_mysql_protocol/java_client_binary.reference delete mode 100644 tests/integration/test_mysql_protocol/java_client_text.reference diff --git a/docker/test/integration/mysql_java_client/MySQLJavaClientTest.java b/docker/test/integration/mysql_java_client/MySQLJavaClientTest.java index 1ac21ffe4b4..445e384ba1a 100644 --- a/docker/test/integration/mysql_java_client/MySQLJavaClientTest.java +++ b/docker/test/integration/mysql_java_client/MySQLJavaClientTest.java @@ -39,8 +39,7 @@ public class MySQLJavaClientTest { // useServerPrepStmts=true -> COM_STMT_PREPARE + COM_STMT_EXECUTE -> binary // useServerPrepStmts=false -> COM_QUERY -> text - String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=%s", - host, port, database, binary); + String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=%s", host, port, database, binary); try { Class.forName("com.mysql.cj.jdbc.Driver"); @@ -67,21 +66,21 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %d\n", getMysqlType(rs, "i8"), rs.getInt("i8")); - System.out.printf("%s, value: %d\n", getMysqlType(rs, "i16"), rs.getInt("i16")); - System.out.printf("%s, value: %d\n", getMysqlType(rs, "i32"), rs.getInt("i32")); - System.out.printf("%s, value: %d\n", getMysqlType(rs, "i64"), rs.getLong("i64")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "i128"), rs.getString("i128")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "i256"), rs.getString("i256")); - System.out.printf("%s, value: %d\n", getMysqlType(rs, "ui8"), rs.getInt("ui8")); - System.out.printf("%s, value: %d\n", getMysqlType(rs, "ui16"), rs.getInt("ui16")); - System.out.printf("%s, value: %d\n", getMysqlType(rs, "ui32"), rs.getLong("ui32")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "ui64"), rs.getString("ui64")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "ui128"), rs.getString("ui128")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "ui256"), rs.getString("ui256")); - System.out.printf("%s, value: %f\n", getMysqlType(rs, "f32"), rs.getFloat("f32")); - System.out.printf("%s, value: %f\n", getMysqlType(rs, "f64"), rs.getFloat("f64")); - System.out.printf("%s, value: %b\n", getMysqlType(rs, "b"), rs.getBoolean("b")); + System.out.printf("%s, value: %d, wasNull: %b\n", getMysqlType(rs, "i8"), rs.getInt("i8"), rs.wasNull()); + System.out.printf("%s, value: %d, wasNull: %b\n", getMysqlType(rs, "i16"), rs.getInt("i16"), rs.wasNull()); + System.out.printf("%s, value: %d, wasNull: %b\n", getMysqlType(rs, "i32"), rs.getInt("i32"), rs.wasNull()); + System.out.printf("%s, value: %d, wasNull: %b\n", getMysqlType(rs, "i64"), rs.getLong("i64"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "i128"), rs.getString("i128"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "i256"), rs.getString("i256"), rs.wasNull()); + System.out.printf("%s, value: %d, wasNull: %b\n", getMysqlType(rs, "ui8"), rs.getInt("ui8"), rs.wasNull()); + System.out.printf("%s, value: %d, wasNull: %b\n", getMysqlType(rs, "ui16"), rs.getInt("ui16"), rs.wasNull()); + System.out.printf("%s, value: %d, wasNull: %b\n", getMysqlType(rs, "ui32"), rs.getLong("ui32"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "ui64"), rs.getString("ui64"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "ui128"), rs.getString("ui128"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "ui256"), rs.getString("ui256"), rs.wasNull()); + System.out.printf("%s, value: %f, wasNull: %b\n", getMysqlType(rs, "f32"), rs.getFloat("f32"), rs.wasNull()); + System.out.printf("%s, value: %f, wasNull: %b\n", getMysqlType(rs, "f64"), rs.getFloat("f64"), rs.wasNull()); + System.out.printf("%s, value: %b, wasNull: %b\n", getMysqlType(rs, "b"), rs.getBoolean("b"), rs.wasNull()); } System.out.println(); } @@ -92,10 +91,10 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "s"), rs.getString("s")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "sn"), rs.getString("sn")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "lc"), rs.getString("lc")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "nlc"), rs.getString("nlc")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "s"), rs.getString("s"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "sn"), rs.getString("sn"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "lc"), rs.getString("lc"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "nlc"), rs.getString("nlc"), rs.wasNull()); } System.out.println(); } @@ -106,10 +105,10 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "ilc"), rs.getInt("ilc")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dlc"), rs.getDate("dlc")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "ilc"), rs.getInt("ilc"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dlc"), rs.getDate("dlc"), rs.wasNull()); // NULL int is represented as zero - System.out.printf("%s, value: %s\n", getMysqlType(rs, "ni"), rs.getInt("ni")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "ni"), rs.getInt("ni"), rs.wasNull()); } System.out.println(); } @@ -120,12 +119,11 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "d32"), rs.getBigDecimal("d32").toPlainString()); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "d64"), rs.getBigDecimal("d64").toPlainString()); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "d128_native"), - rs.getBigDecimal("d128_native").toPlainString()); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "d128_text"), rs.getString("d128_text")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "d256"), rs.getString("d256")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "d32"), rs.getBigDecimal("d32").toPlainString(), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "d64"), rs.getBigDecimal("d64").toPlainString(), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "d128_native"), rs.getBigDecimal("d128_native").toPlainString(), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "d128_text"), rs.getString("d128_text"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "d256"), rs.getString("d256"), rs.wasNull()); } System.out.println(); } @@ -136,12 +134,12 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "d32"), rs.getDate("d32")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_3"), rs.getTimestamp("dt64_3")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_6"), rs.getTimestamp("dt64_6")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_9"), rs.getTimestamp("dt64_9")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "d"), rs.getDate("d"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "d32"), rs.getDate("d32"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_3"), rs.getTimestamp("dt64_3"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_6"), rs.getTimestamp("dt64_6"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_9"), rs.getTimestamp("dt64_9"), rs.wasNull()); } System.out.println(); } @@ -152,13 +150,13 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_0"), rs.getTimestamp("dt64_0")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_1"), rs.getTimestamp("dt64_1")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_2"), rs.getTimestamp("dt64_2")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_4"), rs.getTimestamp("dt64_4")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_5"), rs.getTimestamp("dt64_5")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_7"), rs.getTimestamp("dt64_7")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_8"), rs.getTimestamp("dt64_8")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_0"), rs.getTimestamp("dt64_0"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_1"), rs.getTimestamp("dt64_1"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_2"), rs.getTimestamp("dt64_2"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_4"), rs.getTimestamp("dt64_4"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_5"), rs.getTimestamp("dt64_5"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_7"), rs.getTimestamp("dt64_7"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_8"), rs.getTimestamp("dt64_8"), rs.wasNull()); } System.out.println(); } @@ -169,8 +167,8 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_3"), rs.getTimestamp("dt64_3")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt64_3"), rs.getTimestamp("dt64_3"), rs.wasNull()); } System.out.println(); } @@ -181,10 +179,10 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "a"), rs.getString("a")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "u"), rs.getString("u")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "t"), rs.getString("t")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "m"), rs.getString("m")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "a"), rs.getString("a"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "u"), rs.getString("u"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "t"), rs.getString("t"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "m"), rs.getString("m"), rs.wasNull()); } System.out.println(); } @@ -196,17 +194,15 @@ public class MySQLJavaClientTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "f"), rs.getFloat("f")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d")); - System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt")); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "f"), rs.getFloat("f"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "d"), rs.getDate("d"), rs.wasNull()); + System.out.printf("%s, value: %s, wasNull: %b\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt"), rs.wasNull()); } System.out.println(); } private static String getMysqlType(ResultSet rs, String columnLabel) throws SQLException { ResultSetMetaData meta = rs.getMetaData(); - return String.format("%s type is %s", columnLabel, - MysqlType.getByJdbcType(meta.getColumnType(rs.findColumn(columnLabel)))); + return String.format("%s type is %s", columnLabel, MysqlType.getByJdbcType(meta.getColumnType(rs.findColumn(columnLabel)))); } - } diff --git a/src/Core/MySQL/MySQLUtils.cpp b/src/Core/MySQL/MySQLUtils.cpp index 8dedbdaa630..7ba6c32fd0d 100644 --- a/src/Core/MySQL/MySQLUtils.cpp +++ b/src/Core/MySQL/MySQLUtils.cpp @@ -12,8 +12,7 @@ namespace MySQLProtocol namespace MySQLUtils { -DecimalUtils::DecimalComponents -getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, size_t row_num) +DecimalUtils::DecimalComponents getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, size_t row_num) { const auto * date_time_type = typeid_cast(data_type.get()); @@ -52,14 +51,6 @@ getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, size_t r return components; }; - -ColumnPtr getBaseColumn(const DB::Columns & columns, size_t i) -{ - ColumnPtr col = columns[i]->convertToFullIfNeeded(); - if (col->isNullable()) - return assert_cast(*col).getNestedColumnPtr(); - return col; -}; } } } diff --git a/src/Core/MySQL/MySQLUtils.h b/src/Core/MySQL/MySQLUtils.h index 48b78fcf4d7..e77e9c22ee4 100644 --- a/src/Core/MySQL/MySQLUtils.h +++ b/src/Core/MySQL/MySQLUtils.h @@ -2,7 +2,6 @@ #include "Core/DecimalFunctions.h" #include "DataTypes/IDataType.h" -#include "base/types.h" namespace DB { @@ -13,10 +12,6 @@ namespace MySQLUtils /// Splits DateTime64 column data at a certain row number into whole and fractional part /// Additionally, normalizes the fractional part as if it was scale 6 for MySQL compatibility purposes DecimalUtils::DecimalComponents getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, size_t row_num); - -/// If a column is ColumnSparse/ColumnLowCardinality/ColumnNullable, it is unwrapped in a correct order; -/// otherwise, the original column is returned -ColumnPtr getBaseColumn(const DB::Columns & columns, size_t i); } } } diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index 7f5e3900cb0..2fe6b76d05f 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -4,20 +4,15 @@ #include #include "Common/LocalDate.h" #include "Common/LocalDateTime.h" -#include "Columns/ColumnLowCardinality.h" #include "Columns/ColumnNullable.h" #include "Columns/ColumnVector.h" -#include "Columns/ColumnsDateTime.h" #include "Core/DecimalFunctions.h" #include "DataTypes/DataTypeDateTime64.h" #include "DataTypes/DataTypeLowCardinality.h" -#include "DataTypes/DataTypeNullable.h" -#include "DataTypes/DataTypesNumber.h" #include "Formats/FormatSettings.h" #include "IO/WriteBufferFromString.h" #include "MySQLUtils.h" #include "base/DayNum.h" -#include "base/Decimal.h" #include "base/types.h" namespace DB @@ -30,17 +25,21 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp : row_num(row_num_), columns(columns_), data_types(data_types_), serializations(serializations_) { payload_size = 1 + null_bitmap_size; - FormatSettings format_settings; + static FormatSettings format_settings; for (size_t i = 0; i < columns.size(); ++i) { - ColumnPtr col = MySQLUtils::getBaseColumn(columns, i); - if (col->isNullAt(row_num)) + ColumnPtr col = columns[i]->convertToFullIfNeeded(); + if (col->isNullable()) { - // See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row - size_t byte = (i + 2) / 8; - int bit = 1 << ((i + 2) % 8); - null_bitmap[byte] |= bit; - continue; // NULLs are stored in the null bitmap only + if (columns[i]->isNullAt(row_num)) + { + // See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row + size_t byte = (i + 2) / 8; + int bit = 1 << ((i + 2) % 8); + null_bitmap[byte] |= bit; + continue; // NULLs are stored in the null bitmap only + } + col = assert_cast(*col).getNestedColumnPtr(); } DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]); @@ -145,9 +144,13 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const buffer.write(null_bitmap.data(), null_bitmap_size); for (size_t i = 0; i < columns.size(); ++i) { - ColumnPtr col = MySQLUtils::getBaseColumn(columns, i); - if (col->isNullAt(row_num)) - continue; + ColumnPtr col = columns[i]->convertToFullIfNeeded(); + if (col->isNullable()) + { + if (columns[i]->isNullAt(row_num)) + continue; + col = assert_cast(*col).getNestedColumnPtr(); + } DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]); TypeIndex type_index = data_type->getTypeId(); diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 744227f27dc..82d6e27d98a 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -5,8 +6,8 @@ #include "Common/assert_cast.h" #include "Core/MySQL/IMySQLWritePacket.h" #include "DataTypes/DataTypeLowCardinality.h" -#include "DataTypes/DataTypeNullable.h" #include "DataTypes/DataTypesDecimal.h" + #include "MySQLUtils.h" namespace DB @@ -36,7 +37,9 @@ ResultSetRow::ResultSetRow(const Serializations & serializations, const DataType else if (type_index == TypeIndex::DateTime64) { WriteBufferFromOwnString ostr; - ColumnPtr col = MySQLUtils::getBaseColumn(columns, i); + ColumnPtr col = columns[i]->convertToFullIfNeeded(); + if (col->isNullable()) + col = assert_cast(*col).getNestedColumnPtr(); auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num); writeDateTimeText<'-', ':', ' '>(LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type))), ostr); ostr.write('.'); diff --git a/tests/integration/test_mysql_protocol/java_client.reference b/tests/integration/test_mysql_protocol/java_client.reference new file mode 100644 index 00000000000..0e6ab5b5e79 --- /dev/null +++ b/tests/integration/test_mysql_protocol/java_client.reference @@ -0,0 +1,133 @@ +### testSimpleDataTypes +Row #1 +i8 type is TINYINT, value: -128, wasNull: false +i16 type is SMALLINT, value: -32768, wasNull: false +i32 type is INT, value: -2147483648, wasNull: false +i64 type is BIGINT, value: -9223372036854775808, wasNull: false +i128 type is CHAR, value: -170141183460469231731687303715884105728, wasNull: false +i256 type is CHAR, value: -57896044618658097711785492504343953926634992332820282019728792003956564819968, wasNull: false +ui8 type is TINYINT, value: 120, wasNull: false +ui16 type is SMALLINT, value: 1234, wasNull: false +ui32 type is INT, value: 51234, wasNull: false +ui64 type is BIGINT, value: 421342, wasNull: false +ui128 type is CHAR, value: 15324355, wasNull: false +ui256 type is CHAR, value: 41345135123432, wasNull: false +f32 type is FLOAT, value: -0.796896, wasNull: false +f64 type is DOUBLE, value: -0.113259, wasNull: false +b type is TINYINT, value: true, wasNull: false +Row #2 +i8 type is TINYINT, value: 127, wasNull: false +i16 type is SMALLINT, value: 32767, wasNull: false +i32 type is INT, value: 2147483647, wasNull: false +i64 type is BIGINT, value: 9223372036854775807, wasNull: false +i128 type is CHAR, value: 170141183460469231731687303715884105727, wasNull: false +i256 type is CHAR, value: 57896044618658097711785492504343953926634992332820282019728792003956564819967, wasNull: false +ui8 type is TINYINT, value: 255, wasNull: false +ui16 type is SMALLINT, value: 65535, wasNull: false +ui32 type is INT, value: 4294967295, wasNull: false +ui64 type is BIGINT, value: 18446744073709551615, wasNull: false +ui128 type is CHAR, value: 340282366920938463463374607431768211455, wasNull: false +ui256 type is CHAR, value: 115792089237316195423570985008687907853269984665640564039457584007913129639935, wasNull: false +f32 type is FLOAT, value: 1.234000, wasNull: false +f64 type is DOUBLE, value: 3.352451, wasNull: false +b type is TINYINT, value: false, wasNull: false + +### testStringTypes +Row #1 +s type is CHAR, value: 42, wasNull: false +sn type is CHAR, value: null, wasNull: true +lc type is CHAR, value: test, wasNull: false +nlc type is CHAR, value: null, wasNull: true +Row #2 +s type is CHAR, value: foo, wasNull: false +sn type is CHAR, value: bar, wasNull: false +lc type is CHAR, value: qaz, wasNull: false +nlc type is CHAR, value: qux, wasNull: false + +### testLowCardinalityAndNullableTypes +Row #1 +ilc type is INT, value: -54, wasNull: false +dlc type is DATE, value: 1970-01-01, wasNull: false +ni type is INT, value: 144, wasNull: false +Row #2 +ilc type is INT, value: 42, wasNull: false +dlc type is DATE, value: 2011-02-05, wasNull: false +ni type is INT, value: 0, wasNull: true + +### testDecimalTypes +Row #1 +d32 type is DECIMAL, value: -1.55, wasNull: false +d64 type is DECIMAL, value: 6.03, wasNull: false +d128_native type is DECIMAL, value: 5, wasNull: false +d128_text type is CHAR, value: -1224124.23423, wasNull: false +d256 type is CHAR, value: -54342.3, wasNull: false +Row #2 +d32 type is DECIMAL, value: 1234567.89, wasNull: false +d64 type is DECIMAL, value: 123456789123456.789, wasNull: false +d128_native type is DECIMAL, value: 12345678912345678912.1234567891, wasNull: false +d128_text type is CHAR, value: 1234567.8912345678912345678911234567891, wasNull: false +d256 type is CHAR, value: 12345678912345678912345678911234567891234567891234567891.12345678911234567891, wasNull: false + +### testMiscTypes +Row #1 +a type is CHAR, value: ['foo','bar'], wasNull: false +u type is CHAR, value: 5da5038d-788f-48c6-b510-babb41c538d3, wasNull: false +t type is CHAR, value: (42,'qaz'), wasNull: false +m type is CHAR, value: {'qux':144,'text':255}, wasNull: false + +### testDateTypes +Row #1 +d type is DATE, value: 1970-01-01, wasNull: false +d32 type is DATE, value: 1900-01-01, wasNull: false +dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0, wasNull: false +dt64_3 type is TIMESTAMP, value: 1900-01-01 00:00:00.001, wasNull: false +dt64_6 type is TIMESTAMP, value: 1900-01-01 00:00:00.000001, wasNull: false +dt64_9 type is TIMESTAMP, value: 1900-01-01 00:00:00.0, wasNull: false +Row #2 +d type is DATE, value: 2149-06-06, wasNull: false +d32 type is DATE, value: 2178-04-16, wasNull: false +dt type is TIMESTAMP, value: 2106-02-07 06:28:15.0, wasNull: false +dt64_3 type is TIMESTAMP, value: 2106-02-07 06:28:15.123, wasNull: false +dt64_6 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456, wasNull: false +dt64_9 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456, wasNull: false + +### testUnusualDateTime64Scales +Row #1 +dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0, wasNull: false +dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1, wasNull: false +dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.12, wasNull: false +dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.1234, wasNull: false +dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.12345, wasNull: false +dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456, wasNull: false +dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456, wasNull: false +Row #2 +dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0, wasNull: false +dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1, wasNull: false +dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.01, wasNull: false +dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.0001, wasNull: false +dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.00001, wasNull: false +dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.0, wasNull: false +dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.0, wasNull: false + +### testDateTimeTimezones +Row #1 +dt type is TIMESTAMP, value: 1970-01-01 01:00:00.0, wasNull: false +dt64_3 type is TIMESTAMP, value: 1969-12-31 16:00:00.0, wasNull: false +Row #2 +dt type is TIMESTAMP, value: 2022-09-04 20:31:05.0, wasNull: false +dt64_3 type is TIMESTAMP, value: 2022-09-04 20:31:05.022, wasNull: false + +### testSuspiciousNullableLowCardinalityTypes +Row #1 +f type is FLOAT, value: 1.0, wasNull: false +d type is DATE, value: 2022-04-15, wasNull: false +dt type is TIMESTAMP, value: 2021-06-04 13:55:11.0, wasNull: false +Row #2 +f type is FLOAT, value: 3.14, wasNull: false +d type is DATE, value: 1970-01-01, wasNull: false +dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0, wasNull: false +Row #3 +f type is FLOAT, value: 0.0, wasNull: true +d type is DATE, value: null, wasNull: true +dt type is TIMESTAMP, value: null, wasNull: true + diff --git a/tests/integration/test_mysql_protocol/java_client_binary.reference b/tests/integration/test_mysql_protocol/java_client_binary.reference deleted file mode 100644 index 763b7577487..00000000000 --- a/tests/integration/test_mysql_protocol/java_client_binary.reference +++ /dev/null @@ -1,129 +0,0 @@ -### testSimpleDataTypes -Row #1 -i8 type is TINYINT, value: -128 -i16 type is SMALLINT, value: -32768 -i32 type is INT, value: -2147483648 -i64 type is BIGINT, value: -9223372036854775808 -i128 type is CHAR, value: -170141183460469231731687303715884105728 -i256 type is CHAR, value: -57896044618658097711785492504343953926634992332820282019728792003956564819968 -ui8 type is TINYINT, value: 120 -ui16 type is SMALLINT, value: 1234 -ui32 type is INT, value: 51234 -ui64 type is BIGINT, value: 421342 -ui128 type is CHAR, value: 15324355 -ui256 type is CHAR, value: 41345135123432 -f32 type is FLOAT, value: -0.796896 -f64 type is DOUBLE, value: -0.113259 -b type is TINYINT, value: true -Row #2 -i8 type is TINYINT, value: 127 -i16 type is SMALLINT, value: 32767 -i32 type is INT, value: 2147483647 -i64 type is BIGINT, value: 9223372036854775807 -i128 type is CHAR, value: 170141183460469231731687303715884105727 -i256 type is CHAR, value: 57896044618658097711785492504343953926634992332820282019728792003956564819967 -ui8 type is TINYINT, value: 255 -ui16 type is SMALLINT, value: 65535 -ui32 type is INT, value: 4294967295 -ui64 type is BIGINT, value: 18446744073709551615 -ui128 type is CHAR, value: 340282366920938463463374607431768211455 -ui256 type is CHAR, value: 115792089237316195423570985008687907853269984665640564039457584007913129639935 -f32 type is FLOAT, value: 1.234000 -f64 type is DOUBLE, value: 3.352451 -b type is TINYINT, value: false - -### testStringTypes -Row #1 -s type is CHAR, value: 42 -sn type is CHAR, value: ᴺᵁᴸᴸ -lc type is CHAR, value: test -nlc type is CHAR, value: ᴺᵁᴸᴸ -Row #2 -s type is CHAR, value: foo -sn type is CHAR, value: bar -lc type is CHAR, value: qaz -nlc type is CHAR, value: qux - -### testLowCardinalityAndNullableTypes -Row #1 -ilc type is INT, value: -54 -dlc type is DATE, value: 1970-01-01 -ni type is INT, value: 144 -Row #2 -ilc type is INT, value: 42 -dlc type is DATE, value: 2011-02-05 -ni type is INT, value: 0 - -### testDecimalTypes -Row #1 -d32 type is DECIMAL, value: -1.55 -d64 type is DECIMAL, value: 6.03 -d128_native type is DECIMAL, value: 5 -d128_text type is CHAR, value: -1224124.23423 -d256 type is CHAR, value: -54342.3 -Row #2 -d32 type is DECIMAL, value: 1234567.89 -d64 type is DECIMAL, value: 123456789123456.789 -d128_native type is DECIMAL, value: 12345678912345678912.1234567891 -d128_text type is CHAR, value: 1234567.8912345678912345678911234567891 -d256 type is CHAR, value: 12345678912345678912345678911234567891234567891234567891.12345678911234567891 - -### testMiscTypes -Row #1 -a type is CHAR, value: ['foo','bar'] -u type is CHAR, value: 5da5038d-788f-48c6-b510-babb41c538d3 -t type is CHAR, value: (42,'qaz') -m type is CHAR, value: {'qux':144,'text':255} - -### testDateTypes -Row #1 -d type is DATE, value: 1970-01-01 -d32 type is DATE, value: 1900-01-01 -dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0 -dt64_3 type is TIMESTAMP, value: 1900-01-01 00:00:00.001 -dt64_6 type is TIMESTAMP, value: 1900-01-01 00:00:00.000001 -dt64_9 type is TIMESTAMP, value: 1900-01-01 00:00:00.0 -Row #2 -d type is DATE, value: 2149-06-06 -d32 type is DATE, value: 2178-04-16 -dt type is TIMESTAMP, value: 2106-02-07 06:28:15.0 -dt64_3 type is TIMESTAMP, value: 2106-02-07 06:28:15.123 -dt64_6 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456 -dt64_9 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456 - -### testUnusualDateTime64Scales -Row #1 -dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 -dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1 -dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.12 -dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.1234 -dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.12345 -dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456 -dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456 -Row #2 -dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 -dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1 -dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.01 -dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.0001 -dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.00001 -dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 -dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 - -### testDateTimeTimezones -Row #1 -dt type is TIMESTAMP, value: 1970-01-01 01:00:00.0 -dt64_3 type is TIMESTAMP, value: 1969-12-31 16:00:00.0 -Row #2 -dt type is TIMESTAMP, value: 2022-09-04 20:31:05.0 -dt64_3 type is TIMESTAMP, value: 2022-09-04 20:31:05.022 - -### testSuspiciousNullableLowCardinalityTypes -Row #1 -f type is FLOAT, value: 1.0 -d type is DATE, value: 2022-04-15 -dt type is TIMESTAMP, value: 2021-06-04 13:55:11.0 -Row #2 -f type is FLOAT, value: 3.14 -d type is DATE, value: 1970-01-01 -dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0 - diff --git a/tests/integration/test_mysql_protocol/java_client_test.sql b/tests/integration/test_mysql_protocol/java_client_test.sql index 1bcb29b8bee..311c315bae3 100644 --- a/tests/integration/test_mysql_protocol/java_client_test.sql +++ b/tests/integration/test_mysql_protocol/java_client_test.sql @@ -145,4 +145,5 @@ CREATE OR REPLACE TABLE suspicious_nullable_low_cardinality_types INSERT INTO suspicious_nullable_low_cardinality_types VALUES (1, '2022-04-15', '2021-06-04 13:55:11'), - (3.14, '1970-01-01', '1970-01-01 00:00:00'); + (3.14, '1970-01-01', '1970-01-01 00:00:00'), + (NULL, NULL, NULL); diff --git a/tests/integration/test_mysql_protocol/java_client_text.reference b/tests/integration/test_mysql_protocol/java_client_text.reference deleted file mode 100644 index f0dc390bea6..00000000000 --- a/tests/integration/test_mysql_protocol/java_client_text.reference +++ /dev/null @@ -1,129 +0,0 @@ -### testSimpleDataTypes -Row #1 -i8 type is TINYINT, value: -128 -i16 type is SMALLINT, value: -32768 -i32 type is INT, value: -2147483648 -i64 type is BIGINT, value: -9223372036854775808 -i128 type is CHAR, value: -170141183460469231731687303715884105728 -i256 type is CHAR, value: -57896044618658097711785492504343953926634992332820282019728792003956564819968 -ui8 type is TINYINT, value: 120 -ui16 type is SMALLINT, value: 1234 -ui32 type is INT, value: 51234 -ui64 type is BIGINT, value: 421342 -ui128 type is CHAR, value: 15324355 -ui256 type is CHAR, value: 41345135123432 -f32 type is FLOAT, value: -0.796896 -f64 type is DOUBLE, value: -0.113259 -b type is TINYINT, value: true -Row #2 -i8 type is TINYINT, value: 127 -i16 type is SMALLINT, value: 32767 -i32 type is INT, value: 2147483647 -i64 type is BIGINT, value: 9223372036854775807 -i128 type is CHAR, value: 170141183460469231731687303715884105727 -i256 type is CHAR, value: 57896044618658097711785492504343953926634992332820282019728792003956564819967 -ui8 type is TINYINT, value: 255 -ui16 type is SMALLINT, value: 65535 -ui32 type is INT, value: 4294967295 -ui64 type is BIGINT, value: 18446744073709551615 -ui128 type is CHAR, value: 340282366920938463463374607431768211455 -ui256 type is CHAR, value: 115792089237316195423570985008687907853269984665640564039457584007913129639935 -f32 type is FLOAT, value: 1.234000 -f64 type is DOUBLE, value: 3.352451 -b type is TINYINT, value: false - -### testStringTypes -Row #1 -s type is CHAR, value: 42 -sn type is CHAR, value: null -lc type is CHAR, value: test -nlc type is CHAR, value: null -Row #2 -s type is CHAR, value: foo -sn type is CHAR, value: bar -lc type is CHAR, value: qaz -nlc type is CHAR, value: qux - -### testLowCardinalityAndNullableTypes -Row #1 -ilc type is INT, value: -54 -dlc type is DATE, value: 1970-01-01 -ni type is INT, value: 144 -Row #2 -ilc type is INT, value: 42 -dlc type is DATE, value: 2011-02-05 -ni type is INT, value: 0 - -### testDecimalTypes -Row #1 -d32 type is DECIMAL, value: -1.55 -d64 type is DECIMAL, value: 6.03 -d128_native type is DECIMAL, value: 5 -d128_text type is CHAR, value: -1224124.23423 -d256 type is CHAR, value: -54342.3 -Row #2 -d32 type is DECIMAL, value: 1234567.89 -d64 type is DECIMAL, value: 123456789123456.789 -d128_native type is DECIMAL, value: 12345678912345678912.1234567891 -d128_text type is CHAR, value: 1234567.8912345678912345678911234567891 -d256 type is CHAR, value: 12345678912345678912345678911234567891234567891234567891.12345678911234567891 - -### testMiscTypes -Row #1 -a type is CHAR, value: ['foo','bar'] -u type is CHAR, value: 5da5038d-788f-48c6-b510-babb41c538d3 -t type is CHAR, value: (42,'qaz') -m type is CHAR, value: {'qux':144,'text':255} - -### testDateTypes -Row #1 -d type is DATE, value: 1970-01-01 -d32 type is DATE, value: 1900-01-01 -dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0 -dt64_3 type is TIMESTAMP, value: 1900-01-01 00:00:00.001 -dt64_6 type is TIMESTAMP, value: 1900-01-01 00:00:00.000001 -dt64_9 type is TIMESTAMP, value: 1900-01-01 00:00:00.0 -Row #2 -d type is DATE, value: 2149-06-06 -d32 type is DATE, value: 2178-04-16 -dt type is TIMESTAMP, value: 2106-02-07 06:28:15.0 -dt64_3 type is TIMESTAMP, value: 2106-02-07 06:28:15.123 -dt64_6 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456 -dt64_9 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456 - -### testUnusualDateTime64Scales -Row #1 -dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 -dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1 -dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.12 -dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.1234 -dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.12345 -dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456 -dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456 -Row #2 -dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 -dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1 -dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.01 -dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.0001 -dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.00001 -dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 -dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 - -### testDateTimeTimezones -Row #1 -dt type is TIMESTAMP, value: 1970-01-01 01:00:00.0 -dt64_3 type is TIMESTAMP, value: 1969-12-31 16:00:00.0 -Row #2 -dt type is TIMESTAMP, value: 2022-09-04 20:31:05.0 -dt64_3 type is TIMESTAMP, value: 2022-09-04 20:31:05.022 - -### testSuspiciousNullableLowCardinalityTypes -Row #1 -f type is FLOAT, value: 1.0 -d type is DATE, value: 2022-04-15 -dt type is TIMESTAMP, value: 2021-06-04 13:55:11.0 -Row #2 -f type is FLOAT, value: 3.14 -d type is DATE, value: 1970-01-01 -dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0 - diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 6a6831467f8..7a69d07633c 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -799,7 +799,7 @@ def test_java_client_text(started_cluster, java_container): demux=True, ) - with open(os.path.join(SCRIPT_DIR, "java_client_text.reference")) as fp: + with open(os.path.join(SCRIPT_DIR, "java_client.reference")) as fp: reference = fp.read() assert stdout.decode() == reference @@ -813,7 +813,7 @@ def test_java_client_binary(started_cluster, java_container): demux=True, ) - with open(os.path.join(SCRIPT_DIR, "java_client_binary.reference")) as fp: + with open(os.path.join(SCRIPT_DIR, "java_client.reference")) as fp: reference = fp.read() assert stdout.decode() == reference From e2b25aab0c4032c0b36f2124d02f9b4bfd36d242 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 15:53:38 +0000 Subject: [PATCH 391/813] Fixups, pt. II --- src/Functions/FunctionsConversion.h | 4 ++-- src/Functions/concat.cpp | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e3ec7ebd320..d7c2c70884b 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1247,9 +1247,9 @@ struct ConvertImplGenericToString FormatSettings format_settings; auto serialization = type.getDefaultSerialization(); - for (size_t i = 0; i < size; ++i) + for (size_t row = 0; row < size; ++row) { - serialization->serializeText(col_from, i, write_buffer, format_settings); + serialization->serializeText(col_from, row, write_buffer, format_settings); write_helper.rowWritten(); } diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 9aa6de5d219..f426f662868 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -132,15 +132,15 @@ private: else { /// A non-String/non-FixedString-type argument: use the default serialization to convert it to String - const auto full_column = column->convertToFullIfNeeded(); - const auto serialization = arguments[i].type->getDefaultSerialization(); + auto full_column = column->convertToFullIfNeeded(); + auto serialization = arguments[i].type->getDefaultSerialization(); auto converted_col_str = ColumnString::create(); ColumnStringHelpers::WriteHelper write_helper(*converted_col_str, column->size()); auto & write_buffer = write_helper.getWriteBuffer(); FormatSettings format_settings; - for (size_t j = 0; j < column->size(); ++j) + for (size_t row = 0; row < column->size(); ++row) { - serialization->serializeText(*full_column, j, write_buffer, format_settings); + serialization->serializeText(*full_column, row, write_buffer, format_settings); write_helper.rowWritten(); } write_helper.finalize(); @@ -150,7 +150,7 @@ private: data[i] = &converted_col_str->getChars(); offsets[i] = &converted_col_str->getOffsets(); - /// Keep the refcounted-pointer alive + /// Keep the pointer alive converted_col_ptrs[i] = std::move(converted_col_str); } } From bf1098951508dff6f1ec5e1787aaf4cae6f3de25 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 15 Nov 2023 16:54:47 +0100 Subject: [PATCH 392/813] Fix unexpected parts handling (#56693) * fix unexpected parts handling * Automatic style fix * fix --------- Co-authored-by: robot-clickhouse --- src/Storages/MergeTree/MergeTreeData.cpp | 6 ++- src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 30 ++++++++++++- src/Storages/StorageReplicatedMergeTree.h | 1 + .../test.py | 43 ++++++++++++++++++- 5 files changed, 77 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d5a82fb032c..b101d9b0c96 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1863,6 +1863,9 @@ try is_async ? "asynchronously" : "synchronously"); } + std::this_thread::sleep_for(std::chrono::milliseconds(static_cast(getSettings()->sleep_before_loading_outdated_parts_ms))); + ThreadFuzzer::maybeInjectSleep(); + /// Acquire shared lock because 'relative_data_path' is used while loading parts. TableLockHolder shared_lock; if (is_async) @@ -1875,6 +1878,7 @@ try while (true) { + ThreadFuzzer::maybeInjectSleep(); PartLoadingTree::NodePtr part; { @@ -3982,7 +3986,7 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT LOG_TEST(log, "forcefullyMovePartToDetachedAndRemoveFromMemory: removing {} from data_parts_indexes", part->getNameWithState()); data_parts_indexes.erase(it_part); - if (restore_covered && part->info.level == 0) + if (restore_covered && part->info.level == 0 && part->info.mutation == 0) { LOG_WARNING(log, "Will not recover parts covered by zero-level part {}", part->name); return; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 53876e77376..69307e74d1d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -167,6 +167,7 @@ struct Settings; M(Bool, enable_the_endpoint_id_with_zookeeper_name_prefix, false, "Enable the endpoint id with zookeeper name prefix for the replicated merge tree table", 0) \ M(UInt64, zero_copy_merge_mutation_min_parts_size_sleep_before_lock, 1ULL * 1024 * 1024 * 1024, "If zero copy replication is enabled sleep random amount of time before trying to lock depending on parts size for merge or mutation", 0) \ M(Bool, allow_floating_point_partition_key, false, "Allow floating point as partition key", 0) \ + M(UInt64, sleep_before_loading_outdated_parts_ms, 0, "For testing. Do not change it.", 0) \ \ /** Experimental/work in progress feature. Unsafe for production. */ \ M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 74821a9186c..c56887c085f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1327,6 +1327,20 @@ void StorageReplicatedMergeTree::paranoidCheckForCoveredPartsInZooKeeperOnStart( } void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) +{ + if (checkPartsImpl(skip_sanity_checks)) + return; + + /// We failed to check parts in an optimistic way, and now we need all the parts including Outdated parts to check them correctly. + waitForOutdatedPartsToBeLoaded(); + + if (checkPartsImpl(skip_sanity_checks)) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "checkPartsImpl returned false after loading Outdated parts"); +} + +bool StorageReplicatedMergeTree::checkPartsImpl(bool skip_sanity_checks) { auto zookeeper = getZooKeeper(); @@ -1422,6 +1436,18 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) continue; } + /// We have uncovered unexpected parts, and we are not sure if we can restore them or not. + /// So we have to exit, load all Outdated parts, and check again. + { + std::lock_guard lock(outdated_data_parts_mutex); + if (!outdated_data_parts_loading_finished) + { + LOG_INFO(log, "Outdated parts are not loaded yet, but we may need them to check if unexpected parts can be recovered. " + "Need retry."); + return false; + } + } + /// Part is unexpected and we don't have covering part: it's suspicious uncovered_unexpected_parts.insert(part->name); uncovered_unexpected_parts_rows += part->rows_count; @@ -1478,7 +1504,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) unexpected_parts_rows - uncovered_unexpected_parts_rows); } - if (unexpected_parts_nonnew_rows > 0 || uncovered_unexpected_parts_rows > 0) + if (unexpected_parts_nonnew_rows > 0 || uncovered_unexpected_parts_rows > 0 || !restorable_unexpected_parts.empty()) { LOG_DEBUG(log, sanity_report_debug_fmt, fmt::join(uncovered_unexpected_parts, ", "), fmt::join(restorable_unexpected_parts, ", "), fmt::join(parts_to_fetch, ", "), fmt::join(covered_unexpected_parts, ", "), fmt::join(expected_parts, ", ")); @@ -1503,6 +1529,8 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) LOG_ERROR(log, "Renaming unexpected part {} to ignored_{}{}", part->name, part->name, restore_covered ? ", restoring covered parts" : ""); forcefullyMovePartToDetachedAndRemoveFromMemory(part, "ignored", restore_covered); } + + return true; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 8c90d0e2679..b2a67572adc 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -621,6 +621,7 @@ private: * But if there are too many, throw an exception just in case - it's probably a configuration error. */ void checkParts(bool skip_sanity_checks); + bool checkPartsImpl(bool skip_sanity_checks); /// Synchronize the list of part uuids which are currently pinned. These should be sent to root query executor /// to be used for deduplication. diff --git a/tests/integration/test_max_suspicious_broken_parts_replicated/test.py b/tests/integration/test_max_suspicious_broken_parts_replicated/test.py index 6226240df56..0d009e6b132 100644 --- a/tests/integration/test_max_suspicious_broken_parts_replicated/test.py +++ b/tests/integration/test_max_suspicious_broken_parts_replicated/test.py @@ -81,12 +81,51 @@ def test_unexpected_uncommitted_merge(): detach_table("broken_table") attach_table("broken_table") - assert node.query("SELECT sum(key) FROM broken_table") == "190\n" + # it's not readonly + node.query("INSERT INTO broken_table SELECT 1") + + assert node.query("SELECT sum(key) FROM broken_table") == "191\n" assert ( node.query( "SELECT name FROM system.parts where table = 'broken_table' and active order by name" ) - == "all_0_0_0\nall_1_1_0\n" + == "all_0_0_0\nall_1_1_0\nall_2_2_0\n" + ) + + +def test_unexpected_uncommitted_mutation(): + node.query( + """ + CREATE TABLE broken_table0 (key Int) ENGINE = ReplicatedMergeTree('/tables/broken0', '1') ORDER BY tuple() + SETTINGS max_suspicious_broken_parts = 0, replicated_max_ratio_of_wrong_parts=0, old_parts_lifetime=100500, sleep_before_loading_outdated_parts_ms=10000""" + ) + + node.query("INSERT INTO broken_table0 SELECT number from numbers(10)") + + node.query( + "ALTER TABLE broken_table0 UPDATE key = key * 10 WHERE 1 SETTINGS mutations_sync=1" + ) + + assert node.query("SELECT sum(key) FROM broken_table0") == "450\n" + assert ( + node.query( + "SELECT name FROM system.parts where table = 'broken_table0' and active" + ) + == "all_0_0_0_1\n" + ) + + remove_part_from_zookeeper("/tables/broken0/replicas/1", "all_0_0_0_1") + + detach_table("broken_table0") + attach_table("broken_table0") + + node.query("INSERT INTO broken_table0 SELECT 1") + + # it may remain 45 if the nutation was finalized + sum_key = node.query("SELECT sum(key) FROM broken_table0") + assert sum_key == "46\n" or sum_key == "451\n" + assert "all_0_0_0_1" in node.query( + "SELECT name FROM system.detached_parts where table = 'broken_table0'" ) From 57aa049feff85f5482fdfd0d99f2c02a989257bc Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 15 Nov 2023 16:21:02 +0000 Subject: [PATCH 393/813] Change table name to table UUID, fix test --- src/Interpreters/InterpreterCreateQuery.cpp | 10 ++++++++-- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- ...8_replicated_merge_tree_creation.reference | 8 ++++---- .../02888_replicated_merge_tree_creation.sh | 20 +++++++++++-------- 4 files changed, 26 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index c88b1b5f968..b7738e8c459 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1442,13 +1442,19 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, "ATTACH ... FROM ... query is not supported for {} table engine, " "because such tables do not store any data on disk. Use CREATE instead.", res->getName()); - bool is_replicated_storage = typeid_cast(res.get()) != nullptr; - if (is_replicated_storage) + auto * replicated_storage = typeid_cast(res.get()); + if (replicated_storage) { const auto probability = getContext()->getSettingsRef().create_replicated_merge_tree_fault_injection_probability; std::bernoulli_distribution fault(probability); if (fault(thread_local_rng)) + { + /// We emulate the case when the exception was thrown in StorageReplicatedMergeTree constructor + if (!create.attach) + replicated_storage->dropIfEmpty(); + throw Coordination::Exception(Coordination::Error::ZCONNECTIONLOSS, "Fault injected (during table creation)"); + } } database->createTable(getContext(), create.getTable(), res, query_ptr); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d40e99aae1f..861bfa5d1cf 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -836,7 +836,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_info", toString(getStorageID().getFullTableName()) + "|" + toString(ServerUUID::get()), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_info", toString(getStorageID().uuid) + "|" + toString(ServerUUID::get()), zkutil::CreateMode::Persistent)); Coordination::Responses responses; @@ -869,7 +869,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada const String local_metadata = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); const String local_columns = metadata_snapshot->getColumns().toString(); const String local_metadata_version = toString(metadata_snapshot->getMetadataVersion()); - const String creator_info = toString(getStorageID().getFullTableName()) + "|" + toString(ServerUUID::get()); + const String creator_info = toString(getStorageID().uuid) + "|" + toString(ServerUUID::get()); /// It is possible for the replica to fail after creating ZK nodes without saving local metadata. /// Because of that we need to check whether the replica exists and is newly created. diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation.reference b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.reference index 487b1165348..98fb6a68656 100644 --- a/tests/queries/0_stateless/02888_replicated_merge_tree_creation.reference +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.reference @@ -1,4 +1,4 @@ -2 -2 -2 -2 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh index 34bddc04be1..b3ccef5de30 100755 --- a/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh @@ -1,38 +1,42 @@ #!/usr/bin/env bash -# Tags: zookeeper +# Tags: zookeeper, no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh - ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC" +UUID=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())") + #### 1 - There is only one replica ${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected" + -q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -cm1 "Fault injected" # We will see that the replica is empty and throw the same 'Fault injected' exception as before ${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected" + -q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -cm1 "Fault injected" # We will succeed ${CLICKHOUSE_CLIENT} \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" + -q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" ${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC" #### 2 - There are two replicas ${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected" + -q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -cm1 "Fault injected" ${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \ - -q "CREATE TABLE test_exception_replicated_2 (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r2') ORDER BY date" 2>&1 | grep -c "Fault injected" + -q "CREATE TABLE test_exception_replicated_2 (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r2') ORDER BY date" 2>&1 | grep -cm1 "Fault injected" # We will succeed ${CLICKHOUSE_CLIENT} \ - -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" + -q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" + +# The trash from the second replica creation will not prevent us from dropping the table fully, so we delete it separately +${CLICKHOUSE_CLIENT} -q "SYSTEM DROP REPLICA 'r2' FROM TABLE test_exception_replicated" ${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC" \ No newline at end of file From 0bca553789a7a1bac238d3cabc44f65d3a7c8254 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 15 Nov 2023 17:26:08 +0100 Subject: [PATCH 394/813] Address review comments --- src/Core/MySQL/PacketsProtocolBinary.cpp | 2 +- src/Core/MySQL/PacketsProtocolText.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index 2fe6b76d05f..6a85dc685db 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -25,7 +25,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp : row_num(row_num_), columns(columns_), data_types(data_types_), serializations(serializations_) { payload_size = 1 + null_bitmap_size; - static FormatSettings format_settings; + FormatSettings format_settings; for (size_t i = 0; i < columns.size(); ++i) { ColumnPtr col = columns[i]->convertToFullIfNeeded(); diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 82d6e27d98a..d84f6684671 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -1,5 +1,5 @@ -#include #include +#include #include #include #include From 21a283049c940201848c2d21d4eb8f991452f8e5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 14 Nov 2023 21:31:49 +0100 Subject: [PATCH 395/813] Add the test for clickhouse_backupview to CI. --- docker/test/integration/runner/Dockerfile | 1 + tests/integration/runner | 1 + .../test_backup_restore_new/test.py | 19 +++++++++++++++++++ .../test/{test.py => test_backupview.py} | 4 ++-- 4 files changed, 23 insertions(+), 2 deletions(-) rename utils/backupview/test/{test.py => test_backupview.py} (98%) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 8345e3d5791..458ca2b1da8 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -68,6 +68,7 @@ RUN python3 -m pip install --no-cache-dir \ asyncio \ avro==1.10.2 \ azure-storage-blob \ + boto3 \ cassandra-driver \ confluent-kafka==1.9.2 \ delta-spark==2.3.0 \ diff --git a/tests/integration/runner b/tests/integration/runner index 7be491a9a57..3760bf16b84 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -428,6 +428,7 @@ if __name__ == "__main__": f"--volume={args.library_bridge_binary}:/clickhouse-library-bridge " f"--volume={args.base_configs_dir}:/clickhouse-config " f"--volume={args.cases_dir}:/ClickHouse/tests/integration " + f"--volume={args.utils_dir}/backupview:/ClickHouse/utils/backupview " f"--volume={args.utils_dir}/grpc-client/pb2:/ClickHouse/utils/grpc-client/pb2 " f"--volume=/run:/run/host:ro {dockerd_internal_volume} {env_tags} {env_cleanup} " f"-e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 {use_analyzer} -e PYTHONUNBUFFERED=1 " diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 4a26a470aab..cac458f616d 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -4,11 +4,14 @@ import glob import re import random import os.path +import sys from collections import namedtuple from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry, TSV +script_dir = os.path.dirname(os.path.realpath(__file__)) + cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( "instance", @@ -1559,3 +1562,19 @@ def test_tables_dependency(): ) drop() + + +# Test for the "clickhouse_backupview" utility. + +test_backupview_dir = os.path.abspath( + os.path.join(script_dir, "../../../utils/backupview/test") +) +if test_backupview_dir not in sys.path: + sys.path.append(test_backupview_dir) +import test_backupview as test_backupview_module + + +def test_backupview(): + if instance.is_built_with_sanitizer(): + return # This test is actually for clickhouse_backupview, not for ClickHouse itself. + test_backupview_module.test_backupview_1() diff --git a/utils/backupview/test/test.py b/utils/backupview/test/test_backupview.py similarity index 98% rename from utils/backupview/test/test.py rename to utils/backupview/test/test_backupview.py index 9a1dc5cab2e..5c0d546cbfa 100755 --- a/utils/backupview/test/test.py +++ b/utils/backupview/test/test_backupview.py @@ -1,7 +1,7 @@ #!/usr/bin/env python3 # Tests for the clickhouse_backupview utility. -# Use pytest ./test.py to run. +# Use pytest ./test_backupview.py to run. import pytest @@ -36,7 +36,7 @@ def calculate_total_size(dir): # Actual tests -def test_backup_1(): +def test_backupview_1(): with open_backup(os.path.join(script_dir, "test_backup_1.zip")) as b: assert b.get_subdirs("/") == ["shards"] assert b.dir_exists("/shards") From 3e26161eaa48353fb3f1d0f6df0f2e848123f512 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 15 Nov 2023 18:33:59 +0100 Subject: [PATCH 396/813] Implement asynchronous excute of ALTER TABLE ... MOVE ... TO DISK|VOLUME --- src/Core/Settings.h | 1 + .../MergeTree/BackgroundJobsAssignee.cpp | 3 +- .../MergeTree/BackgroundJobsAssignee.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 110 +++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 5 +- src/Storages/MergeTree/MergeTreePartsMover.h | 1 + .../__init__.py | 1 + .../configs/storage_policy.xml | 38 ++++++ .../test.py | 105 +++++++++++++++++ 9 files changed, 235 insertions(+), 31 deletions(-) create mode 100644 tests/integration/test_move_partition_to_volume_async/__init__.py create mode 100644 tests/integration/test_move_partition_to_volume_async/configs/storage_policy.xml create mode 100644 tests/integration/test_move_partition_to_volume_async/test.py diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3b90a3e068b..ca3d33700c3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -140,6 +140,7 @@ class IColumn; \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ + M(Bool, alter_move_to_space_execute_async, false, "Execute ALTER TABLE MOVE ... TO [DISK|VOLUME] asynchronously", 0) \ \ M(LoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \ M(UInt64, load_balancing_first_offset, 0, "Which replica to preferably send a query when FIRST_OR_RANDOM load balancing strategy is used.", 0) \ diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp index f06ff909799..32714b3b07f 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp @@ -67,10 +67,11 @@ void BackgroundJobsAssignee::scheduleFetchTask(ExecutableTaskPtr fetch_task) } -void BackgroundJobsAssignee::scheduleMoveTask(ExecutableTaskPtr move_task) +bool BackgroundJobsAssignee::scheduleMoveTask(ExecutableTaskPtr move_task) { bool res = getContext()->getMovesExecutor()->trySchedule(move_task); res ? trigger() : postpone(); + return res; } diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.h b/src/Storages/MergeTree/BackgroundJobsAssignee.h index 27e75a79b97..f1c7eadf5f7 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.h +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.h @@ -67,7 +67,7 @@ public: bool scheduleMergeMutateTask(ExecutableTaskPtr merge_task); void scheduleFetchTask(ExecutableTaskPtr fetch_task); - void scheduleMoveTask(ExecutableTaskPtr move_task); + bool scheduleMoveTask(ExecutableTaskPtr move_task); void scheduleCommonTask(ExecutableTaskPtr common_task, bool need_trigger); /// Just call finish diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d5a82fb032c..d5a2204df8d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -192,6 +192,7 @@ namespace ErrorCodes extern const int NOT_INITIALIZED; extern const int SERIALIZATION_ERROR; extern const int TOO_MANY_MUTATIONS; + extern const int CANNOT_SCHEDULE_TASK; } static void checkSuspiciousIndices(const ASTFunction * index_function) @@ -4832,17 +4833,36 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & throw Exception(ErrorCodes::UNKNOWN_DISK, "All parts of partition '{}' are already on disk '{}'", partition_id, disk->getName()); } - MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast(disk), local_context->getReadSettings(), local_context->getWriteSettings()); - switch (moves_outcome) + if (parts_mover.moves_blocker.isCancelled()) + throw Exception(ErrorCodes::ABORTED, "Cannot move parts because moves are manually disabled"); + + auto moving_tagger = checkPartsForMove(parts, std::static_pointer_cast(disk)); + if (moving_tagger->parts_to_move.empty()) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No parts to move are found in partition {}", partition_id); + + const auto & query_settings = local_context->getSettingsRef(); + std::future moves_future = movePartsToSpace(moving_tagger, local_context->getReadSettings(), local_context->getWriteSettings(), query_settings.alter_move_to_space_execute_async); + + if (query_settings.alter_move_to_space_execute_async && moves_future.wait_for(std::chrono::seconds(0)) != std::future_status::ready) { - case MovePartsOutcome::MovesAreCancelled: - throw Exception(ErrorCodes::ABORTED, "Cannot move parts because moves are manually disabled"); - case MovePartsOutcome::NothingToMove: - throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No parts to move are found in partition {}", partition_id); - case MovePartsOutcome::MoveWasPostponedBecauseOfZeroCopy: - throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Move was not finished, because zero copy mode is enabled and someone other is moving the same parts right now"); - case MovePartsOutcome::PartsMoved: - break; + return; + } + else + { + auto moves_outcome = moves_future.get(); + switch (moves_outcome) + { + case MovePartsOutcome::MovesAreCancelled: + throw Exception(ErrorCodes::ABORTED, "Cannot move parts because moves are manually disabled"); + case MovePartsOutcome::NothingToMove: + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No parts to move are found in partition {}", partition_id); + case MovePartsOutcome::MoveWasPostponedBecauseOfZeroCopy: + throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Move was not finished, because zero copy mode is enabled and someone other is moving the same parts right now"); + case MovePartsOutcome::CannotScheduleMove: + throw Exception(ErrorCodes::CANNOT_SCHEDULE_TASK, "Cannot schedule move, no free threads, try to wait until all in-progress move finish or increase "); + case MovePartsOutcome::PartsMoved: + break; + } } } @@ -4895,17 +4915,36 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String throw Exception(ErrorCodes::UNKNOWN_DISK, "All parts of partition '{}' are already on volume '{}'", partition_id, volume->getName()); } - MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast(volume), local_context->getReadSettings(), local_context->getWriteSettings()); - switch (moves_outcome) + if (parts_mover.moves_blocker.isCancelled()) + throw Exception(ErrorCodes::ABORTED, "Cannot move parts because moves are manually disabled"); + + auto moving_tagger = checkPartsForMove(parts, std::static_pointer_cast(volume)); + if (moving_tagger->parts_to_move.empty()) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No parts to move are found in partition {}", partition_id); + + const auto & query_settings = local_context->getSettingsRef(); + std::future moves_future = movePartsToSpace(moving_tagger, local_context->getReadSettings(), local_context->getWriteSettings(), query_settings.alter_move_to_space_execute_async); + + if (query_settings.alter_move_to_space_execute_async && moves_future.wait_for(std::chrono::seconds(0)) != std::future_status::ready) { - case MovePartsOutcome::MovesAreCancelled: - throw Exception(ErrorCodes::ABORTED, "Cannot move parts because moves are manually disabled"); - case MovePartsOutcome::NothingToMove: - throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No parts to move are found in partition {}", partition_id); - case MovePartsOutcome::MoveWasPostponedBecauseOfZeroCopy: - throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Move was not finished, because zero copy mode is enabled and someone other is moving the same parts right now"); - case MovePartsOutcome::PartsMoved: - break; + return; + } + else + { + auto moves_outcome = moves_future.get(); + switch (moves_outcome) + { + case MovePartsOutcome::MovesAreCancelled: + throw Exception(ErrorCodes::ABORTED, "Cannot move parts because moves are manually disabled"); + case MovePartsOutcome::NothingToMove: + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No parts to move are found in partition {}", partition_id); + case MovePartsOutcome::MoveWasPostponedBecauseOfZeroCopy: + throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Move was not finished, because zero copy mode is enabled and someone other is moving the same parts right now"); + case MovePartsOutcome::CannotScheduleMove: + throw Exception(ErrorCodes::CANNOT_SCHEDULE_TASK, "Cannot schedule move, no free threads, try to wait until all in-progress move finish or increase "); + case MovePartsOutcome::PartsMoved: + break; + } } } @@ -7476,16 +7515,33 @@ bool MergeTreeData::areBackgroundMovesNeeded() const return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1; } -MovePartsOutcome MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const ReadSettings & read_settings, const WriteSettings & write_settings) +std::future MergeTreeData::movePartsToSpace(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool async) { - if (parts_mover.moves_blocker.isCancelled()) - return MovePartsOutcome::MovesAreCancelled; + auto finish_move_promise = std::make_shared>(); + auto finish_move_future = finish_move_promise->get_future(); - auto moving_tagger = checkPartsForMove(parts, space); - if (moving_tagger->parts_to_move.empty()) - return MovePartsOutcome::NothingToMove; + if (async) + { + bool is_scheduled = background_moves_assignee.scheduleMoveTask(std::make_shared( + [this, finish_move_promise, moving_tagger, read_settings, write_settings] () mutable + { + auto outcome = moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ true); - return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ true); + finish_move_promise->set_value(outcome); + + return outcome == MovePartsOutcome::PartsMoved; + }, moves_assignee_trigger, getStorageID())); + + if (!is_scheduled) + finish_move_promise->set_value(MovePartsOutcome::CannotScheduleMove); + } + else + { + auto outcome = moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ true); + finish_move_promise->set_value(outcome); + } + + return finish_move_future; } MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::selectPartsForMove() diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 75e334af69f..4c46980f333 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1359,8 +1359,6 @@ protected: /// method has different implementations for replicated and non replicated /// MergeTree because they store mutations in different way. virtual std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; - /// Moves part to specified space, used in ALTER ... MOVE ... queries - MovePartsOutcome movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const ReadSettings & read_settings, const WriteSettings & write_settings); struct PartBackupEntries { @@ -1513,6 +1511,9 @@ private: using CurrentlyMovingPartsTaggerPtr = std::shared_ptr; + /// Moves part to specified space, used in ALTER ... MOVE ... queries + std::future movePartsToSpace(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool async); + /// Move selected parts to corresponding disks MovePartsOutcome moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index f172dade40e..b9109e51309 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -18,6 +18,7 @@ enum class MovePartsOutcome NothingToMove, MovesAreCancelled, MoveWasPostponedBecauseOfZeroCopy, + CannotScheduleMove, }; /// Active part from storage and destination reservation where it has to be moved diff --git a/tests/integration/test_move_partition_to_volume_async/__init__.py b/tests/integration/test_move_partition_to_volume_async/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_move_partition_to_volume_async/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_move_partition_to_volume_async/configs/storage_policy.xml b/tests/integration/test_move_partition_to_volume_async/configs/storage_policy.xml new file mode 100644 index 00000000000..f5d82656210 --- /dev/null +++ b/tests/integration/test_move_partition_to_volume_async/configs/storage_policy.xml @@ -0,0 +1,38 @@ + + + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + s3 + http://resolver:8083/root/data/ + minio + minio123 + + + + + + +
+ default +
+ + broken_s3 + +
+ + 0.0 +
+
+
+ +
diff --git a/tests/integration/test_move_partition_to_volume_async/test.py b/tests/integration/test_move_partition_to_volume_async/test.py new file mode 100644 index 00000000000..c1c860b6e50 --- /dev/null +++ b/tests/integration/test_move_partition_to_volume_async/test.py @@ -0,0 +1,105 @@ +#!/usr/bin/env python3 +import logging +import time +import os + +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.mock_servers import start_s3_mock, start_mock_servers +from helpers.utility import generate_values, replace_config, SafeThread +from helpers.wait_for_helpers import wait_for_delete_inactive_parts +from helpers.wait_for_helpers import wait_for_delete_empty_parts +from helpers.wait_for_helpers import wait_for_merges + + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +@pytest.fixture(scope="module") +def init_broken_s3(cluster): + yield start_s3_mock(cluster, "broken_s3", "8083") + + +@pytest.fixture(scope="function") +def broken_s3(init_broken_s3): + init_broken_s3.reset() + yield init_broken_s3 + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + main_configs=[ + "configs/storage_policy.xml", + ], + with_minio=True, + ) + + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_async_alter_move(cluster, broken_s3): + node = cluster.instances["node"] + + node.query(""" + CREATE TABLE moving_table_async + ( + key UInt64, + data String + ) + ENGINE MergeTree() + ORDER BY tuple() + SETTINGS storage_policy = 'slow_s3' + """) + + node.query("INSERT INTO moving_table_async SELECT number, randomPrintableASCII(1000) FROM numbers(10000)") + + broken_s3.setup_slow_answers( + timeout=5, + count=1000000, + ) + + node.query("ALTER TABLE moving_table_async MOVE PARTITION tuple() TO DISK 'broken_s3'", settings={'alter_move_to_space_execute_async': True}, timeout=10) + + # not flaky, just introduce some wait + time.sleep(3) + + for i in range(100): + count = node.query("SELECT count() FROM system.moves where table = 'moving_table_async'") + if count == "1\n": + break + time.sleep(0.1) + else: + assert False, "Cannot find any moving background operation" + +def test_sync_alter_move(cluster, broken_s3): + node = cluster.instances["node"] + + node.query(""" + CREATE TABLE moving_table_sync + ( + key UInt64, + data String + ) + ENGINE MergeTree() + ORDER BY tuple() + SETTINGS storage_policy = 'slow_s3' + """) + + node.query("INSERT INTO moving_table_sync SELECT number, randomPrintableASCII(1000) FROM numbers(10000)") + + broken_s3.reset() + + node.query("ALTER TABLE moving_table_sync MOVE PARTITION tuple() TO DISK 'broken_s3'", timeout=30) + # not flaky, just introduce some wait + time.sleep(3) + + assert node.query("SELECT count() FROM system.moves where table = 'moving_table_sync'") == "0\n" + + assert node.query("SELECT disk_name FROM system.parts WHERE table = 'moving_table_sync'") == "broken_s3\n" From 28522d56b76a4191e73d9c94deafa143b1ca6c18 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 15 Nov 2023 17:45:07 +0000 Subject: [PATCH 397/813] Support Iceberg metadata files for metastore tables --- .../DataLakes/Iceberg/IcebergMetadata.cpp | 13 +++++-- .../integration/test_storage_iceberg/test.py | 36 +++++++++++++++++++ 2 files changed, 47 insertions(+), 2 deletions(-) diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index 2b8b7082515..40a81b59c36 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -304,7 +304,9 @@ MutableColumns parseAvro( /** * Each version of table metadata is stored in a `metadata` directory and - * has format: v.metadata.json, where V - metadata version. + * has one of 2 formats: + * 1) v.metadata.json, where V - metadata version. + * 2) -.metadata.json, where V - metadata version */ std::pair getMetadataFileAndVersion(const StorageS3::Configuration & configuration) { @@ -322,7 +324,14 @@ std::pair getMetadataFileAndVersion(const StorageS3::Configuratio for (const auto & path : metadata_files) { String file_name(path.begin() + path.find_last_of('/') + 1, path.end()); - String version_str(file_name.begin() + 1, file_name.begin() + file_name.find_first_of('.')); + String version_str; + /// v.metadata.json + if (file_name.starts_with('v')) + version_str = String(file_name.begin() + 1, file_name.begin() + file_name.find_first_of('.')); + /// -.metadata.json + else + version_str = String(file_name.begin(), file_name.begin() + file_name.find_first_of('-')); + if (!std::all_of(version_str.begin(), version_str.end(), isdigit)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad metadata file name: {}. Expected vN.metadata.json where N is a number", file_name); metadata_files_with_versions.emplace_back(std::stoi(version_str), path); diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 11198a7175b..f6bea26cb15 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -9,6 +9,8 @@ import json import pytest import time import glob +import uuid +import os from pyspark.sql.types import ( StructType, @@ -515,3 +517,37 @@ def test_metadata_file_selection(started_cluster, format_version): create_iceberg_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 + + +@pytest.mark.parametrize("format_version", ["1", "2"]) +def test_metadata_file_format_with_uuid(started_cluster, format_version): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + TABLE_NAME = "test_metadata_selection_" + format_version + + spark.sql( + f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" + ) + + for i in range(50): + spark.sql( + f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)" + ) + + print(os.listdir(f"/iceberg_data/default/{TABLE_NAME}/metadata/")) + for i in range(50): + os.rename( + f"/iceberg_data/default/{TABLE_NAME}/metadata/v{i + 1}.metadata.json", + f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json", + ) + print(os.listdir(f"/iceberg_data/default/{TABLE_NAME}/metadata/")) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + create_iceberg_table(instance, TABLE_NAME) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 From 808c3f36ed84b01f840eedc309511a202972c870 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 15 Nov 2023 17:47:10 +0000 Subject: [PATCH 398/813] Remove prints from test --- tests/integration/test_storage_iceberg/test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index f6bea26cb15..f5b2733eeb8 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -536,13 +536,11 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)" ) - print(os.listdir(f"/iceberg_data/default/{TABLE_NAME}/metadata/")) for i in range(50): os.rename( f"/iceberg_data/default/{TABLE_NAME}/metadata/v{i + 1}.metadata.json", f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json", ) - print(os.listdir(f"/iceberg_data/default/{TABLE_NAME}/metadata/")) files = upload_directory( minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" From af2f986215450f6a728978bb5be861f824e4ee3c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 15 Nov 2023 17:48:54 +0000 Subject: [PATCH 399/813] Automatic style fix --- .../test.py | 50 +++++++++++++++---- 1 file changed, 39 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_move_partition_to_volume_async/test.py b/tests/integration/test_move_partition_to_volume_async/test.py index c1c860b6e50..cdd2ee126c0 100644 --- a/tests/integration/test_move_partition_to_volume_async/test.py +++ b/tests/integration/test_move_partition_to_volume_async/test.py @@ -14,6 +14,7 @@ from helpers.wait_for_helpers import wait_for_merges SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + @pytest.fixture(scope="module") def init_broken_s3(cluster): yield start_s3_mock(cluster, "broken_s3", "8083") @@ -24,6 +25,7 @@ def broken_s3(init_broken_s3): init_broken_s3.reset() yield init_broken_s3 + @pytest.fixture(scope="module") def cluster(): try: @@ -47,7 +49,8 @@ def cluster(): def test_async_alter_move(cluster, broken_s3): node = cluster.instances["node"] - node.query(""" + node.query( + """ CREATE TABLE moving_table_async ( key UInt64, @@ -56,32 +59,43 @@ def test_async_alter_move(cluster, broken_s3): ENGINE MergeTree() ORDER BY tuple() SETTINGS storage_policy = 'slow_s3' - """) + """ + ) - node.query("INSERT INTO moving_table_async SELECT number, randomPrintableASCII(1000) FROM numbers(10000)") + node.query( + "INSERT INTO moving_table_async SELECT number, randomPrintableASCII(1000) FROM numbers(10000)" + ) broken_s3.setup_slow_answers( timeout=5, count=1000000, ) - node.query("ALTER TABLE moving_table_async MOVE PARTITION tuple() TO DISK 'broken_s3'", settings={'alter_move_to_space_execute_async': True}, timeout=10) + node.query( + "ALTER TABLE moving_table_async MOVE PARTITION tuple() TO DISK 'broken_s3'", + settings={"alter_move_to_space_execute_async": True}, + timeout=10, + ) # not flaky, just introduce some wait time.sleep(3) for i in range(100): - count = node.query("SELECT count() FROM system.moves where table = 'moving_table_async'") + count = node.query( + "SELECT count() FROM system.moves where table = 'moving_table_async'" + ) if count == "1\n": break time.sleep(0.1) else: assert False, "Cannot find any moving background operation" + def test_sync_alter_move(cluster, broken_s3): node = cluster.instances["node"] - node.query(""" + node.query( + """ CREATE TABLE moving_table_sync ( key UInt64, @@ -90,16 +104,30 @@ def test_sync_alter_move(cluster, broken_s3): ENGINE MergeTree() ORDER BY tuple() SETTINGS storage_policy = 'slow_s3' - """) + """ + ) - node.query("INSERT INTO moving_table_sync SELECT number, randomPrintableASCII(1000) FROM numbers(10000)") + node.query( + "INSERT INTO moving_table_sync SELECT number, randomPrintableASCII(1000) FROM numbers(10000)" + ) broken_s3.reset() - node.query("ALTER TABLE moving_table_sync MOVE PARTITION tuple() TO DISK 'broken_s3'", timeout=30) + node.query( + "ALTER TABLE moving_table_sync MOVE PARTITION tuple() TO DISK 'broken_s3'", + timeout=30, + ) # not flaky, just introduce some wait time.sleep(3) - assert node.query("SELECT count() FROM system.moves where table = 'moving_table_sync'") == "0\n" + assert ( + node.query("SELECT count() FROM system.moves where table = 'moving_table_sync'") + == "0\n" + ) - assert node.query("SELECT disk_name FROM system.parts WHERE table = 'moving_table_sync'") == "broken_s3\n" + assert ( + node.query( + "SELECT disk_name FROM system.parts WHERE table = 'moving_table_sync'" + ) + == "broken_s3\n" + ) From 6e3e6383ba0ad5b317c8189ac5a654ee5bb9057b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 15 Nov 2023 19:00:27 +0100 Subject: [PATCH 400/813] perf check 2 --- base/poco/Net/src/HTTPServerSession.cpp | 1 - base/poco/Net/src/HTTPSession.cpp | 32 +++++++++----------- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 +- src/IO/S3/PocoHTTPClient.cpp | 1 - 4 files changed, 16 insertions(+), 20 deletions(-) diff --git a/base/poco/Net/src/HTTPServerSession.cpp b/base/poco/Net/src/HTTPServerSession.cpp index f6d3c4e5b92..d4f2b24879e 100644 --- a/base/poco/Net/src/HTTPServerSession.cpp +++ b/base/poco/Net/src/HTTPServerSession.cpp @@ -26,7 +26,6 @@ HTTPServerSession::HTTPServerSession(const StreamSocket& socket, HTTPServerParam _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) { setTimeout(pParams->getTimeout()); - this->socket().setReceiveTimeout(pParams->getTimeout()); } diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 97decded282..9ebbd7d04cd 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -94,24 +94,22 @@ void HTTPSession::setTimeout(const Poco::Timespan& timeout) void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco::Timespan& sendTimeout, const Poco::Timespan& receiveTimeout) { _connectionTimeout = connectionTimeout; - _sendTimeout = sendTimeout; - _receiveTimeout = receiveTimeout; -// if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) -// { -// _sendTimeout = sendTimeout; -// -// if (connected()) -// _socket.setSendTimeout(_sendTimeout); -// } -// -// if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) -// { -// _receiveTimeout = receiveTimeout; -// -// if (connected()) -// _socket.setReceiveTimeout(_receiveTimeout); -// } + if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) + { + _sendTimeout = sendTimeout; + + if (connected()) + _socket.setSendTimeout(_sendTimeout); + } + + if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) + { + _receiveTimeout = receiveTimeout; + + if (connected()) + _socket.setReceiveTimeout(_receiveTimeout); + } } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index b0384daab2d..0232a6eb070 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -67,7 +67,7 @@ std::unique_ptr getClient( config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); client_configuration.wait_on_pool_size_limit = false; - client_configuration.s3_use_adaptive_timeouts = config.getUInt( + client_configuration.s3_use_adaptive_timeouts = config.getBool( config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); /* diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 904e2324145..f681362e607 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include From f3f839205197a73cf3c4b40dd8d67077839701ba Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 15 Nov 2023 17:37:00 +0100 Subject: [PATCH 401/813] upgrade replication protocol --- src/Storages/MergeTree/DataPartsExchange.cpp | 83 ++++++++++++++----- src/Storages/MergeTree/MergeTreeSettings.h | 5 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 8 ++ tests/clickhouse-test | 3 + ...plication_protocol_wait_for_part.reference | 1 + ...916_replication_protocol_wait_for_part.sql | 23 +++++ 6 files changed, 101 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02916_replication_protocol_wait_for_part.reference create mode 100644 tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 4545b2b98ae..7fd6f59ed69 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -65,8 +65,7 @@ constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_UUID = 5; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_ZERO_COPY = 6; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION = 7; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION = 8; -// Reserved for ALTER PRIMARY KEY -// constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PRIMARY_KEY = 9; +constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE = 9; std::string getEndpointId(const std::string & node_id) { @@ -122,7 +121,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write MergeTreePartInfo::fromPartName(part_name, data.format_version); /// We pretend to work as older server version, to be sure that client will correctly process our version - response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION))}); + response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE))}); LOG_TRACE(log, "Sending part {}", part_name); @@ -140,6 +139,29 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write { part = findPart(part_name); + /// Ephemeral zero-copy lock may be lost for PreActive parts + /// do not expose PreActive parts + if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE) + { + bool part_is_ready = part->getState() != MergeTreeDataPartState::PreActive; + writeBinary(part_is_ready, out); + + if (!part_is_ready) + { + LOG_TRACE(log, "Part {} is in PreActive state, reply to the client that part is not ready yet", part_name); + return; + } + } + else + { + bool zero_copy_enabled = data.getSettings()->allow_remote_fs_zero_copy_replication; + if (part->getState() == MergeTreeDataPartState::PreActive && zero_copy_enabled) + { + /// report error, client will try again later, error message would be printed + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in table", part_name); + } + } + CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedSend}; if (part->getDataPartStorage().isStoredOnRemoteDisk()) @@ -357,12 +379,8 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) /// determine the local state of the part, so queries for the parts in these states are completely normal. MergeTreeData::DataPartPtr part; - /// Ephemeral zero-copy lock may be lost for PreActive parts - bool zero_copy_enabled = data.getSettings()->allow_remote_fs_zero_copy_replication; - if (zero_copy_enabled) - part = data.getPartIfExists(name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); - else - part = data.getPartIfExists(name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + part = data.getPartIfExists(name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (part) return part; @@ -424,7 +442,7 @@ std::pair Fetcher::fetchSelected { {"endpoint", endpoint_id}, {"part", part_name}, - {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION)}, + {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE)}, {"compress", "false"} }); @@ -482,17 +500,42 @@ std::pair Fetcher::fetchSelected creds.setPassword(password); } - std::unique_ptr in = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - nullptr, - timeouts, - creds, - DBMS_DEFAULT_BUFFER_SIZE, - 0, /* no redirects */ - static_cast(data_settings->replicated_max_parallel_fetches_for_host)); + std::unique_ptr in; + int server_protocol_version = 0; + bool part_is_ready = true; - int server_protocol_version = parse(in->getResponseCookie("server_protocol_version", "0")); + static const UInt32 part_not_ready_attempts = 5; + static const UInt32 wait_sleep_time_ms = 100; + + for (UInt32 attempt = 1; attempt <= part_not_ready_attempts; ++attempt) + { + in = std::make_unique( + uri, + Poco::Net::HTTPRequest::HTTP_POST, + nullptr, + timeouts, + creds, + DBMS_DEFAULT_BUFFER_SIZE, + 0, /* no redirects */ + static_cast(data_settings->replicated_max_parallel_fetches_for_host)); + + server_protocol_version = parse(in->getResponseCookie("server_protocol_version", "0")); + + if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE) + readBinary(part_is_ready, *in); + + if (part_is_ready) + break; + + sleepForMilliseconds(wait_sleep_time_ms); + + if (blocker.isCancelled()) + throw Exception(ErrorCodes::ABORTED, "Fetching of part was cancelled"); + } + + if (!part_is_ready) + throw Exception(ErrorCodes::ABORTED, "Part {} is still not ready in host {} after {} attempts, try another host", + part_name, host, part_not_ready_attempts); String remote_fs_metadata = parse(in->getResponseCookie("remote_fs_metadata", "")); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 53876e77376..15c54ee3791 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -83,7 +83,8 @@ struct Settings; M(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ M(UInt64, min_delay_to_insert_ms, 10, "Min delay of inserting data into MergeTree table in milliseconds, if there are a lot of unmerged parts in single partition.", 0) \ M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ - 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, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background.", 0) \ + M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 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) \ @@ -121,7 +122,7 @@ struct Settings; M(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(Milliseconds, wait_for_unique_parts_send_before_shutdown_ms, 0, "Before shutdown table will wait for required amount time for unique parts (exist only on current replica) to be fetched by other replicas (0 means disabled).", 0) \ M(Float, fault_probability_before_part_commit, 0, "For testing. Do not change it.", 0) \ - M(Float, fault_probability_after_part_commit, 0, "For testing. Do not change it.", 0) \ + M(Float, fault_probability_after_part_commit, 0, "For testing. Do not change it.", 0) \ \ /** Check delay of replicas settings. */ \ M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 7de5d46c66b..37f808824b5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -924,6 +924,14 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: Coordination::Error multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT if (multi_code == Coordination::Error::ZOK) { + auto sleep_before_commit_local_part_in_replicated_table_ms = storage.getSettings()->sleep_before_commit_local_part_in_replicated_table_ms; + if (sleep_before_commit_local_part_in_replicated_table_ms.totalMilliseconds()) + { + LOG_INFO(log, "committing part {}, triggered sleep_before_commit_local_part_in_replicated_table_ms {}", + part->name, sleep_before_commit_local_part_in_replicated_table_ms.totalMilliseconds()); + sleepForMilliseconds(sleep_before_commit_local_part_in_replicated_table_ms.totalMilliseconds()); + } + part->new_part_was_committed_to_zookeeper_after_rename_on_disk = true; transaction.commit(); storage.merge_selecting_task->schedule(); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index cab7d7e79ff..048f848ff27 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -673,6 +673,9 @@ class MergeTreeSettingsRandomizer: "primary_key_compress_block_size": lambda: random.randint(8000, 100000), "replace_long_file_name_to_hash": lambda: random.randint(0, 1), "max_file_name_length": threshold_generator(0.3, 0.3, 0, 128), + "sleep_before_commit_local_part_in_replicated_table_ms": threshold_generator( + 0.3, 0.3, 0, 250 + ), } @staticmethod diff --git a/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.reference b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql new file mode 100644 index 00000000000..ed9cfd00b45 --- /dev/null +++ b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql @@ -0,0 +1,23 @@ +-- Tags: no-replicated-database, no-fasttest +-- Tag no-replicated-database: different number of replicas + +create table tableIn (n int) + engine=ReplicatedMergeTree('/test/02916/{database}/table', '1') + order by tuple() + settings + storage_policy='s3_cache', + allow_remote_fs_zero_copy_replication=1, + sleep_before_commit_local_part_in_replicated_table_ms=50000; +create table tableOut (n int) + engine=ReplicatedMergeTree('/test/02916/{database}/table', '2') + order by tuple() + settings + storage_policy='s3_cache', + allow_remote_fs_zero_copy_replication=1; + +SET send_logs_level = 'error'; + +insert into tableIn values(1); +insert into tableIn values(2); +system sync replica tableOut; +select count() from tableOut; From d6c58023d50bbe7a77a7497ddc9b1031186bb01c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 15 Nov 2023 18:07:24 +0000 Subject: [PATCH 402/813] Better except for SSL connection failure --- src/Server/TCPHandler.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1da9806b4f5..c0941603d78 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -104,6 +104,7 @@ namespace DB::ErrorCodes extern const int TIMEOUT_EXCEEDED; extern const int SUPPORT_IS_DISABLED; extern const int UNSUPPORTED_METHOD; + extern const int WRONG_PASSWORD; } namespace @@ -1431,8 +1432,11 @@ void TCPHandler::receiveHello() getClientAddress(client_info)); return; } - catch (...) + catch (const Exception & e) { + if (e.code() != DB::ErrorCodes::WRONG_PASSWORD) + throw; + tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication"); } } From 67796b18bbd335a2b3f58968ea664ae7d2883653 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 15 Nov 2023 19:36:23 +0100 Subject: [PATCH 403/813] Add missing test config files --- .../configs/000-config_with_env_subst.xml | 17 +++++++++++++++++ .../configs/010-env_subst_override.xml | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) create mode 100644 tests/integration/test_config_substitutions/configs/000-config_with_env_subst.xml create mode 100644 tests/integration/test_config_substitutions/configs/010-env_subst_override.xml diff --git a/tests/integration/test_config_substitutions/configs/000-config_with_env_subst.xml b/tests/integration/test_config_substitutions/configs/000-config_with_env_subst.xml new file mode 100644 index 00000000000..ffa26488874 --- /dev/null +++ b/tests/integration/test_config_substitutions/configs/000-config_with_env_subst.xml @@ -0,0 +1,17 @@ + + + + + + + + + + default + default + + + + + + diff --git a/tests/integration/test_config_substitutions/configs/010-env_subst_override.xml b/tests/integration/test_config_substitutions/configs/010-env_subst_override.xml new file mode 100644 index 00000000000..2d768b05172 --- /dev/null +++ b/tests/integration/test_config_substitutions/configs/010-env_subst_override.xml @@ -0,0 +1,17 @@ + + + + 424242 + + + + + + default + default + + + + + + From d862dfdf9c753e17896f1c3a9d5cb01e71a5cee3 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Wed, 15 Nov 2023 18:38:23 +0000 Subject: [PATCH 404/813] fix comments Signed-off-by: Jianfei Hu --- src/IO/S3/Credentials.cpp | 53 +++++++++++---------------------------- src/IO/S3/Credentials.h | 11 ++++---- 2 files changed, 21 insertions(+), 43 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 7d6ed094486..bc336634114 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,9 +1,4 @@ -#include -#include #include -#include -#include -#include "Common/Exception.h" #if USE_AWS_S3 @@ -21,22 +16,24 @@ # 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 @@ -65,7 +62,7 @@ bool areCredentialsEmptyOrExpired(const Aws::Auth::AWSCredentials & credentials, } const char SSO_CREDENTIALS_PROVIDER_LOG_TAG[] = "SSOCredentialsProvider"; -const int AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS = 3; +constexpr int AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS = 3; } @@ -275,11 +272,11 @@ String getGCPAvailabilityZoneOrException() boost::split(zone_info, response_data, boost::is_any_of("/")); /// We expect GCP returns a string as "projects/123456789/zones/us-central1a". if (zone_info.size() != 4) - throw DB::Exception(ErrorCodes::GCP_ERROR, "Invalid format of GCP zone information, expect projects//zones/, got {}", response_data); + throw DB::Exception(ErrorCodes::GCP_ERROR, "Invalid format of GCP zone information, expect projects//zones/"); return zone_info[3]; } -String getRunningAvailabilityZoneImpl() +String getRunningAvailabilityZone() { LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); try @@ -302,26 +299,6 @@ String getRunningAvailabilityZoneImpl() } } -std::variant getRunningAvailabilityZoneImplOrException() -{ - try - { - return getRunningAvailabilityZoneImpl(); - } - catch (...) - { - return std::current_exception(); - } -} - -String getRunningAvailabilityZone() -{ - static auto az_or_exception = getRunningAvailabilityZoneImplOrException(); - if (const auto * az = std::get_if(&az_or_exception)) - return *az; - else - std::rethrow_exception(std::get(az_or_exception)); -} AWSEC2InstanceProfileConfigLoader::AWSEC2InstanceProfileConfigLoader(const std::shared_ptr & client_, bool use_secure_pull_) : client(client_) diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index a978679348f..b1666e13757 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -1,12 +1,13 @@ #pragma once -#include -#include -#include #include "config.h" #if USE_AWS_S3 +# include +# include +# include + # include # include # include @@ -22,7 +23,7 @@ namespace DB::S3 inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; /// In GCP metadata service can be accessed via DNS regardless of IPv4 or IPv6. -static constexpr char GCP_METADATA_SERVICE_ENDPOINT[] = "http://metadata.google.internal"; +static inline constexpr char GCP_METADATA_SERVICE_ENDPOINT[] = "http://metadata.google.internal"; /// getRunningAvailabilityZone returns the availability zone of the underlying compute resources where the current process runs. String getRunningAvailabilityZone(); @@ -59,7 +60,7 @@ public: virtual Aws::String getCurrentRegion() const; - friend String getRunningAvailabilityZoneImpl(); + friend String getRunningAvailabilityZone(); private: std::pair getEC2MetadataToken(const std::string & user_agent_string) const; From d0398e3c1d1f55281e65633a4947c807d3c0c022 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Wed, 15 Nov 2023 18:47:28 +0000 Subject: [PATCH 405/813] remove variant header Signed-off-by: Jianfei Hu --- src/IO/S3/Credentials.cpp | 1 - src/IO/S3/Credentials.h | 1 - 2 files changed, 2 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index bc336634114..9ab21465593 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -21,7 +21,6 @@ # include # include -# include # include # include diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index b1666e13757..1f35443adf4 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -6,7 +6,6 @@ # include # include -# include # include # include From fc6a25fad562ccfcec76e83aa040b3d3ba2b9039 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 15 Nov 2023 21:54:19 +0300 Subject: [PATCH 406/813] Update src/Compression/CompressionCodecFPC.cpp Co-authored-by: vdimir --- src/Compression/CompressionCodecFPC.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecFPC.cpp b/src/Compression/CompressionCodecFPC.cpp index ec8efa0fb38..7246082caea 100644 --- a/src/Compression/CompressionCodecFPC.cpp +++ b/src/Compression/CompressionCodecFPC.cpp @@ -440,7 +440,7 @@ private: first = decompressValue(value1, is_dfcm_predictor1); second = decompressValue(value2, is_dfcm_predictor2); - return 1 + tail_size1 + tail_size2; + return expected_size; } static void* valueTail(TUInt& value, UInt32 compressed_size) From 83acec9511e082bd872f2968ec90eb4579abaf57 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 15 Nov 2023 18:58:14 +0000 Subject: [PATCH 407/813] Fix max_threads = 1 case --- src/Processors/QueryPlan/SortingStep.cpp | 2 +- ...568_window_functions_distributed.reference | 19 +++++++++++++++++++ .../01568_window_functions_distributed.sql | 2 ++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index e94e818a4aa..e7149aa8848 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -336,7 +336,7 @@ void SortingStep::fullSort( fullSortStreams(pipeline, sort_settings, result_sort_desc, limit_, skip_partial_sort); /// If there are several streams, then we merge them into one - if (pipeline.getNumStreams() > 1 && partition_by_description.empty()) + if ((pipeline.getNumStreams() > 1 && partition_by_description.empty()) || pipeline.getNumThreads() == 1) { auto transform = std::make_shared( pipeline.getHeader(), diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.reference b/tests/queries/0_stateless/01568_window_functions_distributed.reference index f0fe6884201..29ff2e7133c 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.reference +++ b/tests/queries/0_stateless/01568_window_functions_distributed.reference @@ -51,6 +51,25 @@ select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1, 42 8 42 8 42 8 +select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y SETTINGS max_threads = 1; +6 2 +6 2 +6 2 +6 2 +6 2 +6 2 +24 5 +24 5 +24 5 +24 5 +24 5 +24 5 +42 8 +42 8 +42 8 +42 8 +42 8 +42 8 select distinct sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; 6 2 24 5 diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.sql b/tests/queries/0_stateless/01568_window_functions_distributed.sql index dba450716a0..ecce7b412ba 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.sql +++ b/tests/queries/0_stateless/01568_window_functions_distributed.sql @@ -19,6 +19,8 @@ select sum(number) over w, max(number) over w from t_01568 window w as (partitio select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; +select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y SETTINGS max_threads = 1; + select distinct sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; -- window functions + aggregation w/shards From e37fdaad3330356dc96d68c9ebead9e71ab3ed7a Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 15 Nov 2023 19:17:02 +0000 Subject: [PATCH 408/813] fix issues with history and errors --- programs/server/dashboard.html | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index f3096d72e14..72df69ed809 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -1081,7 +1081,6 @@ function showAuthError(message) { charts.style.opacity = '0'; document.getElementById('add').style.display = 'none'; document.getElementById('edit').style.display = 'none'; - document.getElementById('search-span').style.display = 'none'; const authError = document.getElementById('auth-error'); authError.textContent = message; @@ -1182,12 +1181,13 @@ function enableButtons() { async function reloadAll(do_search) { disableButtons(); try { - if (do_search) { - await searchQueries(document.getElementById('search-query').value); - } updateParams(); - await drawAll(); + search_query = document.getElementById('search-query').value; saveState(); + if (do_search) { + await searchQueries(); + } + await drawAll(); } catch (e) { showAuthError(e.toString()); } @@ -1207,9 +1207,7 @@ function saveState() { window.location.pathname + (window.location.search || '') + '#' + btoa(JSON.stringify(state))); } -async function searchQueries(query) { - search_query = query; - +async function searchQueries() { let {data, error} = await doFetch(search_query); if (error) { throw new Error(error); @@ -1270,10 +1268,11 @@ if (window.location.hash) { async function start() { try { if (queries.length == 0) { - await searchQueries(search_query); + await searchQueries(); } else { regenerate(); } + saveState(); let new_theme = window.localStorage.getItem('theme'); if (new_theme && new_theme != theme) { setTheme(new_theme); From 592c653caca58a95094505ee8015505980a5d01f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 15 Nov 2023 20:26:07 +0000 Subject: [PATCH 409/813] Fixup --- src/Processors/QueryPlan/SortingStep.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index e7149aa8848..641b9036d4c 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -336,7 +336,7 @@ void SortingStep::fullSort( fullSortStreams(pipeline, sort_settings, result_sort_desc, limit_, skip_partial_sort); /// If there are several streams, then we merge them into one - if ((pipeline.getNumStreams() > 1 && partition_by_description.empty()) || pipeline.getNumThreads() == 1) + if (pipeline.getNumStreams() > 1 && (partition_by_description.empty() || pipeline.getNumThreads() == 1)) { auto transform = std::make_shared( pipeline.getHeader(), From ea92dbb1c74c700b4df4172999d6ca504ff593bf Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Wed, 15 Nov 2023 19:18:38 +0000 Subject: [PATCH 410/813] fix build for non USE_S3 case Signed-off-by: Jianfei Hu --- src/IO/S3/Credentials.cpp | 6 ++++-- src/IO/S3/Credentials.h | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 9ab21465593..e25f4551723 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -784,15 +784,17 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( #else +# include + namespace DB { namespace S3 { -String getRunningAvailabilityZone() +std::string getRunningAvailabilityZone() { - throw Poco::Exception("Does not support availability zone detection for non-cloud environment"); + throw std::runtime_error("Does not support availability zone detection for non-cloud environment"); } } diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 1f35443adf4..d8d103a847a 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -25,7 +25,7 @@ inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; static inline constexpr char GCP_METADATA_SERVICE_ENDPOINT[] = "http://metadata.google.internal"; /// getRunningAvailabilityZone returns the availability zone of the underlying compute resources where the current process runs. -String getRunningAvailabilityZone(); +std::string getRunningAvailabilityZone(); class AWSEC2MetadataClient : public Aws::Internal::AWSHttpResourceClient { @@ -189,12 +189,14 @@ public: #else +# include + namespace DB { namespace S3 { -String getRunningAvailabilityZone(); +std::string getRunningAvailabilityZone(); } } From 1e464609107105bce48ef990eaacd1a41ddc43eb Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 16 Nov 2023 05:27:05 +0800 Subject: [PATCH 411/813] Fix nullable primary key in final (#56452) --- src/Processors/QueryPlan/PartsSplitter.cpp | 36 ++++++++++--------- ...02867_nullable_primary_key_final.reference | 1 + .../02867_nullable_primary_key_final.sql | 8 +++++ 3 files changed, 28 insertions(+), 17 deletions(-) diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 90f6f49826c..8bf877cf8b9 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -241,10 +241,6 @@ ASTs buildFilters(const KeyDescription & primary_key, const std::vector ASTs values_ast; for (size_t i = 0; i < values.size(); ++i) { - /// NULL is treated as a terminator for > comparison. - if (values[i].isNull()) - break; - const auto & type = primary_key.data_types.at(i); // PK may contain functions of the table columns, so we need the actual PK AST with all expressions it contains. @@ -255,25 +251,31 @@ ASTs buildFilters(const KeyDescription & primary_key, const std::vector if (type->isNullable()) { pks_ast.push_back(makeASTFunction("isNull", pk_ast)); - values_ast.push_back(std::make_shared(0)); + values_ast.push_back(std::make_shared(values[i].isNull() ? 1 : 0)); pk_ast = makeASTFunction("assumeNotNull", pk_ast); } - ASTPtr component_ast = std::make_shared(values[i]); - auto decayed_type = removeNullable(removeLowCardinality(primary_key.data_types.at(i))); - // Values of some types (e.g. Date, DateTime) are stored in columns as numbers and we get them as just numbers from the index. - // So we need an explicit Cast for them. - if (isColumnedAsNumber(decayed_type->getTypeId()) && !isNumber(decayed_type->getTypeId())) - component_ast = makeASTFunction("cast", std::move(component_ast), std::make_shared(decayed_type->getName())); + pks_ast.push_back(pk_ast); - pks_ast.push_back(std::move(pk_ast)); - values_ast.push_back(std::move(component_ast)); + // If value is null, the comparison is already complete by looking at the null mask column. + // Here we put the pk_ast as a placeholder: (pk_null_mask, pk_ast_not_null) > (value_is_null?, pk_ast_not_null). + if (values[i].isNull()) + { + values_ast.push_back(pk_ast); + } + else + { + ASTPtr component_ast = std::make_shared(values[i]); + auto decayed_type = removeNullable(removeLowCardinality(primary_key.data_types.at(i))); + // Values of some types (e.g. Date, DateTime) are stored in columns as numbers and we get them as just numbers from the index. + // So we need an explicit Cast for them. + if (isColumnedAsNumber(decayed_type->getTypeId()) && !isNumber(decayed_type->getTypeId())) + component_ast = makeASTFunction("cast", std::move(component_ast), std::make_shared(decayed_type->getName())); + + values_ast.push_back(std::move(component_ast)); + } } - /// It indicates (pk1, ...) > (NULL, ...), which is an always false predicate. - if (pks_ast.empty()) - return std::make_shared(0u); - ASTPtr pk_columns_as_tuple = makeASTFunction("tuple", pks_ast); ASTPtr values_as_tuple = makeASTFunction("tuple", values_ast); diff --git a/tests/queries/0_stateless/02867_nullable_primary_key_final.reference b/tests/queries/0_stateless/02867_nullable_primary_key_final.reference index 2e55b120f6e..035932e1bb4 100644 --- a/tests/queries/0_stateless/02867_nullable_primary_key_final.reference +++ b/tests/queries/0_stateless/02867_nullable_primary_key_final.reference @@ -1,2 +1,3 @@ 2023-09-01 2500000000 2023-09-01 166167 +10 diff --git a/tests/queries/0_stateless/02867_nullable_primary_key_final.sql b/tests/queries/0_stateless/02867_nullable_primary_key_final.sql index 05677789459..773a6d35b8d 100644 --- a/tests/queries/0_stateless/02867_nullable_primary_key_final.sql +++ b/tests/queries/0_stateless/02867_nullable_primary_key_final.sql @@ -55,3 +55,11 @@ WHERE f2 = 'x' GROUP BY 1; DROP TABLE t; + +CREATE TABLE t (o Nullable(String), p Nullable(String)) ENGINE = ReplacingMergeTree ORDER BY (p, o) SETTINGS allow_nullable_key = 1, index_granularity = 2; + +INSERT INTO t SELECT number, NULL FROM numbers(10); + +SELECT count() FROM t FINAL; + +DROP TABLE t; From 2508e91856ab9d5745415e30ded0c9a62f2c6c63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 15 Nov 2023 23:10:25 +0100 Subject: [PATCH 412/813] Fix TSAN race in transform --- src/Functions/transform.cpp | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index b7582b37017..f1d2b60f1f4 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -154,7 +154,7 @@ namespace ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - initialize(arguments, result_type); + std::call_once(once, [&] { initialize(arguments, result_type); }); const auto * in = arguments[0].column.get(); @@ -672,11 +672,9 @@ namespace ColumnPtr default_column; bool is_empty = false; - bool initialized = false; - - std::mutex mutex; }; + mutable std::once_flag once; mutable Cache cache; @@ -706,10 +704,6 @@ namespace /// Can be called from different threads. It works only on the first call. void initialize(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const { - std::lock_guard lock(cache.mutex); - if (cache.initialized) - return; - const DataTypePtr & from_type = arguments[0].type; if (from_type->onlyNull()) @@ -824,8 +818,6 @@ namespace } } } - - cache.initialized = true; } }; From 6c013f835e8c2643c44e1d8fd9843693f5a02090 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 15 Nov 2023 23:12:51 +0100 Subject: [PATCH 413/813] Add reproducer --- tests/queries/0_stateless/02917_transform_tsan.reference | 4 ++++ tests/queries/0_stateless/02917_transform_tsan.sql | 2 ++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02917_transform_tsan.reference create mode 100644 tests/queries/0_stateless/02917_transform_tsan.sql diff --git a/tests/queries/0_stateless/02917_transform_tsan.reference b/tests/queries/0_stateless/02917_transform_tsan.reference new file mode 100644 index 00000000000..896ae5f0269 --- /dev/null +++ b/tests/queries/0_stateless/02917_transform_tsan.reference @@ -0,0 +1,4 @@ +\N +\N +\N +\N diff --git a/tests/queries/0_stateless/02917_transform_tsan.sql b/tests/queries/0_stateless/02917_transform_tsan.sql new file mode 100644 index 00000000000..dac79f83d6a --- /dev/null +++ b/tests/queries/0_stateless/02917_transform_tsan.sql @@ -0,0 +1,2 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/56815 +SELECT transform(arrayJoin([NULL, NULL]), [NULL, NULL], [NULL]) GROUP BY GROUPING SETS (('0.1'), ('-0.2147483647')); From 4ff82528f1824510c9dabe070f51b93ea0297218 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 16 Nov 2023 00:01:36 +0100 Subject: [PATCH 414/813] small fixes after review --- src/Common/parseGlobs.cpp | 27 +++++++++++++++------------ src/Storages/StorageFile.cpp | 8 ++++---- 2 files changed, 19 insertions(+), 16 deletions(-) diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index e19393478b4..834111ec754 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -126,31 +126,32 @@ namespace { void expandSelectorGlobImpl(const std::string & path, std::vector & for_match_paths_expanded) { - /// regexp for {expr1,expr2,....}; + /// regexp for {expr1,expr2,....} (a selector glob); /// expr1, expr2,... cannot contain any of these: '{', '}', ',' static const re2::RE2 selector_regex(R"({([^{}*,]+,[^{}*]*[^{}*,])})"); std::string_view path_view(path); std::string_view matched; + // No (more) selector globs found, quit if (!RE2::FindAndConsume(&path_view, selector_regex, &matched)) { for_match_paths_expanded.push_back(path); return; } - Strings expanded_paths; - std::vector anchor_positions; - bool opened = false, closed = false; + bool opened = false; + bool closed = false; - for (std::string::const_iterator it = path.begin(); it != path.end(); it++) + // Looking for first occurrence of {} selector: write down positions of {, } and all intermediate commas + for (auto it = path.begin(); it != path.end(); ++it) { if (*it == '{') { if (opened) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected '{{' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); + "Unexpected '{{' found in path '{}' at position {}.", path, it - path.begin()); anchor_positions.push_back(std::distance(path.begin(), it)); opened = true; } @@ -158,7 +159,7 @@ void expandSelectorGlobImpl(const std::string & path, std::vector & { if (!opened) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected '}}' found in path '{}' at position {}.", path, std::distance(path.begin(), it)); + "Unexpected '}}' found in path '{}' at position {}.", path, it - path.begin()); anchor_positions.push_back(std::distance(path.begin(), it)); closed = true; break; @@ -175,13 +176,15 @@ void expandSelectorGlobImpl(const std::string & path, std::vector & throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid {{}} glob in path {}.", path); - std::string common_prefix = path.substr(0, anchor_positions[0]); - std::string common_suffix = path.substr(anchor_positions[anchor_positions.size()-1] + 1); + // generate result: prefix/{a,b,c}/suffix -> [prefix/a/suffix, prefix/b/suffix, prefix/c/suffix] + std::string common_prefix = path.substr(0, anchor_positions.front()); + std::string common_suffix = path.substr(anchor_positions.back() + 1); for (size_t i = 1; i < anchor_positions.size(); ++i) { - std::string expanded_matcher = common_prefix - + path.substr(anchor_positions[i-1] + 1, (anchor_positions[i] - anchor_positions[i-1] - 1)) - + common_suffix; + std::string current_selection = + path.substr(anchor_positions[i-1] + 1, (anchor_positions[i] - anchor_positions[i-1] - 1)); + + std::string expanded_matcher = common_prefix + current_selection + common_suffix; expandSelectorGlobImpl(expanded_matcher, for_match_paths_expanded); } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 2dffdfa8be9..dd527a50794 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -114,7 +114,7 @@ void listFilesWithRegexpMatchingImpl( const std::string & for_match, size_t & total_bytes_to_read, std::vector & result, - bool recursive = false) + bool recursive) { const size_t first_glob_pos = for_match.find_first_of("*?{"); @@ -128,7 +128,7 @@ void listFilesWithRegexpMatchingImpl( catch (const std::exception &) // NOLINT { /// There is no such file, but we just ignore this. -// throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", for_match); + /// throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", for_match); } return; } @@ -185,7 +185,7 @@ void listFilesWithRegexpMatchingImpl( else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash_after_glob_pos), - total_bytes_to_read, result); + total_bytes_to_read, result, false); } } } @@ -199,7 +199,7 @@ std::vector listFilesWithRegexpMatching( Strings for_match_paths_expanded = expandSelectionGlob(for_match); for (const auto & for_match_expanded : for_match_paths_expanded) - listFilesWithRegexpMatchingImpl("/", for_match_expanded, total_bytes_to_read, result); + listFilesWithRegexpMatchingImpl("/", for_match_expanded, total_bytes_to_read, result, false); return result; } From 3bbb329dd0f091b49a6dd771820110cde0e3a052 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Nov 2023 00:13:05 +0100 Subject: [PATCH 415/813] Fix tests --- src/Server/TCPHandler.cpp | 15 +++++++++++++-- tests/queries/0_stateless/01119_session_log.sql | 2 +- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index e7c40092077..884fc45f763 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -587,8 +587,19 @@ void TCPHandler::runImpl() } catch (const Exception & e) { - /// Authentication failure with interserver secret. - if (e.code() == ErrorCodes::AUTHENTICATION_FAILED) + /// Authentication failure with interserver secret + /// - early exit without trying to send the exception to the client. + /// Because the server should not try to skip (parse, decompress) the remaining packets sent by the client, + /// as it will lead to additional work and unneeded exposure to unauthenticated connections. + + /// Note that the exception AUTHENTICATION_FAILED can be here in two cases: + /// 1. The authentication in receiveHello is skipped with "interserver secret", + /// postponed to receiving the query, and then failed. + /// 2. Receiving exception from a query using a table function to authenticate with another server. + /// In this case, the user is already authenticated with this server, + /// is_interserver_mode is false, and we can send the exception to the client normally. + + if (is_interserver_mode && e.code() == ErrorCodes::AUTHENTICATION_FAILED) throw; state.io.onException(); diff --git a/tests/queries/0_stateless/01119_session_log.sql b/tests/queries/0_stateless/01119_session_log.sql index 55f6228797a..8f6967b89ec 100644 --- a/tests/queries/0_stateless/01119_session_log.sql +++ b/tests/queries/0_stateless/01119_session_log.sql @@ -4,7 +4,7 @@ select * from remote('127.0.0.2', system, one, 'default', ''); select * from remote('127.0.0.2', system, one, 'default', 'wrong password'); -- { serverError AUTHENTICATION_FAILED } select * from remote('127.0.0.2', system, one, 'nonexistsnt_user_1119', ''); -- { serverError AUTHENTICATION_FAILED } set receive_timeout=1; -select * from remote('127.0.0.2', system, one, ' INTERSERVER SECRET ', ''); -- { serverError NO_REMOTE_SHARD_AVAILABLE } +select * from remote('127.0.0.2', system, one, ' INTERSERVER SECRET ', ''); -- { serverError AUTHENTICATION_FAILED } set receive_timeout=300; select * from remote('127.0.0.2', system, one, ' ', ''); -- { serverError AUTHENTICATION_FAILED } From 09bec3c754698f2c7e3cf37a9b02018c7adddc33 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Nov 2023 00:16:02 +0100 Subject: [PATCH 416/813] Fix integration test --- .../test_distributed_inter_server_secret/test.py | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 4276fcffbf9..6e3f1e6e416 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -304,26 +304,20 @@ def test_secure_insert_buffer_async(): def test_secure_disagree(): - with pytest.raises( - QueryRuntimeException, match=".*Interserver authentication failed.*" - ): + with pytest.raises(QueryRuntimeException): n1.query("SELECT * FROM dist_secure_disagree") def test_secure_disagree_insert(): n1.query("TRUNCATE TABLE data") n1.query("INSERT INTO dist_secure_disagree SELECT * FROM numbers(2)") - with pytest.raises( - QueryRuntimeException, match=".*Interserver authentication failed.*" - ): + with pytest.raises(QueryRuntimeException): n1.query( "SYSTEM FLUSH DISTRIBUTED ON CLUSTER secure_disagree dist_secure_disagree" ) - # check the the connection will be re-established + # check that the connection will be re-established # IOW that we will not get "Unknown BlockInfo field" - with pytest.raises( - QueryRuntimeException, match=".*Interserver authentication failed.*" - ): + with pytest.raises(QueryRuntimeException): assert int(n1.query("SELECT count() FROM dist_secure_disagree")) == 0 From e0f6c684655be08ed37139438fadeb7b85048bce Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 16 Nov 2023 02:51:29 +0300 Subject: [PATCH 417/813] Lint --- src/Common/Config/ConfigProcessor.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 2e7372fa9a2..c213b7257d9 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -334,10 +334,7 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, bool source_has_value = with_element.hasChildNodes(); if (source_has_value) for (const auto & attr_name: SUBSTITUTION_ATTRS) - { - if (config_element.hasAttribute(attr_name)) - config_element.removeAttribute(attr_name); - } + config_element.removeAttribute(attr_name); mergeAttributes(config_element, with_element); mergeRecursive(config, config_node, with_node); From 1c4f2454e4d53a52d4fc0b2b492aa77c1e447ab9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 16 Nov 2023 01:38:42 +0000 Subject: [PATCH 418/813] Disable test with database Ordinary --- .../queries/0_stateless/02888_replicated_merge_tree_creation.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh index b3ccef5de30..b9603e75d2e 100755 --- a/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel +# Tags: zookeeper, no-parallel, no-ordinary-database CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From db666bf2bc063eac1668258f890a6766dc3c4430 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Wed, 15 Nov 2023 21:47:35 -0400 Subject: [PATCH 419/813] Disables RU intro section. --- docs/ru/introduction/_category_.yml | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/ru/introduction/_category_.yml b/docs/ru/introduction/_category_.yml index 539f7ab97ed..b3e58207c12 100644 --- a/docs/ru/introduction/_category_.yml +++ b/docs/ru/introduction/_category_.yml @@ -2,6 +2,3 @@ position: 1 label: 'Введение' collapsible: true collapsed: true -link: - type: generated-index - title: Введение From f505181b0d8c485ee02d3e62fc057c22d5d188cc Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Wed, 15 Nov 2023 22:08:10 -0400 Subject: [PATCH 420/813] Adds basics index page to RU introduction. --- docs/ru/introduction/index.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) create mode 100644 docs/ru/introduction/index.md diff --git a/docs/ru/introduction/index.md b/docs/ru/introduction/index.md new file mode 100644 index 00000000000..74a6e4dd135 --- /dev/null +++ b/docs/ru/introduction/index.md @@ -0,0 +1,13 @@ +--- +slug: /ru/introduction/ +sidebar_label: "Введение" +sidebar_position: 8 +--- + +# Введение + +В этом разделе содержится информация о том, как начать работу с ClickHouse. + +- [Отличительные возможности ClickHouse](./distinctive-features.md) +- [Производительность](./performance.md) +- [История ClickHouse](./history.md) From ef17d972ab64c90ff72a7c6c2beba61b522d6fcf Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 16 Nov 2023 05:18:50 +0000 Subject: [PATCH 421/813] Fix SET query formatting --- src/Parsers/ASTSetQuery.cpp | 48 ++++++++++++++++++- .../02916_set_formatting.reference | 11 +++++ .../0_stateless/02916_set_formatting.sql | 13 +++++ 3 files changed, 71 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02916_set_formatting.reference create mode 100644 tests/queries/0_stateless/02916_set_formatting.sql diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index e2c60e8369d..78161b865ee 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -4,11 +4,57 @@ #include #include #include +#include namespace DB { +class FieldVisitorToSetting : public StaticVisitor +{ +public: + template + String operator() (const T & x) const + { + FieldVisitorToString visitor; + return visitor(x); + } + + String operator() (const Map & x) const + { + WriteBufferFromOwnString wb; + + wb << '{'; + + auto it = x.begin(); + while (it != x.end()) + { + if (it != x.begin()) + wb << ", "; + wb << applyVisitor(*this, *it); + ++it; + } + wb << '}'; + + return wb.str(); + } + + String operator() (const Tuple & x) const + { + WriteBufferFromOwnString wb; + + for (auto it = x.begin(); it != x.end(); ++it) + { + if (it != x.begin()) + wb << ":"; + wb << applyVisitor(*this, *it); + } + + return wb.str(); + } +}; + + void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state, bool /*ignore_aliases*/) const { for (const auto & change : changes) @@ -38,7 +84,7 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma if (!format.show_secrets && change.value.tryGet(custom) && custom.isSecret()) format.ostr << " = " << custom.toString(false); else - format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); + format.ostr << " = " << applyVisitor(FieldVisitorToSetting(), change.value); } for (const auto & setting_name : default_settings) diff --git a/tests/queries/0_stateless/02916_set_formatting.reference b/tests/queries/0_stateless/02916_set_formatting.reference new file mode 100644 index 00000000000..34ff52365f9 --- /dev/null +++ b/tests/queries/0_stateless/02916_set_formatting.reference @@ -0,0 +1,11 @@ +SET additional_table_filters = {\'kjsnckjn\':\'ksanmn\', \'dkm\':\'dd\'} +SELECT v FROM t1 SETTINGS additional_table_filters = {\'default.t1\':\'s\'} +Row 1: +────── +statement: CREATE VIEW default.v1 +( + `v` UInt64 +) AS +SELECT v +FROM default.t1 +SETTINGS additional_table_filters = {'default.t1':'s != \'s1%\''} diff --git a/tests/queries/0_stateless/02916_set_formatting.sql b/tests/queries/0_stateless/02916_set_formatting.sql new file mode 100644 index 00000000000..10b875293f1 --- /dev/null +++ b/tests/queries/0_stateless/02916_set_formatting.sql @@ -0,0 +1,13 @@ +SELECT formatQuerySingleLine('set additional_table_filters = {\'kjsnckjn\': \'ksanmn\', \'dkm\': \'dd\'}'); +SELECT formatQuerySingleLine('SELECT v FROM t1 SETTINGS additional_table_filters = {\'default.t1\': \'s\'}'); + +DROP TABLE IF EXISTS t1; +DROP VIEW IF EXISTS v1; + +CREATE TABLE t1 (v UInt64, s String) ENGINE=MergeTree() ORDER BY v; +CREATE VIEW v1 (v UInt64) AS SELECT v FROM t1 SETTINGS additional_table_filters = {'default.t1': 's != \'s1%\''}; + +SHOW CREATE TABLE v1 FORMAT Vertical; + +DROP VIEW v1; +DROP TABLE t1; From 1d7eecaeece1bb2cdadf8c446bd8631592ecfb08 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 16 Nov 2023 06:08:12 +0000 Subject: [PATCH 422/813] Fix failure to start due to table dependency in joinGet --- src/Databases/DDLLoadingDependencyVisitor.cpp | 16 +++++------ .../02916_joinget_dependency.reference | 1 + .../0_stateless/02916_joinget_dependency.sh | 27 +++++++++++++++++++ 3 files changed, 36 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02916_joinget_dependency.reference create mode 100755 tests/queries/0_stateless/02916_joinget_dependency.sh diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index fc362dd8578..77a40f674fd 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -144,22 +144,22 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction const auto * arg = function.arguments->as()->children[arg_idx].get(); - if (const auto * dict_function = arg->as()) + if (const auto * function_arg = arg->as()) { - if (!functionIsDictGet(dict_function->name)) + if (!functionIsJoinGet(function_arg->name) && !functionIsDictGet(function_arg->name)) return; - /// Get the dictionary name from `dict*` function. - const auto * literal_arg = dict_function->arguments->as()->children[0].get(); - const auto * dictionary_name = literal_arg->as(); + /// Get the dictionary name from `dict*` function or the table name from 'joinGet' function. + const auto * literal_arg = function_arg->arguments->as()->children[0].get(); + const auto * name = literal_arg->as(); - if (!dictionary_name) + if (!name) return; - if (dictionary_name->value.getType() != Field::Types::String) + if (name->value.getType() != Field::Types::String) return; - auto maybe_qualified_name = QualifiedTableName::tryParseFromString(dictionary_name->value.get()); + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(name->value.get()); if (!maybe_qualified_name) return; diff --git a/tests/queries/0_stateless/02916_joinget_dependency.reference b/tests/queries/0_stateless/02916_joinget_dependency.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02916_joinget_dependency.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02916_joinget_dependency.sh b/tests/queries/0_stateless/02916_joinget_dependency.sh new file mode 100755 index 00000000000..6477ae8c967 --- /dev/null +++ b/tests/queries/0_stateless/02916_joinget_dependency.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# We test the dependency on the DROP + +$CLICKHOUSE_CLIENT -nm -q " + DROP TABLE IF EXISTS Sub_distributed; + DROP TABLE IF EXISTS Sub; + DROP TABLE IF EXISTS Mapping; + + CREATE TABLE Mapping (Id UInt64, RegionId UInt64) ENGINE = Join(ANY,LEFT,Id); + INSERT INTO Mapping VALUES (1,1); + CREATE TABLE Sub (Id UInt64, PropertyId UInt64) ENGINE = MergeTree() PRIMARY KEY (Id) ORDER BY (Id); + CREATE TABLE Sub_distributed (Id UInt64, PropertyId UInt64)ENGINE = Distributed('test_shard_localhost', $CLICKHOUSE_DATABASE, Sub, joinGet('$CLICKHOUSE_DATABASE.Mapping','RegionId',PropertyId));" + +$CLICKHOUSE_CLIENT -q " + DROP TABLE Mapping; +" 2>&1 | grep -cm1 "HAVE_DEPENDENT_OBJECTS" + +$CLICKHOUSE_CLIENT -nm -q " + DROP TABLE Sub_distributed; + DROP TABLE Sub; + DROP TABLE Mapping; +" \ No newline at end of file From 7623153d3841d58f282915e24f9d032ee6d84e94 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 16 Nov 2023 07:01:49 +0000 Subject: [PATCH 423/813] Fix flattening existing Nested columns during ADD COLUMN --- src/Storages/AlterCommands.cpp | 21 +++++++++++++++---- .../02916_addcolumn_nested.reference | 3 +++ .../0_stateless/02916_addcolumn_nested.sql | 17 +++++++++++++++ 3 files changed, 37 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02916_addcolumn_nested.reference create mode 100644 tests/queries/0_stateless/02916_addcolumn_nested.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 98bfa3b3f57..7eeaa2d4594 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -395,11 +395,24 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) column.ttl = ttl; - metadata.columns.add(column, after_column, first); - - /// Slow, because each time a list is copied if (context->getSettingsRef().flatten_nested) - metadata.columns.flattenNested(); + { + StorageInMemoryMetadata temporary_metadata; + temporary_metadata.columns.add(column, /*after_column*/ "", /*first*/ true); + temporary_metadata.columns.flattenNested(); + + const auto transformed_columns = temporary_metadata.columns.getAll(); + + for (auto it = transformed_columns.rbegin(); it != transformed_columns.rend(); it++) + { + const auto & transformed_column = temporary_metadata.columns.get(it->name); + metadata.columns.add(transformed_column, after_column, first); + } + } + else + { + metadata.columns.add(column, after_column, first); + } } else if (type == DROP_COLUMN) { diff --git a/tests/queries/0_stateless/02916_addcolumn_nested.reference b/tests/queries/0_stateless/02916_addcolumn_nested.reference new file mode 100644 index 00000000000..869d4336c62 --- /dev/null +++ b/tests/queries/0_stateless/02916_addcolumn_nested.reference @@ -0,0 +1,3 @@ +CREATE TABLE default.nested_table\n(\n `id` UInt64,\n `first` Nested(a Int8, b String)\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.nested_table\n(\n `id` UInt64,\n `second.c` Array(Int8),\n `second.d` Array(String),\n `first` Nested(a Int8, b String)\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.nested_table\n(\n `third` Nested(e Int8, f String),\n `id` UInt64,\n `second.c` Array(Int8),\n `second.d` Array(String),\n `first` Nested(a Int8, b String)\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02916_addcolumn_nested.sql b/tests/queries/0_stateless/02916_addcolumn_nested.sql new file mode 100644 index 00000000000..b23854824b5 --- /dev/null +++ b/tests/queries/0_stateless/02916_addcolumn_nested.sql @@ -0,0 +1,17 @@ +SET flatten_nested = 0; + +DROP TABLE IF EXISTS nested_table; +CREATE TABLE nested_table (id UInt64, first Nested(a Int8, b String)) ENGINE = MergeTree() ORDER BY id; +SHOW CREATE nested_table; + +SET flatten_nested = 1; + +ALTER TABLE nested_table ADD COLUMN second Nested(c Int8, d String) AFTER id; +SHOW CREATE nested_table; + +SET flatten_nested = 0; + +ALTER TABLE nested_table ADD COLUMN third Nested(e Int8, f String) FIRST; +SHOW CREATE nested_table; + +DROP TABLE nested_table; From cffc6611e000faa19cac35dccfc093d77ae7e315 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Thu, 16 Nov 2023 07:05:41 +0000 Subject: [PATCH 424/813] Empty commit. From 052134b143f59c1a2061beebec73009c34077c8c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Nov 2023 07:32:11 +0000 Subject: [PATCH 425/813] Better --- docker/test/stateless/stress_tests.lib | 16 ++++++++++------ tests/config/config.d/zookeeper.xml | 2 +- .../config.d/zookeeper_fault_injection.xml | 1 + 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index edcf2bc7bee..551461b6eca 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -53,7 +53,7 @@ function configure() > /etc/clickhouse-server/config.d/keeper_port.xml.tmp sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml - function randomize_keeper_config_boolean_value { + function randomize_config_boolean_value { value=$(($RANDOM % 2)) sudo cat /etc/clickhouse-server/config.d/$2.xml \ | sed "s|<$1>[01]|<$1>$value|" \ @@ -63,16 +63,20 @@ function configure() if [[ -n "$RANDOMIZE_KEEPER_FEATURE_FLAGS" ]] && [[ "$RANDOMIZE_KEEPER_FEATURE_FLAGS" -eq 1 ]]; then # Randomize all Keeper feature flags - randomize_keeper_config_boolean_value filtered_list keeper_port - randomize_keeper_config_boolean_value multi_read keeper_port - randomize_keeper_config_boolean_value check_not_exists keeper_port - randomize_keeper_config_boolean_value create_if_not_exists keeper_port + randomize_config_boolean_value filtered_list keeper_port + randomize_config_boolean_value multi_read keeper_port + randomize_config_boolean_value check_not_exists keeper_port + randomize_config_boolean_value create_if_not_exists keeper_port fi sudo chown clickhouse /etc/clickhouse-server/config.d/keeper_port.xml sudo chgrp clickhouse /etc/clickhouse-server/config.d/keeper_port.xml - randomize_keeper_config_boolean_value use_compression zookeeper + if [[ -n "$ZOOKEEPER_FAULT_INJECTION" ]] && [[ "$ZOOKEEPER_FAULT_INJECTION" -eq 1 ]]; then + randomize_config_boolean_value use_compression zookeeper_fault_injection + else + randomize_config_boolean_value use_compression zookeeper + fi # for clickhouse-server (via service) echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index a54149e6617..ce402f4850b 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -2,7 +2,7 @@ random - true + 1 127.0.0.1 9181 diff --git a/tests/config/config.d/zookeeper_fault_injection.xml b/tests/config/config.d/zookeeper_fault_injection.xml index a339e1f0fba..75b96064817 100644 --- a/tests/config/config.d/zookeeper_fault_injection.xml +++ b/tests/config/config.d/zookeeper_fault_injection.xml @@ -1,5 +1,6 @@ + 1 localhost 9181 From 69f214cdbcc85808d77893112f7c560285747a09 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Thu, 16 Nov 2023 08:04:57 +0000 Subject: [PATCH 426/813] fix comments. Signed-off-by: Jianfei Hu --- src/IO/S3/Credentials.cpp | 32 +++++++++++++++++++------------- src/IO/S3/Credentials.h | 1 - 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index e25f4551723..73763853713 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,4 +1,15 @@ #include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +} #if USE_AWS_S3 @@ -16,11 +27,9 @@ # include # include -# include # include # include -# include # include # include @@ -42,7 +51,6 @@ namespace ErrorCodes { extern const int AWS_ERROR; extern const int GCP_ERROR; - extern const int UNSUPPORTED_METHOD; } namespace S3 @@ -280,20 +288,20 @@ String getRunningAvailabilityZone() LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); try { - auto aws_az = AWSEC2MetadataClient::getAvailabilityZoneOrException(); - return aws_az; + return AWSEC2MetadataClient::getAvailabilityZoneOrException(); } - catch (const std::exception & aws_ex) + catch (...) { + auto aws_ex_msg = getExceptionMessage(std::current_exception(), false); try { - auto gcp_zone = getGCPAvailabilityZoneOrException(); - return gcp_zone; + return getGCPAvailabilityZoneOrException(); } - catch (const std::exception & gcp_ex) + catch (...) { + auto gcp_ex_msg = getExceptionMessage(std::current_exception(), false); throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Failed to find the availability zone, tried AWS and GCP. AWS Error: {}\nGCP Error: {}", aws_ex.what(), gcp_ex.what()); + "Failed to find the availability zone, tried AWS and GCP. AWS Error: {}\nGCP Error: {}", aws_ex_msg, gcp_ex_msg); } } } @@ -784,8 +792,6 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( #else -# include - namespace DB { @@ -794,7 +800,7 @@ namespace S3 std::string getRunningAvailabilityZone() { - throw std::runtime_error("Does not support availability zone detection for non-cloud environment"); + throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Does not support availability zone detection for non-cloud environment"); } } diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index d8d103a847a..ad73de23486 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -4,7 +4,6 @@ #if USE_AWS_S3 -# include # include # include From 3b226a10feac18ae16836f5e627d48b413c12cc1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Nov 2023 09:50:34 +0100 Subject: [PATCH 427/813] Fix test --- .../0_stateless/01555_system_distribution_queue_mask.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql index 7ade1d24c59..3a90765226a 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -17,7 +17,7 @@ system stop distributed sends dist_01555; insert into dist_01555 values (1)(2); -- since test_cluster_with_incorrect_pw contains incorrect password ignore error -system flush distributed dist_01555; -- { clientError ATTEMPT_TO_READ_AFTER_EOF } +system flush distributed dist_01555; -- { serverError 516 } select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; drop table dist_01555; @@ -30,7 +30,7 @@ create table dist_01555 (key Int) Engine=Distributed(test_cluster_with_incorrect insert into dist_01555 values (1)(2); -- since test_cluster_with_incorrect_pw contains incorrect password ignore error -system flush distributed dist_01555; -- { clientError ATTEMPT_TO_READ_AFTER_EOF } +system flush distributed dist_01555; -- { serverError 516 } select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; drop table dist_01555; From 2ddc6132f9b20f4b891d5f3a40508876a4fc0b0b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Nov 2023 10:23:54 +0100 Subject: [PATCH 428/813] Revert "Better except for SSL authentication failure" --- src/Server/TCPHandler.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c0bcb5cb89a..f929d0f5ff9 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -104,7 +104,6 @@ namespace DB::ErrorCodes extern const int TIMEOUT_EXCEEDED; extern const int SUPPORT_IS_DISABLED; extern const int UNSUPPORTED_METHOD; - extern const int WRONG_PASSWORD; } namespace @@ -1432,11 +1431,8 @@ void TCPHandler::receiveHello() getClientAddress(client_info)); return; } - catch (const Exception & e) + catch (...) { - if (e.code() != DB::ErrorCodes::WRONG_PASSWORD) - throw; - tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication"); } } From 61948e31714f1aa3fa8143a569a72403c5c70408 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 16 Nov 2023 11:12:45 +0100 Subject: [PATCH 429/813] Update src/Core/Settings.h Co-authored-by: Nikita Taranov --- 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 34547aded9c..76016bc70fb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,7 +94,7 @@ class IColumn; 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 aggressive timeouts are enabled first two attempts are made with low receive and send timeout", 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(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) \ From bdeb04f7d3b403a425dd4138fdf4c5fd4c72cb96 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 15 Nov 2023 16:00:54 +0000 Subject: [PATCH 430/813] Exctract JOIN ON visitor from LogicalExpressionOptimizerVisitor --- .../Passes/LogicalExpressionOptimizerPass.cpp | 355 +++++++++--------- 1 file changed, 181 insertions(+), 174 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 9602ef8a743..50410c8c829 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -16,6 +16,181 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +/// Visitor that optimizes logical expressions _only_ in JOIN ON section +class JoinOnLogicalExpressionOptimizerVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + + explicit JoinOnLogicalExpressionOptimizerVisitor(ContextPtr context) + : Base(std::move(context)) + {} + + void enterImpl(QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + + if (!function_node) + return; + + if (function_node->getFunctionName() == "or") + { + tryOptimizeIsNotDistinctOrIsNull(node); + return; + } + } + +private: + void tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node) + { + auto & function_node = node->as(); + assert(function_node.getFunctionName() == "or"); + + QueryTreeNodes or_operands; + + /// Indices of `equals` or `isNotDistinctFrom` functions in the vector above + std::vector equals_functions_indices; + + /** Map from `isNull` argument to indices of operands that contains that `isNull` functions + * `a = b OR (a IS NULL AND b IS NULL) OR (a IS NULL AND c IS NULL)` + * will be mapped to + * { + * a => [(a IS NULL AND b IS NULL), (a IS NULL AND c IS NULL)] + * b => [(a IS NULL AND b IS NULL)] + * c => [(a IS NULL AND c IS NULL)] + * } + * Then for each a <=> b we can find all operands that contains both a IS NULL and b IS NULL + */ + QueryTreeNodePtrWithHashMap> is_null_argument_to_indices; + + for (const auto & argument : function_node.getArguments()) + { + or_operands.push_back(argument); + + auto * argument_function = argument->as(); + if (!argument_function) + continue; + + const auto & func_name = argument_function->getFunctionName(); + if (func_name == "equals" || func_name == "isNotDistinctFrom") + equals_functions_indices.push_back(or_operands.size() - 1); + + if (func_name == "and") + { + for (const auto & and_argument : argument_function->getArguments().getNodes()) + { + auto * and_argument_function = and_argument->as(); + if (and_argument_function && and_argument_function->getFunctionName() == "isNull") + { + const auto & is_null_argument = and_argument_function->getArguments().getNodes()[0]; + is_null_argument_to_indices[is_null_argument].push_back(or_operands.size() - 1); + } + } + } + } + + /// OR operands that are changed to and needs to be re-resolved + std::unordered_set arguments_to_reresolve; + + for (size_t equals_function_idx : equals_functions_indices) + { + auto * equals_function = or_operands[equals_function_idx]->as(); + + /// For a <=> b we are looking for expressions containing both `a IS NULL` and `b IS NULL` combined with AND + const auto & argument_nodes = equals_function->getArguments().getNodes(); + const auto & lhs_is_null_parents = is_null_argument_to_indices[argument_nodes[0]]; + const auto & rhs_is_null_parents = is_null_argument_to_indices[argument_nodes[1]]; + std::unordered_set operands_to_optimize; + std::set_intersection(lhs_is_null_parents.begin(), lhs_is_null_parents.end(), + rhs_is_null_parents.begin(), rhs_is_null_parents.end(), + std::inserter(operands_to_optimize, operands_to_optimize.begin())); + + /// If we have `a = b OR (a IS NULL AND b IS NULL)` we can optimize it to `a <=> b` + if (!operands_to_optimize.empty() && equals_function->getFunctionName() == "equals") + arguments_to_reresolve.insert(equals_function_idx); + + for (size_t to_optimize_idx : operands_to_optimize) + { + /// We are looking for operand `a IS NULL AND b IS NULL AND ...` + auto * operand_to_optimize = or_operands[to_optimize_idx]->as(); + + /// Remove `a IS NULL` and `b IS NULL` arguments from AND + QueryTreeNodes new_arguments; + for (const auto & and_argument : operand_to_optimize->getArguments().getNodes()) + { + bool to_eliminate = false; + + const auto * and_argument_function = and_argument->as(); + if (and_argument_function && and_argument_function->getFunctionName() == "isNull") + { + const auto & is_null_argument = and_argument_function->getArguments().getNodes()[0]; + to_eliminate = (is_null_argument->isEqual(*argument_nodes[0]) || is_null_argument->isEqual(*argument_nodes[1])); + } + + if (to_eliminate) + arguments_to_reresolve.insert(to_optimize_idx); + else + new_arguments.emplace_back(and_argument); + } + /// If less than two arguments left, we will remove or replace the whole AND below + operand_to_optimize->getArguments().getNodes() = std::move(new_arguments); + } + } + + if (arguments_to_reresolve.empty()) + /// Nothing have been changed + return; + + auto and_function_resolver = FunctionFactory::instance().get("and", getContext()); + auto strict_equals_function_resolver = FunctionFactory::instance().get("isNotDistinctFrom", getContext()); + QueryTreeNodes new_or_operands; + for (size_t i = 0; i < or_operands.size(); ++i) + { + if (arguments_to_reresolve.contains(i)) + { + auto * function = or_operands[i]->as(); + if (function->getFunctionName() == "equals") + { + /// Because we removed checks for IS NULL, we should replace `a = b` with `a <=> b` + function->resolveAsFunction(strict_equals_function_resolver); + new_or_operands.emplace_back(std::move(or_operands[i])); + } + else if (function->getFunctionName() == "and") + { + const auto & and_arguments = function->getArguments().getNodes(); + if (and_arguments.size() > 1) + { + function->resolveAsFunction(and_function_resolver); + new_or_operands.emplace_back(std::move(or_operands[i])); + } + else if (and_arguments.size() == 1) + { + /// Replace AND with a single argument with the argument itself + new_or_operands.emplace_back(and_arguments[0]); + } + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function name: '{}'", function->getFunctionName()); + } + else + { + new_or_operands.emplace_back(std::move(or_operands[i])); + } + } + + if (new_or_operands.size() == 1) + { + node = std::move(new_or_operands[0]); + return; + } + + /// Rebuild OR function + auto or_function_resolver = FunctionFactory::instance().get("or", getContext()); + function_node.getArguments().getNodes() = std::move(new_or_operands); + function_node.resolveAsFunction(or_function_resolver); + } +}; + class LogicalExpressionOptimizerVisitor : public InDepthQueryTreeVisitorWithContext { public: @@ -29,13 +204,15 @@ public: { if (auto * join_node = node->as()) { - join_stack.push_back(join_node); + /// Operator <=> is not supported outside of JOIN ON section + if (join_node->hasJoinExpression()) + { + JoinOnLogicalExpressionOptimizerVisitor join_on_visitor(getContext()); + join_on_visitor.visit(join_node->getJoinExpression()); + } return; } - if (!join_stack.empty() && join_stack.back()->getJoinExpression().get() == node.get()) - is_inside_on_section = true; - auto * function_node = node->as(); if (!function_node) @@ -44,10 +221,6 @@ public: if (function_node->getFunctionName() == "or") { tryReplaceOrEqualsChainWithIn(node); - - /// Operator <=> is not supported outside of JOIN ON section - if (is_inside_on_section) - tryOptimizeIsNotDistinctOrIsNull(node); return; } @@ -58,19 +231,6 @@ public: } } - void leaveImpl(QueryTreeNodePtr & node) - { - if (!join_stack.empty() && join_stack.back()->getJoinExpression().get() == node.get()) - is_inside_on_section = false; - - if (auto * join_node = node->as()) - { - assert(join_stack.back() == join_node); - join_stack.pop_back(); - return; - } - } - private: void tryReplaceAndEqualsChainsWithConstant(QueryTreeNodePtr & node) { @@ -264,159 +424,6 @@ private: function_node.getArguments().getNodes() = std::move(or_operands); function_node.resolveAsFunction(or_function_resolver); } - - void tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node) - { - auto & function_node = node->as(); - assert(function_node.getFunctionName() == "or"); - - QueryTreeNodes or_operands; - - /// Indices of `equals` or `isNotDistinctFrom` functions in the vector above - std::vector equals_functions_indices; - - /** Map from `isNull` argument to indices of operands that contains that `isNull` functions - * `a = b OR (a IS NULL AND b IS NULL) OR (a IS NULL AND c IS NULL)` - * will be mapped to - * { - * a => [(a IS NULL AND b IS NULL), (a IS NULL AND c IS NULL)] - * b => [(a IS NULL AND b IS NULL)] - * c => [(a IS NULL AND c IS NULL)] - * } - * Then for each a <=> b we can find all operands that contains both a IS NULL and b IS NULL - */ - QueryTreeNodePtrWithHashMap> is_null_argument_to_indices; - - for (const auto & argument : function_node.getArguments()) - { - or_operands.push_back(argument); - - auto * argument_function = argument->as(); - if (!argument_function) - continue; - - const auto & func_name = argument_function->getFunctionName(); - if (func_name == "equals" || func_name == "isNotDistinctFrom") - equals_functions_indices.push_back(or_operands.size() - 1); - - if (func_name == "and") - { - for (const auto & and_argument : argument_function->getArguments().getNodes()) - { - auto * and_argument_function = and_argument->as(); - if (and_argument_function && and_argument_function->getFunctionName() == "isNull") - { - const auto & is_null_argument = and_argument_function->getArguments().getNodes()[0]; - is_null_argument_to_indices[is_null_argument].push_back(or_operands.size() - 1); - } - } - } - } - - /// OR operands that are changed to and needs to be re-resolved - std::unordered_set arguments_to_reresolve; - - for (size_t equals_function_idx : equals_functions_indices) - { - auto * equals_function = or_operands[equals_function_idx]->as(); - - /// For a <=> b we are looking for expressions containing both `a IS NULL` and `b IS NULL` combined with AND - const auto & argument_nodes = equals_function->getArguments().getNodes(); - const auto & lhs_is_null_parents = is_null_argument_to_indices[argument_nodes[0]]; - const auto & rhs_is_null_parents = is_null_argument_to_indices[argument_nodes[1]]; - std::unordered_set operands_to_optimize; - std::set_intersection(lhs_is_null_parents.begin(), lhs_is_null_parents.end(), - rhs_is_null_parents.begin(), rhs_is_null_parents.end(), - std::inserter(operands_to_optimize, operands_to_optimize.begin())); - - /// If we have `a = b OR (a IS NULL AND b IS NULL)` we can optimize it to `a <=> b` - if (!operands_to_optimize.empty() && equals_function->getFunctionName() == "equals") - arguments_to_reresolve.insert(equals_function_idx); - - for (size_t to_optimize_idx : operands_to_optimize) - { - /// We are looking for operand `a IS NULL AND b IS NULL AND ...` - auto * operand_to_optimize = or_operands[to_optimize_idx]->as(); - - /// Remove `a IS NULL` and `b IS NULL` arguments from AND - QueryTreeNodes new_arguments; - for (const auto & and_argument : operand_to_optimize->getArguments().getNodes()) - { - bool to_eliminate = false; - - const auto * and_argument_function = and_argument->as(); - if (and_argument_function && and_argument_function->getFunctionName() == "isNull") - { - const auto & is_null_argument = and_argument_function->getArguments().getNodes()[0]; - to_eliminate = (is_null_argument->isEqual(*argument_nodes[0]) || is_null_argument->isEqual(*argument_nodes[1])); - } - - if (to_eliminate) - arguments_to_reresolve.insert(to_optimize_idx); - else - new_arguments.emplace_back(and_argument); - } - /// If less than two arguments left, we will remove or replace the whole AND below - operand_to_optimize->getArguments().getNodes() = std::move(new_arguments); - } - } - - - if (arguments_to_reresolve.empty()) - /// Nothing have been changed - return; - - auto and_function_resolver = FunctionFactory::instance().get("and", getContext()); - auto strict_equals_function_resolver = FunctionFactory::instance().get("isNotDistinctFrom", getContext()); - QueryTreeNodes new_or_operands; - for (size_t i = 0; i < or_operands.size(); ++i) - { - if (arguments_to_reresolve.contains(i)) - { - auto * function = or_operands[i]->as(); - if (function->getFunctionName() == "equals") - { - /// Because we removed checks for IS NULL, we should replace `a = b` with `a <=> b` - function->resolveAsFunction(strict_equals_function_resolver); - new_or_operands.emplace_back(std::move(or_operands[i])); - } - else if (function->getFunctionName() == "and") - { - const auto & and_arguments = function->getArguments().getNodes(); - if (and_arguments.size() > 1) - { - function->resolveAsFunction(and_function_resolver); - new_or_operands.emplace_back(std::move(or_operands[i])); - } - else if (and_arguments.size() == 1) - { - /// Replace AND with a single argument with the argument itself - new_or_operands.emplace_back(and_arguments[0]); - } - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function name: '{}'", function->getFunctionName()); - } - else - { - new_or_operands.emplace_back(std::move(or_operands[i])); - } - } - - if (new_or_operands.size() == 1) - { - node = std::move(new_or_operands[0]); - return; - } - - /// Rebuild OR function - auto or_function_resolver = FunctionFactory::instance().get("or", getContext()); - function_node.getArguments().getNodes() = std::move(new_or_operands); - function_node.resolveAsFunction(or_function_resolver); - } - - bool is_inside_on_section = false; - std::deque join_stack; }; void LogicalExpressionOptimizerPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) From 6ad0e9066a65c30aa9abb758dd8f05b99a31bb68 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 15 Nov 2023 17:02:30 +0000 Subject: [PATCH 431/813] Rerun resolve in JoinOnLogicalExpressionOptimizerVisitor --- .../Passes/LogicalExpressionOptimizerPass.cpp | 35 ++++++++++++++----- 1 file changed, 27 insertions(+), 8 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 50410c8c829..e667b603020 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -35,13 +36,27 @@ public: if (function_node->getFunctionName() == "or") { - tryOptimizeIsNotDistinctOrIsNull(node); + bool is_argument_type_changed = tryOptimizeIsNotDistinctOrIsNull(node, getContext()); + if (is_argument_type_changed) + need_rerun_resolve = true; return; } } + void leaveImpl(QueryTreeNodePtr & node) + { + if (!need_rerun_resolve) + return; + + if (auto * function_node = node->as()) + rerunFunctionResolve(function_node, getContext()); + } + private: - void tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node) + bool need_rerun_resolve = false; + + /// Returns true if type of some operand is changed and parent function needs to be re-resolved + static bool tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node, const ContextPtr & context) { auto & function_node = node->as(); assert(function_node.getFunctionName() == "or"); @@ -139,10 +154,12 @@ private: if (arguments_to_reresolve.empty()) /// Nothing have been changed - return; + return false; - auto and_function_resolver = FunctionFactory::instance().get("and", getContext()); - auto strict_equals_function_resolver = FunctionFactory::instance().get("isNotDistinctFrom", getContext()); + auto and_function_resolver = FunctionFactory::instance().get("and", context); + auto strict_equals_function_resolver = FunctionFactory::instance().get("isNotDistinctFrom", context); + + bool need_reresolve = false; QueryTreeNodes new_or_operands; for (size_t i = 0; i < or_operands.size(); ++i) { @@ -151,7 +168,8 @@ private: auto * function = or_operands[i]->as(); if (function->getFunctionName() == "equals") { - /// Because we removed checks for IS NULL, we should replace `a = b` with `a <=> b` + /// We should replace `a = b` with `a <=> b` because we removed checks for IS NULL + need_reresolve = need_reresolve || function->getResultType()->isNullable(); function->resolveAsFunction(strict_equals_function_resolver); new_or_operands.emplace_back(std::move(or_operands[i])); } @@ -181,13 +199,14 @@ private: if (new_or_operands.size() == 1) { node = std::move(new_or_operands[0]); - return; + return need_reresolve; } /// Rebuild OR function - auto or_function_resolver = FunctionFactory::instance().get("or", getContext()); + auto or_function_resolver = FunctionFactory::instance().get("or", context); function_node.getArguments().getNodes() = std::move(new_or_operands); function_node.resolveAsFunction(or_function_resolver); + return need_reresolve; } }; From 21a17f83347ae8ff0579604f13eb62e4cbcd163f Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 16 Nov 2023 12:57:09 +0300 Subject: [PATCH 432/813] Add cancellation hook for moving background operation --- src/Disks/DiskEncrypted.cpp | 10 +- src/Disks/DiskEncrypted.h | 8 +- src/Disks/DiskLocal.cpp | 10 +- src/Disks/DiskLocal.h | 8 +- src/Disks/IDisk.cpp | 52 ++++++-- src/Disks/IDisk.h | 20 ++- .../ObjectStorages/DiskObjectStorage.cpp | 6 +- src/Disks/ObjectStorages/DiskObjectStorage.h | 4 +- .../MergeTree/DataPartStorageOnDiskBase.cpp | 5 +- .../MergeTree/DataPartStorageOnDiskBase.h | 4 +- src/Storages/MergeTree/IDataPartStorage.h | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 9 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 7 +- .../MergeTree/MergeTreePartsMover.cpp | 12 +- tests/integration/test_ttl_move/test.py | 116 +++++++++++++++++- 15 files changed, 240 insertions(+), 35 deletions(-) diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 7bc7c1c7dc4..5ec17701667 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -324,7 +324,13 @@ ReservationPtr DiskEncrypted::reserve(UInt64 bytes) } -void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) +void DiskEncrypted::copyDirectoryContent( + const String & from_dir, + const std::shared_ptr & to_disk, + const String & to_dir, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook) { /// Check if we can copy the file without deciphering. if (isSameDiskType(*this, *to_disk)) @@ -340,7 +346,7 @@ void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::sha auto wrapped_from_path = wrappedPath(from_dir); auto to_delegate = to_disk_enc->delegate; auto wrapped_to_path = to_disk_enc->wrappedPath(to_dir); - delegate->copyDirectoryContent(wrapped_from_path, to_delegate, wrapped_to_path, read_settings, write_settings); + delegate->copyDirectoryContent(wrapped_from_path, to_delegate, wrapped_to_path, read_settings, write_settings, cancellation_hook); return; } } diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 8b4461a8dee..6574d3bf456 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -112,7 +112,13 @@ public: delegate->listFiles(wrapped_path, file_names); } - void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) override; + void copyDirectoryContent( + const String & from_dir, + const std::shared_ptr & to_disk, + const String & to_dir, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook = {}) override; std::unique_ptr readFile( const String & path, diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index c71f6f81de2..394c08e4876 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -432,13 +432,19 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) return typeid(one) == typeid(another); } -void DiskLocal::copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) +void DiskLocal::copyDirectoryContent( + const String & from_dir, + const std::shared_ptr & to_disk, + const String & to_dir, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook) { /// If throttling was configured we cannot use copying directly. if (isSameDiskType(*this, *to_disk) && !read_settings.local_throttler && !write_settings.local_throttler) fs::copy(fs::path(disk_path) / from_dir, fs::path(to_disk->getPath()) / to_dir, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way. else - IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings); + IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings, cancellation_hook); } SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index c52c192d824..767718ecb24 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -65,7 +65,13 @@ public: void replaceFile(const String & from_path, const String & to_path) override; - void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) override; + void copyDirectoryContent( + const String & from_dir, + const std::shared_ptr & to_disk, + const String & to_dir, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook = {}) override; void listFiles(const String & path, std::vector & file_names) const override; diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 1997ce06990..cd5f1f4ef69 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -24,14 +24,21 @@ bool IDisk::isDirectoryEmpty(const String & path) const return !iterateDirectory(path)->isValid(); } -void IDisk::copyFile(const String & from_file_path, IDisk & to_disk, const String & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) /// NOLINT +void IDisk::copyFile( + const String & from_file_path, + IDisk & to_disk, + const String & to_file_path, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook + ) { LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.", getName(), getPath(), from_file_path, to_disk.getName(), to_disk.getPath(), to_file_path); auto in = readFile(from_file_path, read_settings); auto out = to_disk.writeFile(to_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, write_settings); - copyData(*in, *out); + copyData(*in, *out, cancellation_hook); out->finalize(); } @@ -80,7 +87,17 @@ UInt128 IDisk::getEncryptedFileIV(const String &) const using ResultsCollector = std::vector>; -void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, ThreadPool & pool, ResultsCollector & results, bool copy_root_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) +void asyncCopy( + IDisk & from_disk, + String from_path, + IDisk & to_disk, + String to_path, + ThreadPool & pool, + ResultsCollector & results, + bool copy_root_dir, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook) { if (from_disk.isFile(from_path)) { @@ -88,7 +105,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p auto future = promise->get_future(); pool.scheduleOrThrowOnError( - [&from_disk, from_path, &to_disk, to_path, &read_settings, &write_settings, promise, thread_group = CurrentThread::getGroup()]() + [&from_disk, from_path, &to_disk, to_path, &read_settings, &write_settings, promise, thread_group = CurrentThread::getGroup(), &cancellation_hook]() { try { @@ -97,7 +114,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p if (thread_group) CurrentThread::attachToGroup(thread_group); - from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path), read_settings, write_settings); + from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path), read_settings, write_settings, cancellation_hook); promise->set_value(); } catch (...) @@ -119,11 +136,18 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p } for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) - asyncCopy(from_disk, it->path(), to_disk, dest, pool, results, true, read_settings, write_settings); + asyncCopy(from_disk, it->path(), to_disk, dest, pool, results, true, read_settings, write_settings, cancellation_hook); } } -void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr & to_disk, const String & to_path, bool copy_root_dir, const ReadSettings & read_settings, WriteSettings write_settings) +void IDisk::copyThroughBuffers( + const String & from_path, + const std::shared_ptr & to_disk, + const String & to_path, + bool copy_root_dir, + const ReadSettings & read_settings, + WriteSettings write_settings, + const std::function & cancellation_hook) { ResultsCollector results; @@ -131,21 +155,27 @@ void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) +void IDisk::copyDirectoryContent( + const String & from_dir, + const std::shared_ptr & to_disk, + const String & to_dir, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook) { if (!to_disk->exists(to_dir)) to_disk->createDirectories(to_dir); - copyThroughBuffers(from_dir, to_disk, to_dir, /* copy_root_dir= */ false, read_settings, write_settings); + copyThroughBuffers(from_dir, to_disk, to_dir, /* copy_root_dir= */ false, read_settings, write_settings, cancellation_hook); } void IDisk::truncateFile(const String &, size_t) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 6911fd86db2..b2d42f2e732 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -193,7 +193,13 @@ public: virtual void replaceFile(const String & from_path, const String & to_path) = 0; /// Recursively copy files from from_dir to to_dir. Create to_dir if not exists. - virtual void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings); + virtual void copyDirectoryContent( + const String & from_dir, + const std::shared_ptr & to_disk, + const String & to_dir, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook = {}); /// Copy file `from_file_path` to `to_file_path` located at `to_disk`. virtual void copyFile( /// NOLINT @@ -201,7 +207,8 @@ public: IDisk & to_disk, const String & to_file_path, const ReadSettings & read_settings = {}, - const WriteSettings & write_settings = {}); + const WriteSettings & write_settings = {}, + const std::function & cancellation_hook = {}); /// List files at `path` and add their names to `file_names` virtual void listFiles(const String & path, std::vector & file_names) const = 0; @@ -473,7 +480,14 @@ protected: /// Base implementation of the function copy(). /// It just opens two files, reads data by portions from the first file, and writes it to the second one. /// A derived class may override copy() to provide a faster implementation. - void copyThroughBuffers(const String & from_path, const std::shared_ptr & to_disk, const String & to_path, bool copy_root_dir, const ReadSettings & read_settings, WriteSettings write_settings); + void copyThroughBuffers( + const String & from_path, + const std::shared_ptr & to_disk, + const String & to_path, + bool copy_root_dir, + const ReadSettings & read_settings, + WriteSettings write_settings, + const std::function & cancellation_hook); virtual void checkAccessImpl(const String & path); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index c1f053be7c6..739b737fe8f 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -175,7 +175,9 @@ void DiskObjectStorage::copyFile( /// NOLINT IDisk & to_disk, const String & to_file_path, const ReadSettings & read_settings, - const WriteSettings & write_settings) + const WriteSettings & write_settings, + const std::function & cancellation_hook + ) { if (this == &to_disk) { @@ -187,7 +189,7 @@ void DiskObjectStorage::copyFile( /// NOLINT else { /// Copy through buffers - IDisk::copyFile(from_file_path, to_disk, to_file_path, read_settings, write_settings); + IDisk::copyFile(from_file_path, to_disk, to_file_path, read_settings, write_settings, cancellation_hook); } } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 66d1b02aea7..25b39c4d974 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -163,7 +163,9 @@ public: IDisk & to_disk, const String & to_file_path, const ReadSettings & read_settings = {}, - const WriteSettings & write_settings = {}) override; + const WriteSettings & write_settings = {}, + const std::function & cancellation_hook = {} + ) override; void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &) override; diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 7fc8187aee5..0c7c50a687b 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -470,7 +470,8 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart( const DiskPtr & dst_disk, const ReadSettings & read_settings, const WriteSettings & write_settings, - Poco::Logger * log) const + Poco::Logger * log, + const std::function & cancellation_hook) const { String path_to_clone = fs::path(to) / dir_path / ""; auto src_disk = volume->getDisk(); @@ -485,7 +486,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart( try { dst_disk->createDirectories(to); - src_disk->copyDirectoryContent(getRelativePath(), dst_disk, path_to_clone, read_settings, write_settings); + src_disk->copyDirectoryContent(getRelativePath(), dst_disk, path_to_clone, read_settings, write_settings, cancellation_hook); } catch (...) { diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 1826e84c28d..5792e163856 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -74,7 +74,9 @@ public: const DiskPtr & dst_disk, const ReadSettings & read_settings, const WriteSettings & write_settings, - Poco::Logger * log) const override; + Poco::Logger * log, + const std::function & cancellation_hook = {} + ) const override; void rename( std::string new_root_path, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 072cb29626e..e91a35974e3 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -262,7 +262,9 @@ public: const DiskPtr & disk, const ReadSettings & read_settings, const WriteSettings & write_settings, - Poco::Logger * log) const = 0; + Poco::Logger * log, + const std::function & cancellation_hook = {} + ) const = 0; /// Change part's root. from_root should be a prefix path of current root path. /// Right now, this is needed for rename table query. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 9bc72577b25..23410b5fc5c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1798,7 +1798,12 @@ DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix params); } -MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const ReadSettings & read_settings, const WriteSettings & write_settings) const +MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk( + const DiskPtr & disk, + const String & directory_name, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook) const { assertOnDisk(); @@ -1808,7 +1813,7 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & di throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not clone data part {} to empty directory.", name); String path_to_clone = fs::path(storage.relative_data_path) / directory_name / ""; - return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log); + return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log, cancellation_hook); } UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a9659d2f5f4..752a6de39cf 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -380,7 +380,12 @@ public: const DiskTransactionPtr & disk_transaction) const; /// Makes full clone of part in specified subdirectory (relative to storage data directory, e.g. "detached") on another disk - MutableDataPartStoragePtr makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const ReadSettings & read_settings, const WriteSettings & write_settings) const; + MutableDataPartStoragePtr makeCloneOnDisk( + const DiskPtr & disk, + const String & directory_name, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook = {}) const; /// Checks that .bin and .mrk files exist. /// diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 0a3416a2700..d32bc6d1826 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -210,8 +210,12 @@ bool MergeTreePartsMover::selectPartsForMove( MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const MergeTreeMoveEntry & moving_part, const ReadSettings & read_settings, const WriteSettings & write_settings) const { - if (moves_blocker.isCancelled()) - throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts."); + auto cancellation_hook = [&moves_blocker_ = moves_blocker]() + { + if (moves_blocker_.isCancelled()) + throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts."); + }; + cancellation_hook(); auto settings = data->getSettings(); auto part = moving_part.part; @@ -255,12 +259,12 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me { LOG_INFO(log, "Part {} was not fetched, we are the first who move it to another disk, so we will copy it", part->name); cloned_part_storage = part->getDataPartStorage().clonePart( - path_to_clone, part->getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, log); + path_to_clone, part->getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, log, cancellation_hook); } } else { - cloned_part_storage = part->makeCloneOnDisk(disk, MergeTreeData::MOVING_DIR_NAME, read_settings, write_settings); + cloned_part_storage = part->makeCloneOnDisk(disk, MergeTreeData::MOVING_DIR_NAME, read_settings, write_settings, cancellation_hook); } MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage); diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index c1c076277bb..7640ff8f737 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1,5 +1,5 @@ +import inspect import random -import string import threading import time from multiprocessing.dummy import Pool @@ -8,6 +8,8 @@ from helpers.test_tools import assert_logs_contain_with_retry import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry # FIXME: each sleep(1) is a time bomb, and not only this cause false positive # it also makes the test not reliable (i.e. assertions may be wrong, due timing issues) @@ -26,6 +28,7 @@ node1 = cluster.add_instance( with_zookeeper=True, tmpfs=["/jbod1:size=40M", "/jbod2:size=40M", "/external:size=200M"], macros={"shard": 0, "replica": 1}, + stay_alive=True, ) node2 = cluster.add_instance( @@ -1813,3 +1816,114 @@ def test_ttl_move_if_exists(started_cluster, name, dest_type): node2.query("DROP TABLE IF EXISTS {} SYNC".format(name)) except: pass + + +class TestCancelBackgroundMoving: + @pytest.fixture() + def prepare_table(self, request, started_cluster): + name = unique_table_name(request.node.name) + engine = f"ReplicatedMergeTree('/clickhouse/{name}', '1')" + + node1.query( + f""" + CREATE TABLE {name} ( + s1 String, + d1 DateTime + ) ENGINE = {engine} + ORDER BY tuple() + TTL d1 + interval 5 second TO DISK 'external' + SETTINGS storage_policy='small_jbod_with_external' + """ + ) + + node1.query("SYSTEM STOP MOVES") + + # Insert part which is about to move + node1.query( + "INSERT INTO {} (s1, d1) VALUES (randomPrintableASCII({}), toDateTime({}))".format( + name, 10 * 1024 * 1024, time.time() + ) + ) + + # Set low bandwidth to have enough time to cancel part moving + config = inspect.cleandoc( + f""" + + { 256 * 1024 } + + """ + ) + node1.replace_config( + "/etc/clickhouse-server/config.d/disk_throttling.xml", config + ) + node1.restart_clickhouse() + + try: + yield name + finally: + node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + + def test_cancel_background_moving_on_stop_moves_query(self, prepare_table): + name = prepare_table + + # Wait for background moving task to be started + node1.query("SYSTEM START MOVES") + assert_eq_with_retry( + node1, + f"SELECT count() FROM system.moves WHERE table = '{name}'".strip(), + "1", + ) + + # Wait for background moving task to be cancelled + node1.query("SYSTEM STOP MOVES") + assert_logs_contain_with_retry( + node1, "MergeTreeBackgroundExecutor.*Cancelled moving parts" + ) + assert_eq_with_retry( + node1, + f"SELECT count() FROM system.moves WHERE table = '{name}'".strip(), + "0", + ) + + # Ensure that part was not moved + assert set(get_used_disks_for_table(node1, name)) == {"jbod1"} + + def test_cancel_background_moving_on_table_detach(self, prepare_table): + name = prepare_table + + # Wait for background moving task to be started + node1.query("SYSTEM START MOVES") + assert_eq_with_retry( + node1, + f"SELECT count() FROM system.moves WHERE table = '{name}'".strip(), + "1", + ) + + # Wait for background moving task to be cancelled + node1.query(f"DETACH Table {name}") + assert_logs_contain_with_retry( + node1, "MergeTreeBackgroundExecutor.*Cancelled moving parts" + ) + assert_eq_with_retry( + node1, + f"SELECT count() FROM system.moves WHERE table = '{name}'".strip(), + "0", + ) + + def test_cancel_background_moving_on_zookeeper_disconnect(self, prepare_table): + name = prepare_table + + # Wait for background moving task to be started + node1.query("SYSTEM START MOVES") + assert_eq_with_retry( + node1, + f"SELECT count() FROM system.moves WHERE table = '{name}'".strip(), + "1", + ) + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node1) + # Wait for background moving task to be cancelled + assert_logs_contain_with_retry( + node1, "MergeTreeBackgroundExecutor.*Cancelled moving parts" + ) From ee262be6c615f18182b4f26f2956b4c1eb834cb3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Nov 2023 10:59:17 +0000 Subject: [PATCH 433/813] Update comment in universal.sh --- docs/_includes/install/universal.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index 0ae77f464eb..d474aa98e76 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -20,9 +20,9 @@ then fi elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ] then - # If the system has >=ARMv8.2 (https://en.wikipedia.org/wiki/AArch64), choose the corresponding build, else fall back to a v8.0 - # compat build. Unfortunately, the ARM ISA level cannot be read directly, we need to guess from the "features" in /proc/cpuinfo. - # Also, the flags in /proc/cpuinfo are named differently than the flags passed to the compiler (cmake/cpu_features.cmake). + # Dispatch between standard and compatibility builds, see cmake/cpu_features.cmake for details. Unfortunately, (1) the ARM ISA level + # cannot be read directly, we need to guess from the "features" in /proc/cpuinfo, and (2) the flags in /proc/cpuinfo are named + # differently than the flags passed to the compiler in cpu_features.cmake. HAS_ARMV82=$(grep -m 1 'Features' /proc/cpuinfo | awk '/asimd/ && /sha1/ && /aes/ && /atomics/ && /lrcpc/') if [ "${HAS_ARMV82}" ] then From 649d734409fab1eb602026cd1e39b601a0b1673c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 10:13:07 +0000 Subject: [PATCH 434/813] Bump gRPC to v1.56.3 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index a08fe1a3407..bc110c3dc91 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit a08fe1a34075c93bb2d606dd608b9a3953288b81 +Subproject commit bc110c3dc91b77d1e54957871df54fd39f2a49d1 From a49db81b9f10aacee9528a02bea0f7b57dc532a7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 10:15:11 +0000 Subject: [PATCH 435/813] Bump gRPC to v1.57.1 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index bc110c3dc91..fd802577cc0 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit bc110c3dc91b77d1e54957871df54fd39f2a49d1 +Subproject commit fd802577cc06226428c99297d5be3a24f5e3ab96 From 1ba408eb0b2f37e2978376920920c7863a0325b8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 10:38:03 +0000 Subject: [PATCH 436/813] Bump gRPC to v1.58.2 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index fd802577cc0..2e45a02f2b2 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit fd802577cc06226428c99297d5be3a24f5e3ab96 +Subproject commit 2e45a02f2b24e3cc455d1793a469e1dbba894f94 From a250c2bb08eda91f50eebd5bffd5086ef3251b9f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 10:42:41 +0000 Subject: [PATCH 437/813] Bump gRPC to v1.59.2 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 2e45a02f2b2..740e3dfd973 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 2e45a02f2b24e3cc455d1793a469e1dbba894f94 +Subproject commit 740e3dfd97301a52ad8165b65285bcc149d9e817 From ac7fd357e4f75d9b16023450d8fe5df6f3e68cfa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 15 Nov 2023 10:47:10 +0000 Subject: [PATCH 438/813] Bump protobuf to v23.2 --- contrib/google-protobuf | 2 +- contrib/google-protobuf-cmake/CMakeLists.txt | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index 089b89c8d41..5b179151990 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit 089b89c8d4140f0d49fe4222b047a8ea814bc752 +Subproject commit 5b1791519907360781cfe3bebe1c79e5b1b0bcba diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index f6955a3d8ce..3b53ac822da 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -174,6 +174,8 @@ set(libprotobuf_files ${protobuf_source_dir}/src/google/protobuf/message.cc ${protobuf_source_dir}/src/google/protobuf/message_lite.cc ${protobuf_source_dir}/src/google/protobuf/parse_context.cc + ${protobuf_source_dir}/src/google/protobuf/port.cc + ${protobuf_source_dir}/src/google/protobuf/reflection_mode.cc ${protobuf_source_dir}/src/google/protobuf/reflection_ops.cc ${protobuf_source_dir}/src/google/protobuf/repeated_field.cc ${protobuf_source_dir}/src/google/protobuf/repeated_ptr_field.cc @@ -213,6 +215,7 @@ set(libprotoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/enum.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/extension.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/cord_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/enum_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/map_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field_generators/message_field.cc @@ -299,6 +302,13 @@ set(libprotoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/python/pyi_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/retention.cc ${protobuf_source_dir}/src/google/protobuf/compiler/ruby/ruby_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/rust/accessors/accessors.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/rust/accessors/singular_bytes.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/rust/accessors/singular_scalar.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/rust/context.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/rust/generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/rust/message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/rust/naming.cc ${protobuf_source_dir}/src/google/protobuf/compiler/subprocess.cc ${protobuf_source_dir}/src/google/protobuf/compiler/zip_writer.cc ) From ebd42187ad12ce2be24833820f59bb3d81def382 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 16 Nov 2023 12:29:15 +0100 Subject: [PATCH 439/813] Update tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml Co-authored-by: Nikita Taranov --- .../configs/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml index f51b854de75..98c6f551be6 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml @@ -34,7 +34,7 @@ true 1 - 0 + 0 1 20000 From 7d37c0e07073b2a1909e80c4aea45fdc4a35be75 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 16 Nov 2023 12:29:21 +0100 Subject: [PATCH 440/813] Update tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml Co-authored-by: Nikita Taranov --- .../configs/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml index 98c6f551be6..6303e9273fc 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml @@ -11,7 +11,7 @@ true 0 - 0 + 0 20000 From 4a1e207e7a5b02da2b2f6ea46edecd9fe6a9185c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 16 Nov 2023 12:31:00 +0100 Subject: [PATCH 441/813] review notes --- base/poco/Net/src/HTTPSession.cpp | 34 +++++++++++++++++++++---------- src/IO/S3/PocoHTTPClient.cpp | 20 ++++++++++++++---- src/IO/S3/PocoHTTPClient.h | 6 +++--- 3 files changed, 42 insertions(+), 18 deletions(-) diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 9ebbd7d04cd..d303a4c654b 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -93,22 +93,34 @@ void HTTPSession::setTimeout(const Poco::Timespan& timeout) void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco::Timespan& sendTimeout, const Poco::Timespan& receiveTimeout) { - _connectionTimeout = connectionTimeout; - - if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) + try { - _sendTimeout = sendTimeout; + _connectionTimeout = connectionTimeout; - if (connected()) - _socket.setSendTimeout(_sendTimeout); + if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) { + _sendTimeout = sendTimeout; + + if (connected()) + _socket.setSendTimeout(_sendTimeout); + } + + if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) { + _receiveTimeout = receiveTimeout; + + if (connected()) + _socket.setReceiveTimeout(_receiveTimeout); + } } - - if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) + catch (NetException &) { - _receiveTimeout = receiveTimeout; - if (connected()) - _socket.setReceiveTimeout(_receiveTimeout); +#ifndef NDEBUG + // mute exceptions in release + // just in case when changing settings on socket is not allowed + // however it should be OK for timeouts +#else + throw; +#endif } } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index f681362e607..4a1b6def133 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -407,17 +407,29 @@ void PocoHTTPClient::makeRequestInternalImpl( /// This can lead to request signature difference on S3 side. if constexpr (pooled) session = makePooledHTTPSession( - target_uri, getTimeouts(method, first_attempt), http_connection_pool_size, wait_on_pool_size_limit, proxy_configuration); + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true), + http_connection_pool_size, + wait_on_pool_size_limit, + proxy_configuration); else - session = makeHTTPSession(target_uri, getTimeouts(method, first_attempt), proxy_configuration); + session = makeHTTPSession( + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true), + proxy_configuration); } else { if constexpr (pooled) session = makePooledHTTPSession( - target_uri, getTimeouts(method, first_attempt), http_connection_pool_size, wait_on_pool_size_limit); + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true), + http_connection_pool_size, + wait_on_pool_size_limit); else - session = makeHTTPSession(target_uri, getTimeouts(method, first_attempt)); + session = makeHTTPSession( + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true)); } /// In case of error this address will be written to logs diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 14c4fec5dd7..5178d75e7b6 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -55,7 +55,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration size_t http_connection_pool_size = 0; /// See PoolBase::BehaviourOnLimit bool wait_on_pool_size_limit = true; - bool s3_use_adaptive_timeouts = false; + bool s3_use_adaptive_timeouts = true; std::function error_report; @@ -171,7 +171,7 @@ private: Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; - ConnectionTimeouts getTimeouts(const String & method, bool first_attempt, bool first_byte = true) const; + ConnectionTimeouts getTimeouts(const String & method, bool first_attempt, bool first_byte) const; protected: static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); @@ -182,7 +182,7 @@ protected: ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; - bool s3_use_adaptive_timeouts = false; + bool s3_use_adaptive_timeouts = true; bool enable_s3_requests_logging; bool for_disk_s3; From e53e723be81d9ba76595afa7bbe67c1bf8764c4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 16 Nov 2023 12:32:42 +0100 Subject: [PATCH 442/813] Apply same improvement to initializeAggregation --- src/Functions/initializeAggregation.cpp | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Functions/initializeAggregation.cpp b/src/Functions/initializeAggregation.cpp index 83df28808a1..eeeb03aeb30 100644 --- a/src/Functions/initializeAggregation.cpp +++ b/src/Functions/initializeAggregation.cpp @@ -141,10 +141,19 @@ ColumnPtr FunctionInitializeAggregation::executeImpl(const ColumnsWithTypeAndNam that->addBatch(0, input_rows_count, places.data(), 0, aggregate_arguments, arena.get()); } - for (size_t i = 0; i < input_rows_count; ++i) + if (agg_func.isState()) + { /// We should use insertMergeResultInto to insert result into ColumnAggregateFunction /// correctly if result contains AggregateFunction's states - agg_func.insertMergeResultInto(places[i], res_col, arena.get()); + for (size_t i = 0; i < input_rows_count; ++i) + agg_func.insertMergeResultInto(places[i], res_col, arena.get()); + } + else + { + for (size_t i = 0; i < input_rows_count; ++i) + agg_func.insertResultInto(places[i], res_col, arena.get()); + } + return result_holder; } From 84656ed9af9aee3dee1296038bdfdac948333ced Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 16 Nov 2023 11:38:32 +0000 Subject: [PATCH 443/813] track if dashboard was customized, show with opacity --- programs/server/dashboard.html | 29 ++++++++++++++++++++++------- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 72df69ed809..3ed37434235 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -514,6 +514,7 @@ const errorMessages = [ /// Query to fill `queries` list for the dashboard let search_query = `SELECT title, query FROM system.dashboards WHERE dashboard = 'overview'`; +let customized = false; let queries = []; /// Query parameters with predefined default values. @@ -659,9 +660,10 @@ function insertChart(i) { title_text.data = ''; findParamsInQuery(q.query, params); buildParams(); + refreshCustomized(true); + saveState(); const idx = getCurrentIndex(); draw(idx, chart, getParamsForURL(), q.query); - saveState(); } query_editor_confirm.addEventListener('click', editConfirm); @@ -809,6 +811,7 @@ function insertChart(i) { findParamsInQueries(); buildParams(); resize(); + refreshCustomized(true); saveState(); }); @@ -874,8 +877,9 @@ function massEditorApplyChanges() { ({params, queries} = JSON.parse(editor.value)); hideMassEditor(); regenerate(); - drawAll(); + refreshCustomized(true); saveState(); + drawAll(); } document.getElementById('edit').addEventListener('click', e => { @@ -1182,7 +1186,11 @@ async function reloadAll(do_search) { disableButtons(); try { updateParams(); - search_query = document.getElementById('search-query').value; + if (do_search) { + search_query = document.getElementById('search-query').value; + queries = []; + refreshCustomized(false); + } saveState(); if (do_search) { await searchQueries(); @@ -1202,7 +1210,7 @@ document.getElementById('params').onsubmit = function(event) { function saveState() { - const state = { host: host, user: user, queries: queries, params: params, search_query: search_query }; + const state = { host, user, queries, params, search_query, customized }; history.pushState(state, '', window.location.pathname + (window.location.search || '') + '#' + btoa(JSON.stringify(state))); } @@ -1222,7 +1230,6 @@ async function searchQueries() { throw new Error("Wrong data format of the search query."); } - queries = []; for (let i = 0; i < data[0].length; i++) { queries.push({title: data[0][i], query: data[1][i]}); } @@ -1230,11 +1237,19 @@ async function searchQueries() { regenerate(); } +function refreshCustomized(value) { + if (value !== undefined) { + customized = value; + } + document.getElementById('search-span').style.opacity = customized ? 0.5 : 1.0; +} + function regenerate() { document.getElementById('url').value = host; document.getElementById('user').value = user; document.getElementById('password').value = password; document.getElementById('search-query').value = search_query; + refreshCustomized(); findParamsInQueries(); buildParams(); @@ -1253,7 +1268,7 @@ function regenerate() { window.onpopstate = function(event) { if (!event.state) { return; } - ({host, user, queries, params, search_query} = event.state); + ({host, user, queries, params, search_query, customized} = event.state); regenerate(); drawAll(); @@ -1261,7 +1276,7 @@ window.onpopstate = function(event) { if (window.location.hash) { try { - ({host, user, queries, params, search_query} = JSON.parse(atob(window.location.hash.substring(1)))); + ({host, user, queries, params, search_query, customized} = JSON.parse(atob(window.location.hash.substring(1)))); } catch {} } From a0934253deff176da4c2eb21844cf31f3f7a3c61 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Nov 2023 11:39:36 +0000 Subject: [PATCH 444/813] Bump protobuf to v24.4 --- contrib/google-protobuf | 2 +- contrib/google-protobuf-cmake/CMakeLists.txt | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index 5b179151990..0862007f6ca 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit 5b1791519907360781cfe3bebe1c79e5b1b0bcba +Subproject commit 0862007f6ca1f5723c58f10f0ca34f3f25a63b2e diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 3b53ac822da..1ed4133270b 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -82,7 +82,6 @@ set(libprotobuf_lite_files ${protobuf_source_dir}/src/google/protobuf/any_lite.cc ${protobuf_source_dir}/src/google/protobuf/arena.cc ${protobuf_source_dir}/src/google/protobuf/arena_align.cc - ${protobuf_source_dir}/src/google/protobuf/arena_config.cc ${protobuf_source_dir}/src/google/protobuf/arenastring.cc ${protobuf_source_dir}/src/google/protobuf/arenaz_sampler.cc ${protobuf_source_dir}/src/google/protobuf/extension_set.cc @@ -131,17 +130,18 @@ set(libprotobuf_files ${protobuf_source_dir}/src/google/protobuf/any_lite.cc ${protobuf_source_dir}/src/google/protobuf/arena.cc ${protobuf_source_dir}/src/google/protobuf/arena_align.cc - ${protobuf_source_dir}/src/google/protobuf/arena_config.cc ${protobuf_source_dir}/src/google/protobuf/arenastring.cc ${protobuf_source_dir}/src/google/protobuf/arenaz_sampler.cc ${protobuf_source_dir}/src/google/protobuf/compiler/importer.cc ${protobuf_source_dir}/src/google/protobuf/compiler/parser.cc + ${protobuf_source_dir}/src/google/protobuf/cpp_features.pb.cc ${protobuf_source_dir}/src/google/protobuf/descriptor.cc ${protobuf_source_dir}/src/google/protobuf/descriptor.pb.cc ${protobuf_source_dir}/src/google/protobuf/descriptor_database.cc ${protobuf_source_dir}/src/google/protobuf/dynamic_message.cc ${protobuf_source_dir}/src/google/protobuf/extension_set.cc ${protobuf_source_dir}/src/google/protobuf/extension_set_heavy.cc + ${protobuf_source_dir}/src/google/protobuf/feature_resolver.cc ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_bases.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc @@ -175,6 +175,7 @@ set(libprotobuf_files ${protobuf_source_dir}/src/google/protobuf/message_lite.cc ${protobuf_source_dir}/src/google/protobuf/parse_context.cc ${protobuf_source_dir}/src/google/protobuf/port.cc + ${protobuf_source_dir}/src/google/protobuf/raw_ptr.cc ${protobuf_source_dir}/src/google/protobuf/reflection_mode.cc ${protobuf_source_dir}/src/google/protobuf/reflection_ops.cc ${protobuf_source_dir}/src/google/protobuf/repeated_field.cc @@ -210,6 +211,7 @@ add_library(protobuf::libprotobuf ALIAS _libprotobuf) set(libprotoc_files + ${protobuf_source_dir}/src/google/protobuf/compiler/allowlists/editions.cc ${protobuf_source_dir}/src/google/protobuf/compiler/code_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/command_line_interface.cc ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/enum.cc From a0840d36afbddad03c06c54fca17632917bd4d7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 16 Nov 2023 12:48:27 +0100 Subject: [PATCH 445/813] Apply the same to arrayReduce --- src/Functions/array/arrayReduce.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayReduce.cpp b/src/Functions/array/arrayReduce.cpp index ea087f4f9a8..46777ceb05c 100644 --- a/src/Functions/array/arrayReduce.cpp +++ b/src/Functions/array/arrayReduce.cpp @@ -182,10 +182,19 @@ ColumnPtr FunctionArrayReduce::executeImpl(const ColumnsWithTypeAndName & argume that->addBatchArray(0, input_rows_count, places.data(), 0, aggregate_arguments, offsets->data(), arena.get()); } - for (size_t i = 0; i < input_rows_count; ++i) - /// We should use insertMergeResultInto to insert result into ColumnAggregateFunction - /// correctly if result contains AggregateFunction's states - agg_func.insertMergeResultInto(places[i], res_col, arena.get()); + if (agg_func.isState()) + { + for (size_t i = 0; i < input_rows_count; ++i) + /// We should use insertMergeResultInto to insert result into ColumnAggregateFunction + /// correctly if result contains AggregateFunction's states + agg_func.insertMergeResultInto(places[i], res_col, arena.get()); + } + else + { + for (size_t i = 0; i < input_rows_count; ++i) + agg_func.insertResultInto(places[i], res_col, arena.get()); + } + return result_holder; } From e8c14562ab512d5704f2ec05f8e053346c6737fd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Nov 2023 11:50:25 +0000 Subject: [PATCH 446/813] Bump absl to HEAD --- contrib/abseil-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 5655528c418..3bd86026c93 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 5655528c41830f733160de4fb0b99073841bae9e +Subproject commit 3bd86026c93da5a40006fd53403dff9d5f5e30e3 From 504aeb987b220372f4a06a6e1dc8400d80eb3201 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Nov 2023 12:13:13 +0000 Subject: [PATCH 447/813] Better messages --- src/Compression/CompressionCodecFPC.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Compression/CompressionCodecFPC.cpp b/src/Compression/CompressionCodecFPC.cpp index 5763d929f6c..2a6dfda7b5c 100644 --- a/src/Compression/CompressionCodecFPC.cpp +++ b/src/Compression/CompressionCodecFPC.cpp @@ -416,7 +416,7 @@ private: std::to_integer(bytes.front()) & MAX_ZERO_BYTE_COUNT); if (zero_byte_count1 > VALUE_SIZE || zero_byte_count2 > VALUE_SIZE) [[unlikely]] - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Invalid compressed data"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Invalid zero byte count(s): {} and {}", zero_byte_count1, zero_byte_count2); size_t tail_size1 = VALUE_SIZE - zero_byte_count1; size_t tail_size2 = VALUE_SIZE - zero_byte_count2; @@ -424,7 +424,7 @@ private: size_t expected_size = 0; if (__builtin_add_overflow(tail_size1, tail_size2, &expected_size) || __builtin_add_overflow(expected_size, 1, &expected_size)) [[unlikely]] - throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Invalid compressed data"); + throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Overflow occurred while calculating expected size"); if (bytes.size() < expected_size) [[unlikely]] throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Unexpected end of encoded sequence"); From 74ee26752497259141630411a7d66b8ff8e8f4c1 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 16 Nov 2023 12:17:22 +0000 Subject: [PATCH 448/813] refactoring: error is now used not only for auth --- programs/server/dashboard.html | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 3ed37434235..2440f87abcb 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -14,7 +14,7 @@ --moving-shadow-color: rgba(0, 0, 0, 0.5); --input-shadow-color: rgba(0, 255, 0, 1); --error-color: red; - --auth-error-color: white; + --global-error-color: white; --legend-background: rgba(255, 255, 255, 0.75); --title-color: #666; --text-color: black; @@ -277,12 +277,12 @@ } - #auth-error { + #global-error { align-self: center; width: 60%; padding: .5rem; - color: var(--auth-error-color); + color: var(--global-error-color); display: flex; flex-flow: row nowrap; @@ -426,7 +426,7 @@ } #mass-editor-message { - color: var(--auth-error-color); + color: var(--global-error-color); } #charts > div:only-child .display-only-if-more-than-one-chart { @@ -458,7 +458,7 @@
-
+
@@ -1079,26 +1079,26 @@ async function draw(idx, chart, url_params, query) { return true; } -function showAuthError(message) { +function showError(message) { const charts = document.getElementById('charts'); charts.style.height = '0px'; charts.style.opacity = '0'; document.getElementById('add').style.display = 'none'; document.getElementById('edit').style.display = 'none'; - const authError = document.getElementById('auth-error'); - authError.textContent = message; - authError.style.display = 'flex'; + const error = document.getElementById('global-error'); + error.textContent = message; + error.style.display = 'flex'; } -function hideAuthError() { +function hideError() { const charts = document.getElementById('charts'); charts.style.height = 'auto'; charts.style.opacity = '1'; - const authError = document.getElementById('auth-error'); - authError.textContent = ''; - authError.style.display = 'none'; + const error = document.getElementById('global-error'); + error.textContent = ''; + error.style.display = 'none'; } let firstLoad = true; @@ -1108,12 +1108,12 @@ async function drawAll() { const chartsArray = document.getElementsByClassName('chart'); if (!firstLoad) { - hideAuthError(); + hideError(); } await Promise.all([...Array(queries.length)].map(async (_, i) => { return draw(i, chartsArray[i], params, queries[i].query).catch((e) => { if (!firstLoad) { - showAuthError(e.message); + showError(e.message); } return false; }); @@ -1197,7 +1197,7 @@ async function reloadAll(do_search) { } await drawAll(); } catch (e) { - showAuthError(e.toString()); + showError(e.toString()); } enableButtons(); } @@ -1295,7 +1295,7 @@ async function start() { drawAll(); } } catch (e) { - showAuthError(e.toString()); + showError(e.toString()); } } From 3920926cd614385237731394721991144f24997e Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 16 Nov 2023 12:40:15 +0000 Subject: [PATCH 449/813] add docs --- .../en/operations/system-tables/dashboards.md | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 docs/en/operations/system-tables/dashboards.md diff --git a/docs/en/operations/system-tables/dashboards.md b/docs/en/operations/system-tables/dashboards.md new file mode 100644 index 00000000000..1d6876b9f8d --- /dev/null +++ b/docs/en/operations/system-tables/dashboards.md @@ -0,0 +1,68 @@ +--- +slug: /en/operations/system-tables/dashboards +--- +# dashboards + +Contains queries used by `/dashboard` page accessible though [HTTP interface](/docs/en/interfaces/http.md). +This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard. + +:::note +`/dashboard` page can render queries not only from `system.dashboards`, but from any table with the same schema. +This can be useful to create custom dashboards. +::: + +Example: + +``` sql +SELECT * +FROM system.dashboards +WHERE title ILIKE '%CPU%' +``` + +``` text +Row 1: +────── +dashboard: overview +title: CPU Usage (cores) +query: SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSCPUVirtualTimeMicroseconds) / 1000000 +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} + +Row 2: +────── +dashboard: overview +title: CPU Wait +query: SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSCPUWaitMicroseconds) / 1000000 +FROM system.metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} + +Row 3: +────── +dashboard: overview +title: OS CPU Usage (Userspace) +query: SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) +FROM system.asynchronous_metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSUserTimeNormalized' +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} + +Row 4: +────── +dashboard: overview +title: OS CPU Usage (Kernel) +query: SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value) +FROM system.asynchronous_metric_log +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSSystemTimeNormalized' +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} +``` + +Columns: + +- `dashboard` (`String`) - The dashboard name. +- `title` (`String`) - The title of a chart. +- `query` (`String`) - The query to obtain data to be displayed. From 4d16c096a1c7464573c5101fd9068b9d451492dc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 16 Nov 2023 12:09:13 +0000 Subject: [PATCH 450/813] Use ports from cluster --- tests/integration/helpers/cluster.py | 2 ++ tests/integration/test_storage_rabbitmq/test.py | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 729b30ba934..cbc511628f0 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -583,6 +583,7 @@ class ClickHouseCluster: self.rabbitmq_host = "rabbitmq1" self.rabbitmq_ip = None self.rabbitmq_port = 5672 + self.rabbitmq_secure_port = 5671 self.rabbitmq_dir = p.abspath(p.join(self.instances_dir, "rabbitmq")) self.rabbitmq_cookie_file = os.path.join(self.rabbitmq_dir, "erlang.cookie") self.rabbitmq_logs_dir = os.path.join(self.rabbitmq_dir, "logs") @@ -1316,6 +1317,7 @@ class ClickHouseCluster: self.with_rabbitmq = True env_variables["RABBITMQ_HOST"] = self.rabbitmq_host env_variables["RABBITMQ_PORT"] = str(self.rabbitmq_port) + env_variables["RABBITMQ_SECURE_PORT"] = str(self.rabbitmq_secure_port) env_variables["RABBITMQ_LOGS"] = self.rabbitmq_logs_dir env_variables["RABBITMQ_LOGS_FS"] = "bind" env_variables["RABBITMQ_COOKIE_FILE"] = self.rabbitmq_cookie_file diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index adb7f59769a..021cdf54af9 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -110,9 +110,9 @@ def rabbitmq_setup_teardown(): ], ) def test_rabbitmq_select(rabbitmq_cluster, secure): - port = 5672 + port = cluster.rabbitmq_port if secure: - port = 5671 + port = cluster.rabbitmq_secure_port instance.query( """ From bdf038191ac85bb9f38524a96480bbf6704d3a24 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Nov 2023 13:05:37 +0000 Subject: [PATCH 451/813] better test_keeper_broken_logs --- tests/integration/test_keeper_broken_logs/test.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_keeper_broken_logs/test.py b/tests/integration/test_keeper_broken_logs/test.py index e283d946174..49b8d985ee8 100644 --- a/tests/integration/test_keeper_broken_logs/test.py +++ b/tests/integration/test_keeper_broken_logs/test.py @@ -1,13 +1,7 @@ import pytest from helpers.cluster import ClickHouseCluster import helpers.keeper_utils as keeper_utils -import random -import string -import os import time -from multiprocessing.dummy import Pool -from helpers.network import PartitionManager -from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( @@ -82,6 +76,13 @@ def test_single_node_broken_log(started_cluster): node1_conn.close() node1.stop_clickhouse() + + # wait until cluster stabilizes with a new leader + while not keeper_utils.is_leader( + started_cluster, node2 + ) and not keeper_utils.is_leader(started_cluster, node3): + time.sleep(1) + node1.exec_in_container( [ "truncate", From dc644b96171f376b3ebd6c95bf635643c9c32606 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 16 Nov 2023 13:09:30 +0000 Subject: [PATCH 452/813] add backward compatibility to avoid breaking old URLs on new servers --- programs/server/dashboard.html | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 2440f87abcb..123a15c5706 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -1276,7 +1276,11 @@ window.onpopstate = function(event) { if (window.location.hash) { try { - ({host, user, queries, params, search_query, customized} = JSON.parse(atob(window.location.hash.substring(1)))); + let search_query_, customized_; + ({host, user, queries, params, search_query_, customized_} = JSON.parse(atob(window.location.hash.substring(1)))); + // For compatibility with old URLs' hashes + search_query = search_query_ !== undefined ? search_query_ : search_query; + customized = customized_ !== undefined ? customized_ : true; } catch {} } From bb68321fc153be034fb0e2234b59bf6319cdd281 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 16 Nov 2023 09:02:11 +0000 Subject: [PATCH 453/813] More stable test_keeper_reconfig_replace_leader --- tests/integration/helpers/keeper_utils.py | 95 +++++++++++-------- .../test.py | 7 ++ 2 files changed, 65 insertions(+), 37 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 83d0f2969b7..1ca17e923e4 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -37,39 +37,59 @@ class KeeperException(Exception): class KeeperClient(object): SEPARATOR = b"\a\a\a\a\n" - def __init__(self, bin_path: str, host: str, port: int): + def __init__(self, bin_path: str, host: str, port: int, connection_tries=30): self.bin_path = bin_path self.host = host self.port = port - self.proc = subprocess.Popen( - [ - bin_path, - "keeper-client", - "--host", - host, - "--port", - str(port), - "--log-level", - "error", - "--tests-mode", - "--no-confirmation", - ], - stdin=subprocess.PIPE, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - ) + retry_count = 0 - self.poller = select.epoll() - self.poller.register(self.proc.stdout) - self.poller.register(self.proc.stderr) + while True: + try: + self.proc = subprocess.Popen( + [ + bin_path, + "keeper-client", + "--host", + host, + "--port", + str(port), + "--log-level", + "error", + "--tests-mode", + "--no-confirmation", + ], + stdin=subprocess.PIPE, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + ) - self._fd_nums = { - self.proc.stdout.fileno(): self.proc.stdout, - self.proc.stderr.fileno(): self.proc.stderr, - } + self.poller = select.epoll() + self.poller.register(self.proc.stdout) + self.poller.register(self.proc.stderr) - self.stopped = False + self._fd_nums = { + self.proc.stdout.fileno(): self.proc.stdout, + self.proc.stderr.fileno(): self.proc.stderr, + } + + self.stopped = False + + self.get("/keeper", 60.0) + break + except Exception as e: + retry_count += 1 + if ( + "All connection tries failed while connecting to ZooKeeper" + in str(e) + and retry_count < connection_tries + ): + print( + f"Got exception while connecting to Keeper: {e}\nWill reconnect, reconnect count = {retry_count}" + ) + time.sleep(1) + else: + raise def execute_query(self, query: str, timeout: float = 60.0) -> str: output = io.BytesIO() @@ -94,7 +114,7 @@ class KeeperClient(object): output.write(chunk) elif file == self.proc.stderr: - assert self.proc.stdout.readline() == self.SEPARATOR + self.proc.stdout.readline() raise KeeperException(self.proc.stderr.readline().strip().decode()) else: @@ -221,13 +241,12 @@ NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving request def wait_until_connected(cluster, node, port=9181, timeout=30.0): - elapsed = 0.0 + start = time.time() while send_4lw_cmd(cluster, node, "mntr", port) == NOT_SERVING_REQUESTS_ERROR_MSG: time.sleep(0.1) - elapsed += 0.1 - if elapsed >= timeout: + if time.time() - start > timeout: raise Exception( f"{timeout}s timeout while waiting for {node.name} to start serving requests" ) @@ -280,14 +299,16 @@ def wait_configs_equal(left_config: str, right_zk: KeeperClient, timeout: float Check whether get /keeper/config result in left_config is equal to get /keeper/config on right_zk ZK connection. """ - elapsed: float = 0.0 - while sorted(left_config.split("\n")) != sorted( - get_config_str(right_zk).split("\n") - ): + start = time.time() + left_config = sorted(left_config.split("\n")) + while True: + right_config = sorted(get_config_str(right_zk).split("\n")) + if left_config == right_config: + return + time.sleep(1) - elapsed += 1 - if elapsed >= timeout: + if time.time() - start > timeout: raise Exception( f"timeout while checking nodes configs to get equal. " - f"Left: {left_config}, right: {get_config_str(right_zk)}" + f"Left: {left_config}, right: {right_config}" ) diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py index 4cdd48fcf7c..8e621eef279 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -3,6 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance from os.path import join, dirname, realpath +import time import helpers.keeper_utils as ku import typing as tp @@ -83,6 +84,12 @@ def test_reconfig_replace_leader(started_cluster): assert "node3" in config assert "node4" not in config + # wait until cluster stabilizes with a new leader + while not ku.is_leader(started_cluster, node2) and not ku.is_leader( + started_cluster, node3 + ): + time.sleep(1) + # additional 20s wait before removing leader ku.wait_configs_equal(config, zk2, timeout=50) From 85d363fb285b0b278e8cfc413c98a481d90c3476 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Thu, 16 Nov 2023 14:58:52 +0100 Subject: [PATCH 454/813] Update tests --- .../0_stateless/00727_concat.reference | 6 +++++ tests/queries/0_stateless/00727_concat.sql | 23 ++++++++++++++----- .../0_stateless/02233_interpolate_1.sql | 2 +- .../02389_analyzer_nested_lambda.reference | 2 +- .../02389_analyzer_nested_lambda.sql | 2 +- .../02521_analyzer_array_join_crash.reference | 2 +- .../02521_analyzer_array_join_crash.sql | 2 +- 7 files changed, 28 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00727_concat.reference b/tests/queries/0_stateless/00727_concat.reference index 9b6a8b3857b..7c48ba97c2b 100644 --- a/tests/queries/0_stateless/00727_concat.reference +++ b/tests/queries/0_stateless/00727_concat.reference @@ -25,6 +25,7 @@ With bar With foo With bar With 42 +With 42 With fae310ca-d52a-4923-9e9b-02bf67f4b009 With 2023-11-14 With 2123-11-14 @@ -41,6 +42,11 @@ With (42,43) With [(0,0),(10,0),(10,10),(0,10)] With [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] With [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] +-- SimpleAggregateFunction +With 42 +With 4 +-- Nested +With [(\'foo\',\'qaz\'),(\'bar\',\'qux\')] -- NULL arguments \N \N diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index ba76ff53884..7d901514aea 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -1,8 +1,6 @@ -- Tags: no-fasttest -- no-fasttest: json type needs rapidjson library, geo types need s2 geometry --- not tested here: (Simple)AggregateFunction, Nested - SET allow_experimental_object_type = 1; SET allow_suspicious_low_cardinality_types=1; @@ -33,11 +31,12 @@ SELECT concat('With ', materialize('bar' :: LowCardinality(FixedString(3)))); SELECT concat('With ', materialize('foo' :: LowCardinality(Nullable(String)))); SELECT concat('With ', materialize('bar' :: LowCardinality(Nullable(FixedString(3))))); SELECT concat('With ', materialize(42 :: LowCardinality(Nullable(UInt32)))); +SELECT concat('With ', materialize(42 :: LowCardinality(UInt32))); SELECT concat('With ', materialize('fae310ca-d52a-4923-9e9b-02bf67f4b009' :: UUID)); SELECT concat('With ', materialize('2023-11-14' :: Date)); SELECT concat('With ', materialize('2123-11-14' :: Date32)); -SELECT concat('With ', materialize('2023-11-14 05:50:12' :: DateTime)); -SELECT concat('With ', materialize('2023-11-14 05:50:12.123' :: DateTime64(3))); +SELECT concat('With ', materialize('2023-11-14 05:50:12' :: DateTime('Europe/Amsterdam'))); +SELECT concat('With ', materialize('2023-11-14 05:50:12.123' :: DateTime64(3, 'Europe/Amsterdam'))); SELECT concat('With ', materialize('hallo' :: Enum('hallo' = 1))); SELECT concat('With ', materialize(['foo', 'bar'] :: Array(String))); SELECT concat('With ', materialize('{"foo": "bar"}' :: JSON)); @@ -50,14 +49,23 @@ SELECT concat('With ', materialize([(0,0),(10,0),(10,10),(0,10)] :: Ring)); SELECT concat('With ', materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]] :: Polygon)); SELECT concat('With ', materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]] :: MultiPolygon)); +SELECT '-- SimpleAggregateFunction'; +CREATE OR REPLACE TABLE concat_saf_test(x SimpleAggregateFunction(max, Int32)) ENGINE=MergeTree ORDER BY tuple(); +INSERT INTO concat_saf_test VALUES (42); +INSERT INTO concat_saf_test SELECT max(number) FROM numbers(5); +SELECT concat('With ', x) FROM concat_saf_test ORDER BY x DESC; + +SELECT '-- Nested'; +CREATE OR REPLACE TABLE concat_nested_test(kv Nested(k String, v String)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO concat_nested_test VALUES (['foo', 'bar'], ['qaz', 'qux']); +SELECT concat('With ', kv) FROM concat_nested_test; + SELECT '-- NULL arguments'; SELECT concat(NULL, NULL); SELECT concat(NULL, materialize(NULL :: Nullable(UInt64))); SELECT concat(materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); - SELECT concat(42, materialize(NULL :: Nullable(UInt64))); SELECT concat('42', materialize(NULL :: Nullable(UInt64))); - SELECT concat(42, materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); SELECT concat('42', materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); @@ -72,3 +80,6 @@ SELECT concat(42, 144); SELECT concat(42, 144, 255); SELECT CONCAT('Testing the ', 'alias'); + +SELECT concat(); -- { serverError 42 } +SELECT concat(1); -- { serverError 42 } diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index 3d416b27f45..d589a18421b 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -26,7 +26,7 @@ SELECT n, source, inter FROM ( # Test INTERPOLATE with incompatible expression - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 44 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS reverse(inter)); -- { serverError 44 } # Test INTERPOLATE with column from WITH FILL expression - should produce error SELECT n, source, inter FROM ( diff --git a/tests/queries/0_stateless/02389_analyzer_nested_lambda.reference b/tests/queries/0_stateless/02389_analyzer_nested_lambda.reference index 935c53358c0..68eb282a6a1 100644 --- a/tests/queries/0_stateless/02389_analyzer_nested_lambda.reference +++ b/tests/queries/0_stateless/02389_analyzer_nested_lambda.reference @@ -117,5 +117,5 @@ SELECT arrayMap(x -> concat(concat(concat(concat(concat(toString(id), '___\0____ FROM test_table WHERE concat(concat(concat(toString(id), '___\0_______\0____'), toString(id)), concat(toString(id), NULL), toString(id)); SELECT '--'; -- -SELECT arrayMap(x -> concat(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 }; +SELECT arrayMap(x -> splitByChar(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 }; DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql b/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql index 8f8b5537da9..48e84246d1c 100644 --- a/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql +++ b/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql @@ -122,7 +122,7 @@ FROM test_table WHERE concat(concat(concat(toString(id), '___\0_______\0____'), SELECT '--'; -SELECT arrayMap(x -> concat(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 }; +SELECT arrayMap(x -> splitByChar(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 }; DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02521_analyzer_array_join_crash.reference b/tests/queries/0_stateless/02521_analyzer_array_join_crash.reference index 59da8ccad1a..5e7728e0590 100644 --- a/tests/queries/0_stateless/02521_analyzer_array_join_crash.reference +++ b/tests/queries/0_stateless/02521_analyzer_array_join_crash.reference @@ -8,4 +8,4 @@ SELECT id, value_element, value FROM test_table ARRAY JOIN [[1,2,3]] AS value_el 0 [1,2,3] 3 SELECT value_element, value FROM test_table ARRAY JOIN [1048577] AS value_element, arrayMap(x -> value_element, ['']) AS value; 1048577 [1048577] -SELECT arrayFilter(x -> notEmpty(concat(x)), [NULL, NULL]) FROM system.one ARRAY JOIN [1048577] AS elem, arrayMap(x -> concat(x, elem, ''), ['']) AS unused; -- { serverError 44 } +SELECT arrayFilter(x -> notEmpty(concat(x)), [NULL, NULL]) FROM system.one ARRAY JOIN [1048577] AS elem, arrayMap(x -> splitByChar(x, elem), ['']) AS unused; -- { serverError 44 } diff --git a/tests/queries/0_stateless/02521_analyzer_array_join_crash.sql b/tests/queries/0_stateless/02521_analyzer_array_join_crash.sql index c7641a3bee0..53606e01ab7 100644 --- a/tests/queries/0_stateless/02521_analyzer_array_join_crash.sql +++ b/tests/queries/0_stateless/02521_analyzer_array_join_crash.sql @@ -17,7 +17,7 @@ SELECT id, value_element, value FROM test_table ARRAY JOIN [[1,2,3]] AS value_el SELECT value_element, value FROM test_table ARRAY JOIN [1048577] AS value_element, arrayMap(x -> value_element, ['']) AS value; -SELECT arrayFilter(x -> notEmpty(concat(x)), [NULL, NULL]) FROM system.one ARRAY JOIN [1048577] AS elem, arrayMap(x -> concat(x, elem, ''), ['']) AS unused; -- { serverError 44 } +SELECT arrayFilter(x -> notEmpty(concat(x)), [NULL, NULL]) FROM system.one ARRAY JOIN [1048577] AS elem, arrayMap(x -> splitByChar(x, elem), ['']) AS unused; -- { serverError 44 } -- { echoOff } From ddca2c2187d42b39fb139460cd32eb7e71adbce1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 16 Nov 2023 14:29:53 +0100 Subject: [PATCH 455/813] server side waiting --- src/Storages/MergeTree/DataPartsExchange.cpp | 113 ++++++++---------- src/Storages/MergeTree/MergeTreeSettings.h | 4 +- tests/clickhouse-test | 2 +- ...916_replication_protocol_wait_for_part.sql | 7 +- 4 files changed, 57 insertions(+), 69 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 7fd6f59ed69..c39263a0b73 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -65,7 +65,6 @@ constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_UUID = 5; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_ZERO_COPY = 6; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION = 7; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION = 8; -constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE = 9; std::string getEndpointId(const std::string & node_id) { @@ -121,7 +120,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write MergeTreePartInfo::fromPartName(part_name, data.format_version); /// We pretend to work as older server version, to be sure that client will correctly process our version - response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE))}); + response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION))}); LOG_TRACE(log, "Sending part {}", part_name); @@ -139,29 +138,6 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write { part = findPart(part_name); - /// Ephemeral zero-copy lock may be lost for PreActive parts - /// do not expose PreActive parts - if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE) - { - bool part_is_ready = part->getState() != MergeTreeDataPartState::PreActive; - writeBinary(part_is_ready, out); - - if (!part_is_ready) - { - LOG_TRACE(log, "Part {} is in PreActive state, reply to the client that part is not ready yet", part_name); - return; - } - } - else - { - bool zero_copy_enabled = data.getSettings()->allow_remote_fs_zero_copy_replication; - if (part->getState() == MergeTreeDataPartState::PreActive && zero_copy_enabled) - { - /// report error, client will try again later, error message would be printed - throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in table", part_name); - } - } - CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedSend}; if (part->getDataPartStorage().isStoredOnRemoteDisk()) @@ -373,6 +349,25 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( return data_checksums; } +bool wait_loop(UInt32 wait_timeout_ms, std::function pred) +{ + static const UInt32 loop_delay_ms = 5; + + /// this is sleep-based wait, it has to be short + chassert(wait_timeout_ms < 2000); + + if (pred()) + return true; + + Stopwatch timer; + while (!pred() && timer.elapsedMilliseconds() < wait_timeout_ms) + { + sleepForMilliseconds(loop_delay_ms); + } + + return pred(); +} + MergeTreeData::DataPartPtr Service::findPart(const String & name) { /// It is important to include Outdated parts here because remote replicas cannot reliably @@ -381,10 +376,26 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) part = data.getPartIfExists(name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); - if (part) + if (!part) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in table", name); + + bool zero_copy_enabled = data.getSettings()->allow_remote_fs_zero_copy_replication; + if (!zero_copy_enabled) return part; - throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in table", name); + /// Ephemeral zero-copy lock may be lost for PreActive parts + /// do not expose PreActive parts for zero-copy + + static const UInt32 wait_timeout_ms = 1000; + bool pred_result = wait_loop(wait_timeout_ms, [&] () { return part->getState() != MergeTreeDataPartState::PreActive; }); + + if (!pred_result) + throw Exception( + ErrorCodes::ABORTED, + "Part {} is in PreActive state for {} ms. Another host has to be asked.", + name, wait_timeout_ms); + + return part; } Fetcher::Fetcher(StorageReplicatedMergeTree & data_) @@ -442,7 +453,7 @@ std::pair Fetcher::fetchSelected { {"endpoint", endpoint_id}, {"part", part_name}, - {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE)}, + {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION)}, {"compress", "false"} }); @@ -500,43 +511,17 @@ std::pair Fetcher::fetchSelected creds.setPassword(password); } - std::unique_ptr in; - int server_protocol_version = 0; - bool part_is_ready = true; - - static const UInt32 part_not_ready_attempts = 5; - static const UInt32 wait_sleep_time_ms = 100; - - for (UInt32 attempt = 1; attempt <= part_not_ready_attempts; ++attempt) - { - in = std::make_unique( - uri, - Poco::Net::HTTPRequest::HTTP_POST, - nullptr, - timeouts, - creds, - DBMS_DEFAULT_BUFFER_SIZE, - 0, /* no redirects */ - static_cast(data_settings->replicated_max_parallel_fetches_for_host)); - - server_protocol_version = parse(in->getResponseCookie("server_protocol_version", "0")); - - if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_WAITING_PREACTIVE) - readBinary(part_is_ready, *in); - - if (part_is_ready) - break; - - sleepForMilliseconds(wait_sleep_time_ms); - - if (blocker.isCancelled()) - throw Exception(ErrorCodes::ABORTED, "Fetching of part was cancelled"); - } - - if (!part_is_ready) - throw Exception(ErrorCodes::ABORTED, "Part {} is still not ready in host {} after {} attempts, try another host", - part_name, host, part_not_ready_attempts); + std::unique_ptr in = std::make_unique( + uri, + Poco::Net::HTTPRequest::HTTP_POST, + nullptr, + timeouts, + creds, + DBMS_DEFAULT_BUFFER_SIZE, + 0, /* no redirects */ + static_cast(data_settings->replicated_max_parallel_fetches_for_host)); + int server_protocol_version = parse(in->getResponseCookie("server_protocol_version", "0")); String remote_fs_metadata = parse(in->getResponseCookie("remote_fs_metadata", "")); DiskPtr preffered_disk = disk; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 15c54ee3791..41476bab5b1 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -83,7 +83,7 @@ struct Settings; M(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ M(UInt64, min_delay_to_insert_ms, 10, "Min delay of inserting data into MergeTree table in milliseconds, if there are a lot of unmerged parts in single partition.", 0) \ M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ - 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, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background.", 0) \ M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ \ /* Part removal settings. */ \ @@ -122,7 +122,7 @@ struct Settings; M(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(Milliseconds, wait_for_unique_parts_send_before_shutdown_ms, 0, "Before shutdown table will wait for required amount time for unique parts (exist only on current replica) to be fetched by other replicas (0 means disabled).", 0) \ M(Float, fault_probability_before_part_commit, 0, "For testing. Do not change it.", 0) \ - M(Float, fault_probability_after_part_commit, 0, "For testing. Do not change it.", 0) \ + M(Float, fault_probability_after_part_commit, 0, "For testing. Do not change it.", 0) \ \ /** Check delay of replicas settings. */ \ M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 048f848ff27..053bd040bce 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -674,7 +674,7 @@ class MergeTreeSettingsRandomizer: "replace_long_file_name_to_hash": lambda: random.randint(0, 1), "max_file_name_length": threshold_generator(0.3, 0.3, 0, 128), "sleep_before_commit_local_part_in_replicated_table_ms": threshold_generator( - 0.3, 0.3, 0, 250 + 0.7, 0.7, 0, 100 ), } diff --git a/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql index ed9cfd00b45..97ef33f96e8 100644 --- a/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql +++ b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql @@ -7,7 +7,7 @@ create table tableIn (n int) settings storage_policy='s3_cache', allow_remote_fs_zero_copy_replication=1, - sleep_before_commit_local_part_in_replicated_table_ms=50000; + sleep_before_commit_local_part_in_replicated_table_ms=5000; create table tableOut (n int) engine=ReplicatedMergeTree('/test/02916/{database}/table', '2') order by tuple() @@ -15,9 +15,12 @@ create table tableOut (n int) storage_policy='s3_cache', allow_remote_fs_zero_copy_replication=1; -SET send_logs_level = 'error'; +SET send_logs_level='error'; insert into tableIn values(1); insert into tableIn values(2); system sync replica tableOut; select count() from tableOut; + +drop table tableIn +drop table tableOut From b13cd9792b599d3686a73f9114fadda7d0e090f0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Nov 2023 14:46:57 +0000 Subject: [PATCH 456/813] Fix cross build --- contrib/google-protobuf-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 1ed4133270b..89bdbb89eca 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -369,7 +369,7 @@ else () "-Dprotobuf_BUILD_PROTOC_BINARIES=1" "-DABSL_ROOT_DIR=${abseil_source_dir}" "-DABSL_ENABLE_INSTALL=0" - "${protobuf_source_dir}/cmake" + "${protobuf_source_dir}" WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" COMMAND_ECHO STDOUT) From 7c758d1b2d872df670b547a540f0853cf19436f3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 16 Nov 2023 15:30:16 +0100 Subject: [PATCH 457/813] Define the INIT_ENVIRONMENT through instances tags --- tests/ci/worker/init_runner.sh | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index a2737575955..0b610a108f3 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -5,10 +5,10 @@ The following content is embedded into the s3 object via the script deploy-runner-init.sh {staging,production} with additional helping information -In the `user data` you should define as the following -with appropriate as 'staging' or 'production': +In the `user data` you should define as the following text +between `### COPY BELOW` and `### COPY ABOVE` -### COPY AFTER +### COPY BELOW Content-Type: multipart/mixed; boundary="//" MIME-Version: 1.0 @@ -29,11 +29,14 @@ Content-Transfer-Encoding: 7bit Content-Disposition: attachment; filename="userdata.txt" #!/bin/bash -aws s3 cp s3://github-runners-data/cloud-init/.sh /tmp/cloud-init.sh +INSTANCE_ID=$(ec2metadata --instance-id) +INIT_ENVIRONMENT=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:init-environment'].Value" --output text) +echo "Downloading and using $INIT_ENVIRONMENT cloud-init.sh" +aws s3 cp "s3://github-runners-data/cloud-init/${INIT_ENVIRONMENT:-production}.sh" /tmp/cloud-init.sh chmod 0700 /tmp/cloud-init.sh exec bash /tmp/cloud-init.sh --// -### COPY BEFORE +### COPY ABOVE EOF # THE SCRIPT START From f02ed02e4a4e4f4f90f29f6594c2b7ea2c5a8cee Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 16 Nov 2023 16:35:34 +0100 Subject: [PATCH 458/813] Terminate spot instances older than 1h --- tests/ci/worker/init_runner.sh | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 0b610a108f3..017d847739f 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -130,6 +130,23 @@ terminate_and_exit() { declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh +check_spot_instance_is_old() { + # This function should be executed ONLY BETWEEN runnings. + # It's unsafe to execute while the runner is working! + local LIFE_CYCLE + LIFE_CYCLE=$(curl -s --fail http://169.254.169.254/latest/meta-data/instance-life-cycle) + if [ "$LIFE_CYCLE" == "spot" ]; then + local UPTIME + UPTIME=$(< /proc/uptime) + UPTIME=${UPTIME%%.*} + if (( 3600 < UPTIME )); then + echo "The spot instance has uptime $UPTIME, it's time to shut it down" + return 0 + fi + fi + return 1 +} + check_proceed_spot_termination() { # The function checks and proceeds spot instance termination if exists # The event for spot instance termination @@ -161,6 +178,7 @@ no_terminating_metadata() { # The event for rebalance recommendation. Not strict, so we have some room to make a decision here if curl -s --fail http://169.254.169.254/latest/meta-data/events/recommendations/rebalance; then echo 'Received recommendation to rebalance, checking the uptime' + local UPTIME UPTIME=$(< /proc/uptime) UPTIME=${UPTIME%%.*} # We don't shutdown the instances younger than 30m @@ -302,6 +320,7 @@ while true; do # If runner is not active, check that it needs to terminate itself echo "Checking if the instance suppose to terminate" no_terminating_metadata || terminate_on_event + check_spot_instance_is_old && terminate_and_exit check_proceed_spot_termination echo "Going to configure runner" @@ -311,6 +330,7 @@ while true; do echo "Another one check to avoid race between runner and infrastructure" no_terminating_metadata || terminate_on_event + check_spot_instance_is_old && terminate_and_exit check_proceed_spot_termination echo "Run" From cc64397e9282091d073386a5fd912689257b1445 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 16 Nov 2023 19:21:58 +0300 Subject: [PATCH 459/813] Planner support transactions --- src/Planner/Planner.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 89166316261..12e8d795347 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -116,7 +116,7 @@ namespace void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context) { const auto & query_context = planner_context->getQueryContext(); - if (query_context->getSettingsRef().throw_on_unsupported_query_inside_transaction) + if (!query_context->getSettingsRef().throw_on_unsupported_query_inside_transaction) return; if (!query_context->getCurrentTransaction()) @@ -130,13 +130,11 @@ void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context) else if (auto * table_function_node = table_expression->as()) storage = table_function_node->getStorage(); - if (storage->supportsTransactions()) - continue; - - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Storage {} (table {}) does not support transactions", - storage->getName(), - storage->getStorageID().getNameForLogs()); + if (storage && !storage->supportsTransactions()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Storage {} (table {}) does not support transactions", + storage->getName(), + storage->getStorageID().getNameForLogs()); } } @@ -1333,9 +1331,9 @@ void Planner::buildPlanForQueryNode() query_node.getHaving() = {}; } - checkStoragesSupportTransactions(planner_context); collectSets(query_tree, *planner_context); collectTableExpressionData(query_tree, planner_context); + checkStoragesSupportTransactions(planner_context); if (!select_query_options.only_analyze) collectFiltersForAnalysis(query_tree, planner_context); From 00569baf4b9703891918be979f192d425dc81a2e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 16 Nov 2023 19:23:55 +0300 Subject: [PATCH 460/813] Updated analyzer failed tests --- tests/analyzer_tech_debt.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e155ee72ebb..d969b9e6fad 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -6,7 +6,6 @@ 01064_incremental_streaming_from_2_src_with_feedback 01083_expressions_in_engine_arguments 01155_rename_move_materialized_view -01173_transaction_control_queries 01214_test_storage_merge_aliases_with_where 01244_optimize_distributed_group_by_sharding_key 01268_mv_scalars @@ -30,7 +29,6 @@ 02139_MV_with_scalar_subquery 02174_cte_scalar_cache_mv 02302_s3_file_pruning -02345_implicit_transaction 02352_grouby_shadows_arg 02354_annoy 02428_parameterized_view From 49c58e76099b6a3771182c9bd093159054d707a0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 16 Nov 2023 16:24:09 +0000 Subject: [PATCH 461/813] Disable RabbitMQ secure connection test in intergatiion test with TSAN --- tests/integration/test_storage_rabbitmq/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 021cdf54af9..4f4d4dec02f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -110,6 +110,9 @@ def rabbitmq_setup_teardown(): ], ) def test_rabbitmq_select(rabbitmq_cluster, secure): + if secure and instance.is_built_with_memory_sanitizer(): + pytest.skip("Data races: see https://github.com/ClickHouse/ClickHouse/issues/56866") + port = cluster.rabbitmq_port if secure: port = cluster.rabbitmq_secure_port From d03a1aab7bea331738d915d23b7353ccbbac9cf4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 16 Nov 2023 16:39:57 +0000 Subject: [PATCH 462/813] Automatic style fix --- tests/integration/test_storage_rabbitmq/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 4f4d4dec02f..cb34f7203d6 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -111,7 +111,9 @@ def rabbitmq_setup_teardown(): ) def test_rabbitmq_select(rabbitmq_cluster, secure): if secure and instance.is_built_with_memory_sanitizer(): - pytest.skip("Data races: see https://github.com/ClickHouse/ClickHouse/issues/56866") + pytest.skip( + "Data races: see https://github.com/ClickHouse/ClickHouse/issues/56866" + ) port = cluster.rabbitmq_port if secure: From 472cfdc86d73b0a8135f2f135c4fa6118fcfa287 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 16 Nov 2023 17:47:51 +0100 Subject: [PATCH 463/813] Review fix --- src/Core/Settings.h | 2 +- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 5 +++-- src/IO/ReadSettings.h | 2 +- src/Interpreters/Context.cpp | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ee503322e2a..3a65496adc2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -723,7 +723,7 @@ class IColumn; M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \ M(UInt64, filesystem_cache_max_download_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be downloaded by a single query", 0) \ M(Bool, throw_on_error_from_cache_on_write_operations, false, "Ignore error from cache when caching on write operations (INSERT, merges)", 0) \ - M(UInt64, filesystem_cache_getorset_batch_size, 20, "A batch size for holding file segments for a single read range", 0) \ + M(UInt64, filesystem_cache_segments_batch_size, 20, "Limit on size of a single batch of file segments that a read buffer can request from cache. Too low value will lead to excessive requests to cache, too large may slow down eviction from cache", 0) \ \ M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \ M(Bool, enable_filesystem_read_prefetches_log, false, "Log to system.filesystem prefetch_log during query. Should be used only for testing or debugging, not recommended to be turned on by default", 0) \ diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 27d0b6706a6..06ee9eb4e85 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -116,18 +116,19 @@ void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch() { + chassert(!file_segments || file_segments->empty()); size_t size = getRemainingSizeToRead(); if (!size) return false; if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) { - file_segments = cache->get(cache_key, file_offset_of_buffer_end, size, settings.filesystem_cache_getorset_batch_size); + file_segments = cache->get(cache_key, file_offset_of_buffer_end, size, settings.filesystem_cache_segments_batch_size); } else { CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); - file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_getorset_batch_size); + file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size); } return !file_segments->empty(); } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 197ae563d25..4c8a6cb020a 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -100,7 +100,7 @@ struct ReadSettings bool enable_filesystem_cache_log = false; /// Don't populate cache when the read is not part of query execution (e.g. background thread). bool avoid_readthrough_cache_outside_query_context = true; - size_t filesystem_cache_getorset_batch_size = 100; + size_t filesystem_cache_segments_batch_size = 20; size_t filesystem_cache_max_download_size = (128UL * 1024 * 1024 * 1024); bool skip_download_if_exceeds_query_cache = true; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c2b1db2fe18..ec56412b74b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4761,7 +4761,7 @@ ReadSettings Context::getReadSettings() const res.enable_filesystem_cache = settings.enable_filesystem_cache; res.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; res.enable_filesystem_cache_log = settings.enable_filesystem_cache_log; - res.filesystem_cache_getorset_batch_size = settings.filesystem_cache_getorset_batch_size; + res.filesystem_cache_segments_batch_size = settings.filesystem_cache_segments_batch_size; res.filesystem_cache_max_download_size = settings.filesystem_cache_max_download_size; res.skip_download_if_exceeds_query_cache = settings.skip_download_if_exceeds_query_cache; From 2ec96f9e9e737b2805f0408c3639c19760b445af Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 16 Nov 2023 17:56:06 +0100 Subject: [PATCH 464/813] Update 01052_window_view_proc_tumble_to_now.sh --- .../queries/0_stateless/01052_window_view_proc_tumble_to_now.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh index e75b7d9570b..4325ebeed24 100755 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh @@ -16,7 +16,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY tumble(now('US/Samoa'), INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY tumble(now('US/Samoa'), INTERVAL '10' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); EOF From 409f781c0907f4862ddf57d80c6e1adc1ad8b77e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Nov 2023 18:10:26 +0100 Subject: [PATCH 465/813] Fix test --- tests/queries/0_stateless/01119_session_log.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01119_session_log.sql b/tests/queries/0_stateless/01119_session_log.sql index 8f6967b89ec..55f6228797a 100644 --- a/tests/queries/0_stateless/01119_session_log.sql +++ b/tests/queries/0_stateless/01119_session_log.sql @@ -4,7 +4,7 @@ select * from remote('127.0.0.2', system, one, 'default', ''); select * from remote('127.0.0.2', system, one, 'default', 'wrong password'); -- { serverError AUTHENTICATION_FAILED } select * from remote('127.0.0.2', system, one, 'nonexistsnt_user_1119', ''); -- { serverError AUTHENTICATION_FAILED } set receive_timeout=1; -select * from remote('127.0.0.2', system, one, ' INTERSERVER SECRET ', ''); -- { serverError AUTHENTICATION_FAILED } +select * from remote('127.0.0.2', system, one, ' INTERSERVER SECRET ', ''); -- { serverError NO_REMOTE_SHARD_AVAILABLE } set receive_timeout=300; select * from remote('127.0.0.2', system, one, ' ', ''); -- { serverError AUTHENTICATION_FAILED } From d24757bbbf6b4050429d5e2d4f1e7ecfd9fed93d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 16 Nov 2023 18:11:40 +0100 Subject: [PATCH 466/813] Add assertions --- src/Interpreters/Cache/FileCache.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 5de24977db5..0591038fc1d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -801,6 +801,11 @@ void FileCache::removePathIfExists(const String & path) void FileCache::removeAllReleasable() { assertInitialized(); + +#ifdef ABORT_ON_LOGICAL_ERROR + assertCacheCorrectness(); +#endif + metadata.removeAllKeys(/* if_releasable */true); if (stash) From fdf5cfdec09d8816c1fe067fc548c6b43ba37497 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 16 Nov 2023 18:54:09 +0100 Subject: [PATCH 467/813] Update FileCacheSettings.cpp --- src/Interpreters/Cache/FileCacheSettings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index ad414530c2a..e333d9a3cd8 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -58,7 +58,7 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin if (has("load_metadata_threads")) load_metadata_threads = get_uint("load_metadata_threads"); - + if (boundary_alignment > max_file_segment_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `boundary_alignment` cannot exceed `max_file_segment_size`"); } From bcf6a66aa2f50909dcd38b7cd7f9f8e2908aeba3 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 16 Nov 2023 09:58:00 -0800 Subject: [PATCH 468/813] Switch from DWARF 4 to 5 and see what happens (#56770) * Switch from DWARF 4 to 5 and see what happens * Update comment in Dwarf.h about which versions are supported --- CMakeLists.txt | 2 +- src/Common/Dwarf.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4bba7d14e25..73a84bd9b5a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -320,7 +320,7 @@ endif () set (COMPILER_FLAGS "${COMPILER_FLAGS}") # Our built-in unwinder only supports DWARF version up to 4. -set (DEBUG_INFO_FLAGS "-g -gdwarf-4") +set (DEBUG_INFO_FLAGS "-g") # Disable omit frame pointer compiler optimization using -fno-omit-frame-pointer option(DISABLE_OMIT_FRAME_POINTER "Disable omit frame pointer compiler optimization" OFF) diff --git a/src/Common/Dwarf.h b/src/Common/Dwarf.h index 3b717305d70..8f8df3acd1a 100644 --- a/src/Common/Dwarf.h +++ b/src/Common/Dwarf.h @@ -46,8 +46,8 @@ class Elf; * can parse Debug Information Entries (DIEs), abbreviations, attributes (of * all forms), and we can interpret bytecode for the line number VM. * - * We can interpret DWARF records of version 2, 3, or 4, although we don't - * actually support many of the version 4 features (such as VLIW, multiple + * We can interpret DWARF records of version 2, 3, 4, or 5, although we don't + * actually support many of the features of versions 4 and 5 (such as VLIW, multiple * operations per instruction) * * Note that the DWARF record parser does not allocate heap memory at all. From 427d9fff7092def0debeba02e9650ef5cf156755 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 16 Nov 2023 19:13:25 +0100 Subject: [PATCH 469/813] Set limit for memory --- docker/test/fuzzer/run-fuzzer.sh | 1 + tests/ci/stress.py | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index af1ce0c4dd4..fd977b4b420 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -234,6 +234,7 @@ quit # SC2046: Quote this to prevent word splitting. Actually I need word splitting. # shellcheck disable=SC2012,SC2046 timeout -s TERM --preserve-status 30m clickhouse-client \ + --max-memory-usage-for-client=1000000000 \ --receive_timeout=10 \ --receive_data_timeout_ms=10000 \ --stacktrace \ diff --git a/tests/ci/stress.py b/tests/ci/stress.py index ae918363df7..0a6b379e714 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -125,7 +125,7 @@ def call_with_retry(query: str, timeout: int = 30, retry_count: int = 5) -> None def make_query_command(query: str) -> str: return ( f'clickhouse client -q "{query}" --max_untracked_memory=1Gi ' - "--memory_profiler_step=1Gi --max_memory_usage_for_user=0" + "--memory_profiler_step=1Gi --max_memory_usage_for_user=0 --max-memory-usage-for-client=1000000000" ) From d83cf03c30f5cde7d7442c6f01da04f8fb7c2a3a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 16 Nov 2023 19:21:27 +0100 Subject: [PATCH 470/813] no randomization sleep_before_commit_local_part_in_replicated_table_ms --- tests/clickhouse-test | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 053bd040bce..cab7d7e79ff 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -673,9 +673,6 @@ class MergeTreeSettingsRandomizer: "primary_key_compress_block_size": lambda: random.randint(8000, 100000), "replace_long_file_name_to_hash": lambda: random.randint(0, 1), "max_file_name_length": threshold_generator(0.3, 0.3, 0, 128), - "sleep_before_commit_local_part_in_replicated_table_ms": threshold_generator( - 0.7, 0.7, 0, 100 - ), } @staticmethod From 2a1467b8da19bfb91edcb1c4f986697f7c2f13e3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 16 Nov 2023 19:33:17 +0100 Subject: [PATCH 471/813] Update test_storage_s3_queue/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 9f41cfd176d..ec27b732634 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -734,7 +734,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): create_mv(instance, table_name, dst_table_name) total_values = generate_random_files( - started_cluster, files_path, files_to_generate, row_num=1 + started_cluster, files_path, files_to_generate, row_num=50 ) def get_count(node, table_name): From 28e0c51e3fdcf4692ad18d9e535a9ff4771d3955 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Nov 2023 20:46:17 +0100 Subject: [PATCH 472/813] Update avg_weighted.xml (#56797) --- tests/performance/avg_weighted.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/performance/avg_weighted.xml b/tests/performance/avg_weighted.xml index 5aa89b08c35..edf3c19fdfa 100644 --- a/tests/performance/avg_weighted.xml +++ b/tests/performance/avg_weighted.xml @@ -27,10 +27,6 @@ SELECT avg(num_f) FROM perf_avg FORMAT Null SELECT avgWeighted(num_f, num) FROM perf_avg FORMAT Null SELECT avgWeighted(num_f, num_f) FROM perf_avg FORMAT Null - SELECT avgWeighted(num_f, num_u) FROM perf_avg FORMAT Null - SELECT avgWeighted(num_u, num_f) FROM perf_avg FORMAT Null - SELECT avgWeighted(num_u, num) FROM perf_avg FORMAT Null - SELECT avgWeighted(num_u, num_u) FROM perf_avg FORMAT Null SELECT avgWeighted(num_f, num_f) FROM perf_avg FORMAT Null SELECT avgWeighted(toNullable(num_f), num_f) FROM perf_avg FORMAT Null From e533abef75f9eb304ceea09a74afb37672728420 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 16 Nov 2023 20:54:52 +0100 Subject: [PATCH 473/813] Resolve 01572_kill_window_function flakiness --- .../0_stateless/01572_kill_window_function.sh | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01572_kill_window_function.sh b/tests/queries/0_stateless/01572_kill_window_function.sh index 7103b7f7210..de6de3510a0 100755 --- a/tests/queries/0_stateless/01572_kill_window_function.sh +++ b/tests/queries/0_stateless/01572_kill_window_function.sh @@ -6,21 +6,20 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail +function wait_for_query_to_start() +{ + while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done +} + # Run a test query that takes very long to run. query_id="01572_kill_window_function-$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query_id="$query_id" --query "SELECT count(1048575) OVER (PARTITION BY intDiv(NULL, number) ORDER BY number DESC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1048575 FOLLOWING) FROM numbers(255, 1048575)" >/dev/null 2>&1 & +$CLICKHOUSE_CLIENT --query_id="$query_id" --query "SELECT sum(number) OVER (PARTITION BY number % 10 ORDER BY number DESC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 99999 FOLLOWING) FROM numbers(0, 10000000) format Null;" >/dev/null 2>&1 & client_pid=$! echo Started -# Use one query to both kill the test query and verify that it has started, -# because if we try to kill it before it starts, the test will fail. -while [ -z "$($CLICKHOUSE_CLIENT --query "kill query where query_id = '$query_id' and current_database = currentDatabase()")" ] -do - # If we don't yet see the query in the process list, the client should still - # be running. The query is very long. - kill -0 -- $client_pid - sleep 1 -done +wait_for_query_to_start $query_id + +$CLICKHOUSE_CLIENT --query "kill query where query_id = '$query_id' and current_database = currentDatabase() format Null" echo Sent kill request # Wait for the client to terminate. From ac3100747575e9f86493dfeb32a081734fac3a5c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Nov 2023 21:00:39 +0100 Subject: [PATCH 474/813] Update DatabaseReplicated.cpp (#56796) --- src/Databases/DatabaseReplicated.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index a643eafdd14..5da20c42465 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1083,12 +1083,14 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep } LOG_INFO(log, "All tables are created successfully"); - if (max_log_ptr_at_creation != 0) + chassert(max_log_ptr_at_creation || our_log_ptr); + UInt32 first_entry_to_mark_finished = new_replica ? max_log_ptr_at_creation : our_log_ptr; + if (first_entry_to_mark_finished) { /// If the replica is new and some of the queries applied during recovery /// where issued after the replica was created, then other nodes might be /// waiting for this node to notify them that the query was applied. - for (UInt32 ptr = max_log_ptr_at_creation; ptr <= max_log_ptr; ++ptr) + for (UInt32 ptr = first_entry_to_mark_finished; ptr <= max_log_ptr; ++ptr) { auto entry_name = DDLTaskBase::getLogEntryName(ptr); auto path = fs::path(zookeeper_path) / "log" / entry_name / "finished" / getFullReplicaName(); From 3765a1c77b586455565940b217d2a40f3d49333d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 16 Nov 2023 16:07:49 -0400 Subject: [PATCH 475/813] Update random-functions.md --- docs/en/sql-reference/functions/random-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index 13c29329f41..6fd31e8d25c 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -6,9 +6,9 @@ sidebar_label: Random Numbers # Functions for Generating Random Numbers -All functions in this section accept zero or one arguments. The only use of the argument (if provided) is to prevent prevent [common subexpression -elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) such that two different execution of the same random -function in a query return different random values. +All functions in this section accept zero or one arguments. The only use of the argument (if provided) is to prevent [common subexpression +elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) such that two different executions within a row of the same random +function return different random values. Related content - Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) From 1abcb28624ee4d204e8e1723ad58e87d1d80be56 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Nov 2023 23:32:17 +0100 Subject: [PATCH 476/813] Remove ctest --- CMakeLists.txt | 1 - cmake/add_check.cmake | 19 ------------------- src/CMakeLists.txt | 4 ---- tests/CMakeLists.txt | 26 -------------------------- 4 files changed, 50 deletions(-) delete mode 100644 cmake/add_check.cmake delete mode 100644 tests/CMakeLists.txt diff --git a/CMakeLists.txt b/CMakeLists.txt index 9f05b2b78ce..4fe7a1e05e7 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -561,7 +561,6 @@ option(CHECK_LARGE_OBJECT_SIZES "Check that there are no large object files afte add_subdirectory (base) add_subdirectory (src) add_subdirectory (programs) -add_subdirectory (tests) add_subdirectory (utils) if (FUZZER) diff --git a/cmake/add_check.cmake b/cmake/add_check.cmake deleted file mode 100644 index ba30ee8676f..00000000000 --- a/cmake/add_check.cmake +++ /dev/null @@ -1,19 +0,0 @@ -# Adding test output on failure -enable_testing () - -if (NOT TARGET check) - if (CMAKE_CONFIGURATION_TYPES) - add_custom_target (check COMMAND ${CMAKE_CTEST_COMMAND} - --force-new-ctest-process --output-on-failure --build-config "$" - WORKING_DIRECTORY ${PROJECT_BINARY_DIR}) - else () - add_custom_target (check COMMAND ${CMAKE_CTEST_COMMAND} - --force-new-ctest-process --output-on-failure - WORKING_DIRECTORY ${PROJECT_BINARY_DIR}) - endif () -endif () - -macro (add_check target) - add_test (NAME test_${target} COMMAND ${target} WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR}) - add_dependencies (check ${target}) -endmacro (add_check) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 5254743e154..3733295e9b4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -600,8 +600,6 @@ if (TARGET ch_rust::skim) dbms_target_link_libraries(PUBLIC ch_rust::skim) endif() -include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake") - if (ENABLE_TESTS) macro (grep_gtest_sources BASE_DIR DST_VAR) # Cold match files that are not in tests/ directories @@ -645,6 +643,4 @@ if (ENABLE_TESTS) if (TARGET ch_contrib::parquet) target_link_libraries(unit_tests_dbms PRIVATE ch_contrib::parquet) endif() - - add_check(unit_tests_dbms) endif () diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt deleted file mode 100644 index 22c89aaafa7..00000000000 --- a/tests/CMakeLists.txt +++ /dev/null @@ -1,26 +0,0 @@ -enable_testing() - -# Run tests with "ninja check" or "make check" -if (TARGET check) - message (STATUS "Target check already exists") -else () - include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake) -endif () - -option (ENABLE_CLICKHOUSE_TEST "Install clickhouse-test script and relevant tests scenarios" OFF) - -if (ENABLE_CLICKHOUSE_TEST) - install (PROGRAMS clickhouse-test DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - install ( - DIRECTORY queries performance config - DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test - USE_SOURCE_PERMISSIONS - COMPONENT clickhouse - PATTERN "CMakeLists.txt" EXCLUDE - PATTERN ".gitignore" EXCLUDE - ) -endif () - -if (ENABLE_TEST_INTEGRATION) - add_subdirectory (integration) -endif () From c72136b123685f6e4704b9d39122f637ee95ce0c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 08:07:32 +0100 Subject: [PATCH 477/813] Simpler CMake --- contrib/arrow-cmake/CMakeLists.txt | 16 ++++++++-------- contrib/azure-cmake/CMakeLists.txt | 3 +-- contrib/thrift-cmake/CMakeLists.txt | 6 ------ 3 files changed, 9 insertions(+), 16 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 71133451889..935fc886330 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -77,16 +77,16 @@ set(FLATBUFFERS_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/flatbuffers") set(FLATBUFFERS_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/flatbuffers") set(FLATBUFFERS_INCLUDE_DIR "${FLATBUFFERS_SRC_DIR}/include") -# set flatbuffers CMake options -set(FLATBUFFERS_BUILD_FLATLIB ON CACHE BOOL "Enable the build of the flatbuffers library") -set(FLATBUFFERS_BUILD_SHAREDLIB OFF CACHE BOOL "Disable the build of the flatbuffers shared library") -set(FLATBUFFERS_BUILD_TESTS OFF CACHE BOOL "Skip flatbuffers tests") +set(FLATBUFFERS_SRCS + ${FLATBUFFERS_SRC_DIR}/src/idl_parser.cpp + ${FLATBUFFERS_SRC_DIR}/src/idl_gen_text.cpp + ${FLATBUFFERS_SRC_DIR}/src/reflection.cpp + ${FLATBUFFERS_SRC_DIR}/src/util.cpp) -add_subdirectory(${FLATBUFFERS_SRC_DIR} "${FLATBUFFERS_BINARY_DIR}") +add_library(_flatbuffers STATIC ${FLATBUFFERS_SRCS}) +target_include_directories(_flatbuffers PUBLIC ${FLATBUFFERS_INCLUDE_DIR}) +target_compile_definitions(_flatbuffers PRIVATE -DFLATBUFFERS_LOCALE_INDEPENDENT=0) -add_library(_flatbuffers INTERFACE) -target_link_libraries(_flatbuffers INTERFACE flatbuffers) -target_include_directories(_flatbuffers INTERFACE ${FLATBUFFERS_INCLUDE_DIR}) # === hdfs # NOTE: cannot use ch_contrib::hdfs since it's INCLUDE_DIRECTORIES does not includes trailing "hdfs/" diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 7aba81259d3..bb44c993e79 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -48,9 +48,8 @@ set(AZURE_SDK_INCLUDES "${AZURE_SDK_LIBRARY_DIR}/storage/azure-storage-blobs/inc/" ) -include("${AZURE_DIR}/cmake-modules/AzureTransportAdapters.cmake") - add_library(_azure_sdk ${AZURE_SDK_UNIFIED_SRC}) +target_compile_definitions(_azure_sdk PRIVATE BUILD_CURL_HTTP_TRANSPORT_ADAPTER) # Originally, on Windows azure-core is built with bcrypt and crypt32 by default if (TARGET OpenSSL::SSL) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index d6aa6b9e5f2..89a444cfb83 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -47,8 +47,6 @@ set(thriftcpp_threads_SOURCES "${LIBRARY_DIR}/src/thrift/concurrency/Mutex.cpp" ) -include("${ClickHouse_SOURCE_DIR}/contrib/thrift/build/cmake/ConfigureChecks.cmake") # makes config.h - set (HAVE_ARPA_INET_H 1) set (HAVE_FCNTL_H 1) set (HAVE_GETOPT_H 1) @@ -81,10 +79,6 @@ if (OS_LINUX AND NOT USE_MUSL) set (STRERROR_R_CHAR_P 1) endif () -#set(PACKAGE ${PACKAGE_NAME}) -#set(PACKAGE_STRING "${PACKAGE_NAME} ${PACKAGE_VERSION}") -#set(VERSION ${thrift_VERSION}) - # generate a config.h file configure_file("${CMAKE_CURRENT_SOURCE_DIR}/build/cmake/config.h.in" "${CMAKE_CURRENT_BINARY_DIR}/thrift/config.h") From 5496e2d6ac483beeccca03a2ed92b586e33f28dd Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 17 Nov 2023 10:24:47 +0300 Subject: [PATCH 478/813] test for #56790 --- .../00059_shard_global_in_mergetree.reference | 8 +++++++ .../00059_shard_global_in_mergetree.sql | 24 +++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/00059_shard_global_in_mergetree.reference create mode 100644 tests/queries/0_stateless/00059_shard_global_in_mergetree.sql diff --git a/tests/queries/0_stateless/00059_shard_global_in_mergetree.reference b/tests/queries/0_stateless/00059_shard_global_in_mergetree.reference new file mode 100644 index 00000000000..829419dc759 --- /dev/null +++ b/tests/queries/0_stateless/00059_shard_global_in_mergetree.reference @@ -0,0 +1,8 @@ +20 +20 +20 +20 +20 +20 +20 +20 diff --git a/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql b/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql new file mode 100644 index 00000000000..b85560d2bea --- /dev/null +++ b/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql @@ -0,0 +1,24 @@ +-- Tags: shard + +-- test for #56790 + +CREATE TABLE test_local (x Int64) +ENGINE = MergeTree order by x as select * from numbers(10); + +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local); + +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local); + +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where 'XXX' global in (select 'XXX'); + +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * global in (select * from test_local); + +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * in (select * from test_local); + +set prefer_localhost_replica=0; + +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where 'XXX' global in (select 'XXX'); + +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * global in (select * from test_local); + +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * in (select * from test_local); From c4f46c7ce575fbfcb52c69343534b9539b3f719d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 17 Nov 2023 10:27:02 +0300 Subject: [PATCH 479/813] test for #56790 --- .../0_stateless/00059_shard_global_in_mergetree.reference | 1 - tests/queries/0_stateless/00059_shard_global_in_mergetree.sql | 2 -- 2 files changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/00059_shard_global_in_mergetree.reference b/tests/queries/0_stateless/00059_shard_global_in_mergetree.reference index 829419dc759..208e649c056 100644 --- a/tests/queries/0_stateless/00059_shard_global_in_mergetree.reference +++ b/tests/queries/0_stateless/00059_shard_global_in_mergetree.reference @@ -5,4 +5,3 @@ 20 20 20 -20 diff --git a/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql b/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql index b85560d2bea..cbd4245a486 100644 --- a/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql +++ b/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql @@ -7,8 +7,6 @@ ENGINE = MergeTree order by x as select * from numbers(10); select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local); -select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local); - select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where 'XXX' global in (select 'XXX'); select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * global in (select * from test_local); From 224b282d947daf5275e9a38d1c62e8887eb44868 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 08:27:19 +0100 Subject: [PATCH 480/813] Remove garbage --- contrib/cassandra-cmake/CMakeLists.txt | 4 ---- contrib/qpl-cmake/CMakeLists.txt | 3 +-- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/contrib/cassandra-cmake/CMakeLists.txt b/contrib/cassandra-cmake/CMakeLists.txt index 32611e0e151..9e729c436d5 100644 --- a/contrib/cassandra-cmake/CMakeLists.txt +++ b/contrib/cassandra-cmake/CMakeLists.txt @@ -83,10 +83,6 @@ set(HAVE_MEMCPY 1) set(HAVE_LONG_LONG 1) set(HAVE_UINT16_T 1) -configure_file("${CASS_SRC_DIR}/third_party/sparsehash/config.h.cmake" "${CMAKE_CURRENT_BINARY_DIR}/sparsehash/internal/sparseconfig.h") - - - # Determine random availability if (OS_LINUX) #set (HAVE_GETRANDOM 1) - not on every Linux kernel diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 4e6c66fe731..19501209b26 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -16,8 +16,7 @@ function(GetLibraryVersion _content _outputVar) SET(${_outputVar} ${CMAKE_MATCH_1} PARENT_SCOPE) endfunction() -FILE(READ "${QPL_PROJECT_DIR}/CMakeLists.txt" HEADER_CONTENT) -GetLibraryVersion("${HEADER_CONTENT}" QPL_VERSION) +set (QPL_VERSION 1.2.0) message(STATUS "Intel QPL version: ${QPL_VERSION}") From df3c066591758813c99422bab11f0811e1d487d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 08:42:39 +0100 Subject: [PATCH 481/813] Remove more trash --- contrib/arrow-cmake/CMakeLists.txt | 1 - contrib/cassandra-cmake/CMakeLists.txt | 7 +++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 935fc886330..96d1f4adda7 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -127,7 +127,6 @@ set(ORC_SRCS "${ORC_SOURCE_SRC_DIR}/BpackingDefault.hh" "${ORC_SOURCE_SRC_DIR}/ByteRLE.cc" "${ORC_SOURCE_SRC_DIR}/ByteRLE.hh" - "${ORC_SOURCE_SRC_DIR}/CMakeLists.txt" "${ORC_SOURCE_SRC_DIR}/ColumnPrinter.cc" "${ORC_SOURCE_SRC_DIR}/ColumnReader.cc" "${ORC_SOURCE_SRC_DIR}/ColumnReader.hh" diff --git a/contrib/cassandra-cmake/CMakeLists.txt b/contrib/cassandra-cmake/CMakeLists.txt index 9e729c436d5..0082364c130 100644 --- a/contrib/cassandra-cmake/CMakeLists.txt +++ b/contrib/cassandra-cmake/CMakeLists.txt @@ -68,8 +68,7 @@ list(APPEND INCLUDE_DIRS ${CASS_SRC_DIR}/third_party/hdr_histogram ${CASS_SRC_DIR}/third_party/http-parser ${CASS_SRC_DIR}/third_party/mt19937_64 - ${CASS_SRC_DIR}/third_party/rapidjson/rapidjson - ${CASS_SRC_DIR}/third_party/sparsehash/src) + ${CASS_SRC_DIR}/third_party/rapidjson/rapidjson) list(APPEND INCLUDE_DIRS ${CASS_INCLUDE_DIR} ${CASS_SRC_DIR}) @@ -112,17 +111,17 @@ configure_file( ${CASS_ROOT_DIR}/driver_config.hpp.in ${CMAKE_CURRENT_BINARY_DIR}/driver_config.hpp) - add_library(_cassandra ${SOURCES} $ $ $) -target_link_libraries(_cassandra ch_contrib::zlib ch_contrib::minizip) +target_link_libraries(_cassandra ch_contrib::zlib ch_contrib::minizip ch_contrib::sparsehash) target_include_directories(_cassandra PRIVATE ${CMAKE_CURRENT_BINARY_DIR} ${INCLUDE_DIRS}) target_include_directories(_cassandra SYSTEM BEFORE PUBLIC ${CASS_INCLUDE_DIR}) target_compile_definitions(_cassandra PRIVATE CASS_BUILDING) +target_compile_definitions(_cassandra PRIVATE -DSPARSEHASH_HASH=std::hash -Dsparsehash=google) target_link_libraries(_cassandra ch_contrib::uv) From f73b3e10ac0c4094cbc907cad2533ea73ae916b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 08:46:45 +0100 Subject: [PATCH 482/813] Ensure no new dependencies --- docker/packager/binary/build.sh | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 6b6374d08c9..ad31397c8d9 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -22,6 +22,7 @@ if [ "$EXTRACT_TOOLCHAIN_DARWIN" = "1" ]; then fi fi + # Uncomment to debug ccache. Don't put ccache log in /output right away, or it # will be confusingly packed into the "performance" package. # export CCACHE_LOGFILE=/build/ccache.log @@ -32,6 +33,16 @@ mkdir -p /build/build_docker cd /build/build_docker rm -f CMakeCache.txt + +# We don't want to depend on any third-party CMake files. +# To check it, find and delete them. + +grep -o -P '"contrib/[^"]+"' ../.gitmodules | + grep -v -P 'llvm-project|abseil-cpp|qpl|grpc|corrosion' | + xargs -I@ find @ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' | + xargs rm + + if [ -n "$MAKE_DEB" ]; then rm -rf /build/packages/root # NOTE: this is for backward compatibility with previous releases, From 163b8a814e4bd0247819c4a0c826cd28f1554bc0 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 16 Nov 2023 19:44:24 +0300 Subject: [PATCH 483/813] Fix clang-tidy issues and disable parallel execution of tests --- programs/disks/CommandCopy.cpp | 2 +- src/Disks/DiskEncrypted.cpp | 2 +- src/Disks/DiskEncrypted.h | 2 +- src/Disks/DiskLocal.h | 2 +- src/Disks/IDisk.cpp | 4 ++-- src/Disks/IDisk.h | 2 +- src/Storages/MergeTree/DataPartStorageOnDiskBase.h | 2 +- src/Storages/MergeTree/IDataPartStorage.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/localBackup.cpp | 2 +- tests/integration/parallel_skip.json | 6 +++++- tests/integration/test_ttl_move/test.py | 5 ++++- 12 files changed, 20 insertions(+), 13 deletions(-) diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index 296fc708411..421e4038d12 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -57,7 +57,7 @@ public: String relative_path_from = validatePathAndGetAsRelative(path_from); String relative_path_to = validatePathAndGetAsRelative(path_to); - disk_from->copyDirectoryContent(relative_path_from, disk_to, relative_path_to, /* read_settings= */ {}, /* write_settings= */ {}); + disk_from->copyDirectoryContent(relative_path_from, disk_to, relative_path_to, /* read_settings= */ {}, /* write_settings= */ {}, /* cancellation_hook= */ {}); } }; } diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 5ec17701667..ac81899156a 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -353,7 +353,7 @@ void DiskEncrypted::copyDirectoryContent( } /// Copy the file through buffers with deciphering. - IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings); + IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings, cancellation_hook); } std::unique_ptr DiskEncrypted::readFile( diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 6574d3bf456..9c497ba4cdc 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -118,7 +118,7 @@ public: const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings, - const std::function & cancellation_hook = {}) override; + const std::function & cancellation_hook) override; std::unique_ptr readFile( const String & path, diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 767718ecb24..affce5a847e 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -71,7 +71,7 @@ public: const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings, - const std::function & cancellation_hook = {}) override; + const std::function & cancellation_hook) override; void listFiles(const String & path, std::vector & file_names) const override; diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index cd5f1f4ef69..5426f8d0904 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -24,7 +24,7 @@ bool IDisk::isDirectoryEmpty(const String & path) const return !iterateDirectory(path)->isValid(); } -void IDisk::copyFile( +void IDisk::copyFile( /// NOLINT const String & from_file_path, IDisk & to_disk, const String & to_file_path, @@ -160,7 +160,7 @@ void IDisk::copyThroughBuffers( for (auto & result : results) result.wait(); for (auto & result : results) - result.get(); /// May rethrow an exception + result.get(); /// May rethrow an exception } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index b2d42f2e732..b2f40d7f36d 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -199,7 +199,7 @@ public: const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings, - const std::function & cancellation_hook = {}); + const std::function & cancellation_hook); /// Copy file `from_file_path` to `to_file_path` located at `to_disk`. virtual void copyFile( /// NOLINT diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 5792e163856..339acce5953 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -75,7 +75,7 @@ public: const ReadSettings & read_settings, const WriteSettings & write_settings, Poco::Logger * log, - const std::function & cancellation_hook = {} + const std::function & cancellation_hook ) const override; void rename( diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index e91a35974e3..59251e40626 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -263,7 +263,7 @@ public: const ReadSettings & read_settings, const WriteSettings & write_settings, Poco::Logger * log, - const std::function & cancellation_hook = {} + const std::function & cancellation_hook ) const = 0; /// Change part's root. from_root should be a prefix path of current root path. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 752a6de39cf..8a694fb9857 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -385,7 +385,7 @@ public: const String & directory_name, const ReadSettings & read_settings, const WriteSettings & write_settings, - const std::function & cancellation_hook = {}) const; + const std::function & cancellation_hook) const; /// Checks that .bin and .mrk files exist. /// diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index c84e13b167f..0698848fa70 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -170,7 +170,7 @@ void localBackup( else if (copy_instead_of_hardlinks) { CleanupOnFail cleanup([disk, destination_path]() { disk->removeRecursive(destination_path); }); - disk->copyDirectoryContent(source_path, disk, destination_path, read_settings, write_settings); + disk->copyDirectoryContent(source_path, disk, destination_path, read_settings, write_settings, /*cancellation_hook=*/{}); cleanup.success(); } else diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index d056225fee4..33dd85aceaf 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -92,5 +92,9 @@ "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query", - "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_client_suggestions_load" + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_client_suggestions_load", + + "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_stop_moves_query", + "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_table_detach", + "test_ttl_move/test.py::TestCancelBackgroundMoving::test_cancel_background_moving_on_zookeeper_disconnect" ] diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 7640ff8f737..2f18a0a1afa 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1925,5 +1925,8 @@ class TestCancelBackgroundMoving: pm.drop_instance_zk_connections(node1) # Wait for background moving task to be cancelled assert_logs_contain_with_retry( - node1, "MergeTreeBackgroundExecutor.*Cancelled moving parts" + node1, + "MergeTreeBackgroundExecutor.*Cancelled moving parts", + retry_count=30, + sleep_time=1, ) From c7d8465897e6d51a71d9c28bec51ba676fb70fa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 09:12:49 +0100 Subject: [PATCH 484/813] Ensure no new dependencies --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index ad31397c8d9..d469b359d1a 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -39,7 +39,7 @@ rm -f CMakeCache.txt grep -o -P '"contrib/[^"]+"' ../.gitmodules | grep -v -P 'llvm-project|abseil-cpp|qpl|grpc|corrosion' | - xargs -I@ find @ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' | + xargs -I@ find ../@ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' | xargs rm From f456ac97fe7c411f897a432e80055e7ed3599ad2 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 17 Nov 2023 04:19:22 -0400 Subject: [PATCH 485/813] fix currentdatabase issue --- .../00059_shard_global_in_mergetree.sql | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql b/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql index cbd4245a486..62eec6f324b 100644 --- a/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql +++ b/tests/queries/0_stateless/00059_shard_global_in_mergetree.sql @@ -2,21 +2,24 @@ -- test for #56790 -CREATE TABLE test_local (x Int64) -ENGINE = MergeTree order by x as select * from numbers(10); +DROP TABLE IF EXISTS test_local; + +CREATE TABLE test_local (x Int64) ENGINE = MergeTree order by x as select * from numbers(10); select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local); select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where 'XXX' global in (select 'XXX'); -select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * global in (select * from test_local); +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * global in (select * from numbers(10)); -select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * in (select * from test_local); +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * in (select * from numbers(10)); set prefer_localhost_replica=0; select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where 'XXX' global in (select 'XXX'); -select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * global in (select * from test_local); +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * global in (select * from numbers(10)); -select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * in (select * from test_local); +select count() from remote('127.0.0.1,127.0.0.2', currentDatabase(), test_local) where * in (select * from numbers(10)); + +DROP TABLE test_local; From 864dd32b05c2eb40baf545279b15cc7aacc5937a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 09:48:42 +0100 Subject: [PATCH 486/813] Remove garbage --- .../client_scripts/allin1_ssb.sh | 530 ------------------ .../client_scripts/client_stressing_test.py | 278 --------- .../client_scripts/queries_ssb.sql | 10 - .../client_scripts/run_ssb.sh | 6 - .../database_dir/deflate/config_deflate.xml | 49 -- .../deflate_s2/config_deflate_s2.xml | 49 -- .../database_dir/lz4/config_lz4.xml | 49 -- .../database_dir/lz4_s2/config_lz4_s2.xml | 49 -- .../database_dir/zstd/config_zstd.xml | 49 -- .../database_dir/zstd_s2/config_zstd_s2.xml | 49 -- 10 files changed, 1118 deletions(-) delete mode 100644 contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh delete mode 100644 contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py delete mode 100644 contrib/qpl-cmake/benchmark_sample/client_scripts/queries_ssb.sql delete mode 100644 contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh delete mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/deflate/config_deflate.xml delete mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/deflate_s2/config_deflate_s2.xml delete mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/lz4/config_lz4.xml delete mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/lz4_s2/config_lz4_s2.xml delete mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/zstd/config_zstd.xml delete mode 100644 contrib/qpl-cmake/benchmark_sample/database_dir/zstd_s2/config_zstd_s2.xml diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh b/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh deleted file mode 100644 index 31017b565b6..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/allin1_ssb.sh +++ /dev/null @@ -1,530 +0,0 @@ -#!/bin/bash -ckhost="localhost" -ckport=("9000" "9001" "9002" "9003") -WORKING_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)/.." -OUTPUT_DIR="${WORKING_DIR}/output" -LOG_DIR="${OUTPUT_DIR}/log" -RAWDATA_DIR="${WORKING_DIR}/rawdata_dir" -database_dir="${WORKING_DIR}/database_dir" -CLIENT_SCRIPTS_DIR="${WORKING_DIR}/client_scripts" -LOG_PACK_FILE="$(date +%Y-%m-%d-%H-%M-%S)" -QUERY_FILE="queries_ssb.sql" -SERVER_BIND_CMD[0]="numactl -m 0 -N 0" -SERVER_BIND_CMD[1]="numactl -m 0 -N 0" -SERVER_BIND_CMD[2]="numactl -m 1 -N 1" -SERVER_BIND_CMD[3]="numactl -m 1 -N 1" -CLIENT_BIND_CMD="" -SSB_GEN_FACTOR=20 -TABLE_NAME="lineorder_flat" -TALBE_ROWS="119994608" -CODEC_CONFIG="lz4 deflate zstd" - -# define instance number -inst_num=$1 -if [ ! -n "$1" ]; then - echo "Please clarify instance number from 1,2,3 or 4" - exit 1 -else - echo "Benchmarking with instance number:$1" -fi - -if [ ! -d "$OUTPUT_DIR" ]; then -mkdir $OUTPUT_DIR -fi -if [ ! -d "$LOG_DIR" ]; then -mkdir $LOG_DIR -fi -if [ ! -d "$RAWDATA_DIR" ]; then -mkdir $RAWDATA_DIR -fi - -# define different directories -dir_server=("" "_s2" "_s3" "_s4") -ckreadSql=" - CREATE TABLE customer - ( - C_CUSTKEY UInt32, - C_NAME String, - C_ADDRESS String, - C_CITY LowCardinality(String), - C_NATION LowCardinality(String), - C_REGION LowCardinality(String), - C_PHONE String, - C_MKTSEGMENT LowCardinality(String) - ) - ENGINE = MergeTree ORDER BY (C_CUSTKEY); - - CREATE TABLE lineorder - ( - LO_ORDERKEY UInt32, - LO_LINENUMBER UInt8, - LO_CUSTKEY UInt32, - LO_PARTKEY UInt32, - LO_SUPPKEY UInt32, - LO_ORDERDATE Date, - LO_ORDERPRIORITY LowCardinality(String), - LO_SHIPPRIORITY UInt8, - LO_QUANTITY UInt8, - LO_EXTENDEDPRICE UInt32, - LO_ORDTOTALPRICE UInt32, - LO_DISCOUNT UInt8, - LO_REVENUE UInt32, - LO_SUPPLYCOST UInt32, - LO_TAX UInt8, - LO_COMMITDATE Date, - LO_SHIPMODE LowCardinality(String) - ) - ENGINE = MergeTree PARTITION BY toYear(LO_ORDERDATE) ORDER BY (LO_ORDERDATE, LO_ORDERKEY); - - CREATE TABLE part - ( - P_PARTKEY UInt32, - P_NAME String, - P_MFGR LowCardinality(String), - P_CATEGORY LowCardinality(String), - P_BRAND LowCardinality(String), - P_COLOR LowCardinality(String), - P_TYPE LowCardinality(String), - P_SIZE UInt8, - P_CONTAINER LowCardinality(String) - ) - ENGINE = MergeTree ORDER BY P_PARTKEY; - - CREATE TABLE supplier - ( - S_SUPPKEY UInt32, - S_NAME String, - S_ADDRESS String, - S_CITY LowCardinality(String), - S_NATION LowCardinality(String), - S_REGION LowCardinality(String), - S_PHONE String - ) - ENGINE = MergeTree ORDER BY S_SUPPKEY; -" -supplier_table=" - CREATE TABLE supplier - ( - S_SUPPKEY UInt32, - S_NAME String, - S_ADDRESS String, - S_CITY LowCardinality(String), - S_NATION LowCardinality(String), - S_REGION LowCardinality(String), - S_PHONE String - ) - ENGINE = MergeTree ORDER BY S_SUPPKEY; -" -part_table=" - CREATE TABLE part - ( - P_PARTKEY UInt32, - P_NAME String, - P_MFGR LowCardinality(String), - P_CATEGORY LowCardinality(String), - P_BRAND LowCardinality(String), - P_COLOR LowCardinality(String), - P_TYPE LowCardinality(String), - P_SIZE UInt8, - P_CONTAINER LowCardinality(String) - ) - ENGINE = MergeTree ORDER BY P_PARTKEY; -" -lineorder_table=" - CREATE TABLE lineorder - ( - LO_ORDERKEY UInt32, - LO_LINENUMBER UInt8, - LO_CUSTKEY UInt32, - LO_PARTKEY UInt32, - LO_SUPPKEY UInt32, - LO_ORDERDATE Date, - LO_ORDERPRIORITY LowCardinality(String), - LO_SHIPPRIORITY UInt8, - LO_QUANTITY UInt8, - LO_EXTENDEDPRICE UInt32, - LO_ORDTOTALPRICE UInt32, - LO_DISCOUNT UInt8, - LO_REVENUE UInt32, - LO_SUPPLYCOST UInt32, - LO_TAX UInt8, - LO_COMMITDATE Date, - LO_SHIPMODE LowCardinality(String) - ) - ENGINE = MergeTree PARTITION BY toYear(LO_ORDERDATE) ORDER BY (LO_ORDERDATE, LO_ORDERKEY); -" -customer_table=" - CREATE TABLE customer - ( - C_CUSTKEY UInt32, - C_NAME String, - C_ADDRESS String, - C_CITY LowCardinality(String), - C_NATION LowCardinality(String), - C_REGION LowCardinality(String), - C_PHONE String, - C_MKTSEGMENT LowCardinality(String) - ) - ENGINE = MergeTree ORDER BY (C_CUSTKEY); -" - -lineorder_flat_table=" - SET max_memory_usage = 20000000000; - CREATE TABLE lineorder_flat - ENGINE = MergeTree - PARTITION BY toYear(LO_ORDERDATE) - ORDER BY (LO_ORDERDATE, LO_ORDERKEY) AS - SELECT - l.LO_ORDERKEY AS LO_ORDERKEY, - l.LO_LINENUMBER AS LO_LINENUMBER, - l.LO_CUSTKEY AS LO_CUSTKEY, - l.LO_PARTKEY AS LO_PARTKEY, - l.LO_SUPPKEY AS LO_SUPPKEY, - l.LO_ORDERDATE AS LO_ORDERDATE, - l.LO_ORDERPRIORITY AS LO_ORDERPRIORITY, - l.LO_SHIPPRIORITY AS LO_SHIPPRIORITY, - l.LO_QUANTITY AS LO_QUANTITY, - l.LO_EXTENDEDPRICE AS LO_EXTENDEDPRICE, - l.LO_ORDTOTALPRICE AS LO_ORDTOTALPRICE, - l.LO_DISCOUNT AS LO_DISCOUNT, - l.LO_REVENUE AS LO_REVENUE, - l.LO_SUPPLYCOST AS LO_SUPPLYCOST, - l.LO_TAX AS LO_TAX, - l.LO_COMMITDATE AS LO_COMMITDATE, - l.LO_SHIPMODE AS LO_SHIPMODE, - c.C_NAME AS C_NAME, - c.C_ADDRESS AS C_ADDRESS, - c.C_CITY AS C_CITY, - c.C_NATION AS C_NATION, - c.C_REGION AS C_REGION, - c.C_PHONE AS C_PHONE, - c.C_MKTSEGMENT AS C_MKTSEGMENT, - s.S_NAME AS S_NAME, - s.S_ADDRESS AS S_ADDRESS, - s.S_CITY AS S_CITY, - s.S_NATION AS S_NATION, - s.S_REGION AS S_REGION, - s.S_PHONE AS S_PHONE, - p.P_NAME AS P_NAME, - p.P_MFGR AS P_MFGR, - p.P_CATEGORY AS P_CATEGORY, - p.P_BRAND AS P_BRAND, - p.P_COLOR AS P_COLOR, - p.P_TYPE AS P_TYPE, - p.P_SIZE AS P_SIZE, - p.P_CONTAINER AS P_CONTAINER - FROM lineorder AS l - INNER JOIN customer AS c ON c.C_CUSTKEY = l.LO_CUSTKEY - INNER JOIN supplier AS s ON s.S_SUPPKEY = l.LO_SUPPKEY - INNER JOIN part AS p ON p.P_PARTKEY = l.LO_PARTKEY; - show settings ilike 'max_memory_usage'; -" - -function insert_data(){ - echo "insert_data:$1" - create_table_prefix="clickhouse client --host ${ckhost} --port $2 --multiquery -q" - insert_data_prefix="clickhouse client --query " - case $1 in - all) - clickhouse client --host ${ckhost} --port $2 --multiquery -q"$ckreadSql" && { - ${insert_data_prefix} "INSERT INTO customer FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/customer.tbl --port=$2 - ${insert_data_prefix} "INSERT INTO part FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/part.tbl --port=$2 - ${insert_data_prefix} "INSERT INTO supplier FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/supplier.tbl --port=$2 - ${insert_data_prefix} "INSERT INTO lineorder FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/lineorder.tbl --port=$2 - } - ${create_table_prefix}"${lineorder_flat_table}" - ;; - customer) - echo ${create_table_prefix}\"${customer_table}\" - ${create_table_prefix}"${customer_table}" && { - echo "${insert_data_prefix} \"INSERT INTO $1 FORMAT CSV\" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2" - ${insert_data_prefix} "INSERT INTO $1 FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2 - } - ;; - part) - echo ${create_table_prefix}\"${part_table}\" - ${create_table_prefix}"${part_table}" && { - echo "${insert_data_prefix} \"INSERT INTO $1 FORMAT CSV\" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2" - ${insert_data_prefix} "INSERT INTO $1 FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2 - } - ;; - supplier) - echo ${create_table_prefix}"${supplier_table}" - ${create_table_prefix}"${supplier_table}" && { - echo "${insert_data_prefix} \"INSERT INTO $1 FORMAT CSV\" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2" - ${insert_data_prefix} "INSERT INTO $1 FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2 - } - ;; - lineorder) - echo ${create_table_prefix}"${lineorder_table}" - ${create_table_prefix}"${lineorder_table}" && { - echo "${insert_data_prefix} \"INSERT INTO $1 FORMAT CSV\" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2" - ${insert_data_prefix} "INSERT INTO $1 FORMAT CSV" < ${RAWDATA_DIR}/ssb-dbgen/$1.tbl --port=$2 - } - ;; - lineorder_flat) - echo ${create_table_prefix}"${lineorder_flat_table}" - ${create_table_prefix}"${lineorder_flat_table}" - return 0 - ;; - *) - exit 0 - ;; - - esac -} - -function check_sql(){ - select_sql="select * from "$1" limit 1" - clickhouse client --host ${ckhost} --port $2 --multiquery -q"${select_sql}" -} - -function check_table(){ - checknum=0 - source_tables="customer part supplier lineorder lineorder_flat" - test_tables=${1:-${source_tables}} - echo "Checking table data required in server..." - for i in $(seq 0 $[inst_num-1]) - do - for j in `echo ${test_tables}` - do - check_sql $j ${ckport[i]} &> /dev/null || { - let checknum+=1 && insert_data "$j" ${ckport[i]} - } - done - done - - for i in $(seq 0 $[inst_num-1]) - do - echo "clickhouse client --host ${ckhost} --port ${ckport[i]} -m -q\"select count() from ${TABLE_NAME};\"" - var=$(clickhouse client --host ${ckhost} --port ${ckport[i]} -m -q"select count() from ${TABLE_NAME};") - if [ $var -eq $TALBE_ROWS ];then - echo "Instance_${i} Table data integrity check OK -> Rows:$var" - else - echo "Instance_${i} Table data integrity check Failed -> Rows:$var" - exit 1 - fi - done - if [ $checknum -gt 0 ];then - echo "Need sleep 10s after first table data insertion...$checknum" - sleep 10 - fi -} - -function check_instance(){ -instance_alive=0 -for i in {1..10} -do - sleep 1 - netstat -nltp | grep ${1} > /dev/null - if [ $? -ne 1 ];then - instance_alive=1 - break - fi - -done - -if [ $instance_alive -eq 0 ];then - echo "check_instance -> clickhouse server instance faild to launch due to 10s timeout!" - exit 1 -else - echo "check_instance -> clickhouse server instance launch successfully!" -fi -} - -function start_clickhouse_for_insertion(){ - echo "start_clickhouse_for_insertion" - for i in $(seq 0 $[inst_num-1]) - do - echo "cd ${database_dir}/$1${dir_server[i]}" - echo "${SERVER_BIND_CMD[i]} clickhouse server -C config_${1}${dir_server[i]}.xml >&${LOG_DIR}/${1}_${i}_server_log& > /dev/null" - - cd ${database_dir}/$1${dir_server[i]} - ${SERVER_BIND_CMD[i]} clickhouse server -C config_${1}${dir_server[i]}.xml >&${LOG_DIR}/${1}_${i}_server_log& > /dev/null - check_instance ${ckport[i]} - done -} - -function start_clickhouse_for_stressing(){ - echo "start_clickhouse_for_stressing" - for i in $(seq 0 $[inst_num-1]) - do - echo "cd ${database_dir}/$1${dir_server[i]}" - echo "${SERVER_BIND_CMD[i]} clickhouse server -C config_${1}${dir_server[i]}.xml >&/dev/null&" - - cd ${database_dir}/$1${dir_server[i]} - ${SERVER_BIND_CMD[i]} clickhouse server -C config_${1}${dir_server[i]}.xml >&/dev/null& - check_instance ${ckport[i]} - done -} -yum -y install git make gcc sudo net-tools &> /dev/null -pip3 install clickhouse_driver numpy &> /dev/null -test -d ${RAWDATA_DIR}/ssb-dbgen || git clone https://github.com/vadimtk/ssb-dbgen.git ${RAWDATA_DIR}/ssb-dbgen && cd ${RAWDATA_DIR}/ssb-dbgen - -if [ ! -f ${RAWDATA_DIR}/ssb-dbgen/dbgen ];then - make && { - test -f ${RAWDATA_DIR}/ssb-dbgen/customer.tbl || echo y |./dbgen -s ${SSB_GEN_FACTOR} -T c - test -f ${RAWDATA_DIR}/ssb-dbgen/part.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T p - test -f ${RAWDATA_DIR}/ssb-dbgen/supplier.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T s - test -f ${RAWDATA_DIR}/ssb-dbgen/date.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T d - test -f ${RAWDATA_DIR}/ssb-dbgen/lineorder.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T l - } -else - test -f ${RAWDATA_DIR}/ssb-dbgen/customer.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T c - test -f ${RAWDATA_DIR}/ssb-dbgen/part.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T p - test -f ${RAWDATA_DIR}/ssb-dbgen/supplier.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T s - test -f ${RAWDATA_DIR}/ssb-dbgen/date.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T d - test -f ${RAWDATA_DIR}/ssb-dbgen/lineorder.tbl || echo y | ./dbgen -s ${SSB_GEN_FACTOR} -T l - -fi - -filenum=`find ${RAWDATA_DIR}/ssb-dbgen/ -name "*.tbl" | wc -l` - -if [ $filenum -ne 5 ];then - echo "generate ssb data file *.tbl faild" - exit 1 -fi - -function kill_instance(){ -instance_alive=1 -for i in {1..2} -do - pkill clickhouse && sleep 5 - instance_alive=0 - for i in $(seq 0 $[inst_num-1]) - do - netstat -nltp | grep ${ckport[i]} > /dev/null - if [ $? -ne 1 ];then - instance_alive=1 - break; - fi - done - if [ $instance_alive -eq 0 ];then - break; - fi -done -if [ $instance_alive -eq 0 ];then - echo "kill_instance OK!" -else - echo "kill_instance Failed -> clickhouse server instance still alive due to 10s timeout" - exit 1 -fi -} - -function run_test(){ -is_xml=0 -for i in $(seq 0 $[inst_num-1]) -do - if [ -f ${database_dir}/${1}${dir_server[i]}/config_${1}${dir_server[i]}.xml ]; then - is_xml=$[is_xml+1] - fi -done -if [ $is_xml -eq $inst_num ];then - echo "Benchmark with $inst_num instance" - start_clickhouse_for_insertion ${1} - - for i in $(seq 0 $[inst_num-1]) - do - clickhouse client --host ${ckhost} --port ${ckport[i]} -m -q"show databases;" >/dev/null - done - - if [ $? -eq 0 ];then - check_table - fi - kill_instance - - if [ $1 == "deflate" ];then - test -f ${LOG_DIR}/${1}_server_log && deflatemsg=`cat ${LOG_DIR}/${1}_server_log | grep DeflateJobHWPool` - if [ -n "$deflatemsg" ];then - echo ------------------------------------------------------ - echo $deflatemsg - echo ------------------------------------------------------ - fi - fi - echo "Check table data required in server_${1} -> Done! " - - start_clickhouse_for_stressing ${1} - for i in $(seq 0 $[inst_num-1]) - do - clickhouse client --host ${ckhost} --port ${ckport[i]} -m -q"show databases;" >/dev/null - done - if [ $? -eq 0 ];then - test -d ${CLIENT_SCRIPTS_DIR} && cd ${CLIENT_SCRIPTS_DIR} - echo "Client stressing... " - echo "${CLIENT_BIND_CMD} python3 client_stressing_test.py ${QUERY_FILE} $inst_num &> ${LOG_DIR}/${1}.log" - ${CLIENT_BIND_CMD} python3 client_stressing_test.py ${QUERY_FILE} $inst_num &> ${LOG_DIR}/${1}.log - echo "Completed client stressing, checking log... " - finish_log=`grep "Finished" ${LOG_DIR}/${1}.log | wc -l` - if [ $finish_log -eq 1 ] ;then - kill_instance - test -f ${LOG_DIR}/${1}.log && echo "${1}.log ===> ${LOG_DIR}/${1}.log" - else - kill_instance - echo "No find 'Finished' in client log -> Performance test may fail" - exit 1 - - fi - - else - echo "${1} clickhouse server start fail" - exit 1 - fi -else - echo "clickhouse server start fail -> Please check xml files required in ${database_dir} for each instance" - exit 1 - -fi -} -function clear_log(){ - if [ -d "$LOG_DIR" ]; then - cd ${LOG_DIR} && rm -rf * - fi -} - -function gather_log_for_codec(){ - cd ${OUTPUT_DIR} && mkdir -p ${LOG_PACK_FILE}/${1} - cp -rf ${LOG_DIR} ${OUTPUT_DIR}/${LOG_PACK_FILE}/${1} -} - -function pack_log(){ - if [ -e "${OUTPUT_DIR}/run.log" ]; then - cp ${OUTPUT_DIR}/run.log ${OUTPUT_DIR}/${LOG_PACK_FILE}/ - fi - echo "Please check all log information in ${OUTPUT_DIR}/${LOG_PACK_FILE}" -} - -function setup_check(){ - - iax_dev_num=`accel-config list | grep iax | wc -l` - if [ $iax_dev_num -eq 0 ] ;then - iax_dev_num=`accel-config list | grep iax | wc -l` - if [ $iax_dev_num -eq 0 ] ;then - echo "No IAA devices available -> Please check IAA hardware setup manually!" - exit 1 - else - echo "IAA enabled devices number:$iax_dev_num" - fi - else - echo "IAA enabled devices number:$iax_dev_num" - fi - libaccel_version=`accel-config -v` - clickhouser_version=`clickhouse server --version` - kernel_dxd_log=`dmesg | grep dxd` - echo "libaccel_version:$libaccel_version" - echo "clickhouser_version:$clickhouser_version" - echo -e "idxd section in kernel log:\n$kernel_dxd_log" -} - -setup_check -export CLICKHOUSE_WATCHDOG_ENABLE=0 -for i in ${CODEC_CONFIG[@]} -do - clear_log - codec=${i} - echo "run test------------$codec" - run_test $codec - gather_log_for_codec $codec -done - -pack_log -echo "Done." \ No newline at end of file diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py b/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py deleted file mode 100644 index f12381a198c..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/client_stressing_test.py +++ /dev/null @@ -1,278 +0,0 @@ -from operator import eq -import os -import random -import time -import sys -from clickhouse_driver import Client -import numpy as np -import subprocess -import multiprocessing -from multiprocessing import Manager - -warmup_runs = 10 -calculated_runs = 10 -seconds = 30 -max_instances_number = 8 -retest_number = 3 -retest_tolerance = 10 - - -def checkInt(str): - try: - int(str) - return True - except ValueError: - return False - - -def setup_client(index): - if index < 4: - port_idx = index - else: - port_idx = index + 4 - client = Client( - host="localhost", - database="default", - user="default", - password="", - port="900%d" % port_idx, - ) - union_mode_query = "SET union_default_mode='DISTINCT'" - client.execute(union_mode_query) - return client - - -def warm_client(clientN, clientL, query, loop): - for c_idx in range(clientN): - for _ in range(loop): - clientL[c_idx].execute(query) - - -def read_queries(queries_list): - queries = list() - queries_id = list() - with open(queries_list, "r") as f: - for line in f: - line = line.rstrip() - line = line.split("$") - queries_id.append(line[0]) - queries.append(line[1]) - return queries_id, queries - - -def run_task(client, cname, query, loop, query_latency): - start_time = time.time() - for i in range(loop): - client.execute(query) - query_latency.append(client.last_query.elapsed) - - end_time = time.time() - p95 = np.percentile(query_latency, 95) - print( - "CLIENT: {0} end. -> P95: %f, qps: %f".format(cname) - % (p95, loop / (end_time - start_time)) - ) - - -def run_multi_clients(clientN, clientList, query, loop): - client_pids = {} - start_time = time.time() - manager = multiprocessing.Manager() - query_latency_list0 = manager.list() - query_latency_list1 = manager.list() - query_latency_list2 = manager.list() - query_latency_list3 = manager.list() - query_latency_list4 = manager.list() - query_latency_list5 = manager.list() - query_latency_list6 = manager.list() - query_latency_list7 = manager.list() - - for c_idx in range(clientN): - client_name = "Role_%d" % c_idx - if c_idx == 0: - client_pids[c_idx] = multiprocessing.Process( - target=run_task, - args=(clientList[c_idx], client_name, query, loop, query_latency_list0), - ) - elif c_idx == 1: - client_pids[c_idx] = multiprocessing.Process( - target=run_task, - args=(clientList[c_idx], client_name, query, loop, query_latency_list1), - ) - elif c_idx == 2: - client_pids[c_idx] = multiprocessing.Process( - target=run_task, - args=(clientList[c_idx], client_name, query, loop, query_latency_list2), - ) - elif c_idx == 3: - client_pids[c_idx] = multiprocessing.Process( - target=run_task, - args=(clientList[c_idx], client_name, query, loop, query_latency_list3), - ) - elif c_idx == 4: - client_pids[c_idx] = multiprocessing.Process( - target=run_task, - args=(clientList[c_idx], client_name, query, loop, query_latency_list4), - ) - elif c_idx == 5: - client_pids[c_idx] = multiprocessing.Process( - target=run_task, - args=(clientList[c_idx], client_name, query, loop, query_latency_list5), - ) - elif c_idx == 6: - client_pids[c_idx] = multiprocessing.Process( - target=run_task, - args=(clientList[c_idx], client_name, query, loop, query_latency_list6), - ) - elif c_idx == 7: - client_pids[c_idx] = multiprocessing.Process( - target=run_task, - args=(clientList[c_idx], client_name, query, loop, query_latency_list7), - ) - else: - print("ERROR: CLIENT number dismatch!!") - exit() - print("CLIENT: %s start" % client_name) - client_pids[c_idx].start() - - for c_idx in range(clientN): - client_pids[c_idx].join() - end_time = time.time() - totalT = end_time - start_time - - query_latencyTotal = list() - for item in query_latency_list0: - query_latencyTotal.append(item) - for item in query_latency_list1: - query_latencyTotal.append(item) - for item in query_latency_list2: - query_latencyTotal.append(item) - for item in query_latency_list3: - query_latencyTotal.append(item) - for item in query_latency_list4: - query_latencyTotal.append(item) - for item in query_latency_list5: - query_latencyTotal.append(item) - for item in query_latency_list6: - query_latencyTotal.append(item) - for item in query_latency_list7: - query_latencyTotal.append(item) - - totalP95 = np.percentile(query_latencyTotal, 95) * 1000 - return totalT, totalP95 - - -def run_task_caculated(client, cname, query, loop): - query_latency = list() - start_time = time.time() - for i in range(loop): - client.execute(query) - query_latency.append(client.last_query.elapsed) - end_time = time.time() - p95 = np.percentile(query_latency, 95) - - -def run_multi_clients_caculated(clientN, clientList, query, loop): - client_pids = {} - start_time = time.time() - for c_idx in range(clientN): - client_name = "Role_%d" % c_idx - client_pids[c_idx] = multiprocessing.Process( - target=run_task_caculated, - args=(clientList[c_idx], client_name, query, loop), - ) - client_pids[c_idx].start() - for c_idx in range(clientN): - client_pids[c_idx].join() - end_time = time.time() - totalT = end_time - start_time - return totalT - - -if __name__ == "__main__": - client_number = 1 - queries = list() - queries_id = list() - - if len(sys.argv) != 3: - print( - "usage: python3 client_stressing_test.py [queries_file_path] [client_number]" - ) - sys.exit() - else: - queries_list = sys.argv[1] - client_number = int(sys.argv[2]) - print( - "queries_file_path: %s, client_number: %d" % (queries_list, client_number) - ) - if not os.path.isfile(queries_list) or not os.access(queries_list, os.R_OK): - print("please check the right path for queries file") - sys.exit() - if ( - not checkInt(sys.argv[2]) - or int(sys.argv[2]) > max_instances_number - or int(sys.argv[2]) < 1 - ): - print("client_number should be in [1~%d]" % max_instances_number) - sys.exit() - - client_list = {} - queries_id, queries = read_queries(queries_list) - - for c_idx in range(client_number): - client_list[c_idx] = setup_client(c_idx) - # clear cache - os.system("sync; echo 3 > /proc/sys/vm/drop_caches") - - print("###Polit Run Begin") - for i in queries: - warm_client(client_number, client_list, i, 1) - print("###Polit Run End -> Start stressing....") - - query_index = 0 - for q in queries: - print( - "\n###START -> Index: %d, ID: %s, Query: %s" - % (query_index, queries_id[query_index], q) - ) - warm_client(client_number, client_list, q, warmup_runs) - print("###Warm Done!") - for j in range(0, retest_number): - totalT = run_multi_clients_caculated( - client_number, client_list, q, calculated_runs - ) - curr_loop = int(seconds * calculated_runs / totalT) + 1 - print( - "###Calculation Done! -> loopN: %d, expected seconds:%d" - % (curr_loop, seconds) - ) - - print("###Stress Running! -> %d iterations......" % curr_loop) - - totalT, totalP95 = run_multi_clients( - client_number, client_list, q, curr_loop - ) - - if totalT > (seconds - retest_tolerance) and totalT < ( - seconds + retest_tolerance - ): - break - else: - print( - "###totalT:%d is far way from expected seconds:%d. Run again ->j:%d!" - % (totalT, seconds, j) - ) - - print( - "###Completed! -> ID: %s, clientN: %d, totalT: %.2f s, latencyAVG: %.2f ms, P95: %.2f ms, QPS_Final: %.2f" - % ( - queries_id[query_index], - client_number, - totalT, - totalT * 1000 / (curr_loop * client_number), - totalP95, - ((curr_loop * client_number) / totalT), - ) - ) - query_index += 1 - print("###Finished!") diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/queries_ssb.sql b/contrib/qpl-cmake/benchmark_sample/client_scripts/queries_ssb.sql deleted file mode 100644 index abf2df6503a..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/queries_ssb.sql +++ /dev/null @@ -1,10 +0,0 @@ -Q1.1$SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE toYear(LO_ORDERDATE) = 1993 AND LO_DISCOUNT BETWEEN 1 AND 3 AND LO_QUANTITY < 25; -Q2.1$SELECT sum(LO_REVENUE),toYear(LO_ORDERDATE) AS year,P_BRAND FROM lineorder_flat WHERE P_CATEGORY = 'MFGR#12' AND S_REGION = 'AMERICA' GROUP BY year,P_BRAND ORDER BY year,P_BRAND; -Q2.2$SELECT sum(LO_REVENUE),toYear(LO_ORDERDATE) AS year,P_BRAND FROM lineorder_flat WHERE P_BRAND >= 'MFGR#2221' AND P_BRAND <= 'MFGR#2228' AND S_REGION = 'ASIA' GROUP BY year,P_BRAND ORDER BY year,P_BRAND; -Q2.3$SELECT sum(LO_REVENUE),toYear(LO_ORDERDATE) AS year,P_BRAND FROM lineorder_flat WHERE P_BRAND = 'MFGR#2239' AND S_REGION = 'EUROPE' GROUP BY year,P_BRAND ORDER BY year,P_BRAND; -Q3.1$SELECT C_NATION,S_NATION,toYear(LO_ORDERDATE) AS year,sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE C_REGION = 'ASIA' AND S_REGION = 'ASIA' AND year >= 1992 AND year <= 1997 GROUP BY C_NATION,S_NATION,year ORDER BY year ASC,revenue DESC; -Q3.2$SELECT C_CITY,S_CITY,toYear(LO_ORDERDATE) AS year,sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE C_NATION = 'UNITED STATES' AND S_NATION = 'UNITED STATES' AND year >= 1992 AND year <= 1997 GROUP BY C_CITY,S_CITY,year ORDER BY year ASC,revenue DESC; -Q3.3$SELECT C_CITY,S_CITY,toYear(LO_ORDERDATE) AS year,sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE (C_CITY = 'UNITED KI1' OR C_CITY = 'UNITED KI5') AND (S_CITY = 'UNITED KI1' OR S_CITY = 'UNITED KI5') AND year >= 1992 AND year <= 1997 GROUP BY C_CITY,S_CITY,year ORDER BY year ASC,revenue DESC; -Q4.1$SELECT toYear(LO_ORDERDATE) AS year,C_NATION,sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') GROUP BY year,C_NATION ORDER BY year ASC,C_NATION ASC; -Q4.2$SELECT toYear(LO_ORDERDATE) AS year,S_NATION,P_CATEGORY,sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (year = 1997 OR year = 1998) AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') GROUP BY year,S_NATION,P_CATEGORY ORDER BY year ASC,S_NATION ASC,P_CATEGORY ASC; -Q4.3$SELECT toYear(LO_ORDERDATE) AS year,S_CITY,P_BRAND,sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE S_NATION = 'UNITED STATES' AND (year = 1997 OR year = 1998) AND P_CATEGORY = 'MFGR#14' GROUP BY year,S_CITY,P_BRAND ORDER BY year ASC,S_CITY ASC,P_BRAND ASC; diff --git a/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh b/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh deleted file mode 100644 index 6067b1058f2..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/client_scripts/run_ssb.sh +++ /dev/null @@ -1,6 +0,0 @@ -WORKING_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)/.." -if [ ! -d "${WORKING_DIR}/output" ]; then -mkdir ${WORKING_DIR}/output -fi -bash allin1_ssb.sh 2 > ${WORKING_DIR}/output/run.log -echo "Please check log in: ${WORKING_DIR}/output/run.log" \ No newline at end of file diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/deflate/config_deflate.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/deflate/config_deflate.xml deleted file mode 100644 index ab77a9cdcbe..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/database_dir/deflate/config_deflate.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - - trace - true - - - 8123 - 9000 - 9004 - - ./ - - 8589934592 - 5368709120 - true - - - - deflate_qpl - - - - - - - - - ::/0 - - - default - default - 1 - - - - - - - - - - - diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/deflate_s2/config_deflate_s2.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/deflate_s2/config_deflate_s2.xml deleted file mode 100644 index b71456486f5..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/database_dir/deflate_s2/config_deflate_s2.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - - trace - true - - - 8124 - 9001 - 9005 - - ./ - - 8589934592 - 5368709120 - true - - - - deflate_qpl - - - - - - - - - ::/0 - - - default - default - 1 - - - - - - - - - - - diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/lz4/config_lz4.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/lz4/config_lz4.xml deleted file mode 100644 index f4dc59b60aa..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/database_dir/lz4/config_lz4.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - - trace - true - - - 8123 - 9000 - 9004 - - ./ - - 8589934592 - 5368709120 - true - - - - lz4 - - - - - - - - - ::/0 - - - default - default - 1 - - - - - - - - - - - diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/lz4_s2/config_lz4_s2.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/lz4_s2/config_lz4_s2.xml deleted file mode 100644 index 357db8942d7..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/database_dir/lz4_s2/config_lz4_s2.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - - trace - true - - - 8124 - 9001 - 9005 - - ./ - - 8589934592 - 5368709120 - true - - - - lz4 - - - - - - - - - ::/0 - - - default - default - 1 - - - - - - - - - - - diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/zstd/config_zstd.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/zstd/config_zstd.xml deleted file mode 100644 index 1c4c738edaf..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/database_dir/zstd/config_zstd.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - - trace - true - - - 8123 - 9000 - 9004 - - ./ - - 8589934592 - 5368709120 - true - - - - zstd - - - - - - - - - ::/0 - - - default - default - 1 - - - - - - - - - - - diff --git a/contrib/qpl-cmake/benchmark_sample/database_dir/zstd_s2/config_zstd_s2.xml b/contrib/qpl-cmake/benchmark_sample/database_dir/zstd_s2/config_zstd_s2.xml deleted file mode 100644 index f3db01b7739..00000000000 --- a/contrib/qpl-cmake/benchmark_sample/database_dir/zstd_s2/config_zstd_s2.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - - trace - true - - - 8124 - 9001 - 9005 - - ./ - - 8589934592 - 5368709120 - true - - - - zstd - - - - - - - - - ::/0 - - - default - default - 1 - - - - - - - - - - - From c65607484e51a5e9aa8f59612e7817b899bb88ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 10:03:10 +0100 Subject: [PATCH 487/813] Remove garbage --- contrib/qpl-cmake/CMakeLists.txt | 464 ++++++++++++++++++++++++++++--- docker/packager/binary/build.sh | 2 +- 2 files changed, 419 insertions(+), 47 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 19501209b26..7a84048e16b 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -27,16 +27,422 @@ message(STATUS "Intel QPL version: ${QPL_VERSION}") # The qpl submodule comes with its own version of isal. It contains code which does not exist in upstream isal. It would be nice to link # only upstream isal (ch_contrib::isal) but at this point we can't. -include("${QPL_PROJECT_DIR}/cmake/CompileOptions.cmake") +# ========================================================================== +# Copyright (C) 2022 Intel Corporation +# +# SPDX-License-Identifier: MIT +# ========================================================================== + +set(QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS "-fno-exceptions;-fno-rtti") + +function(modify_standard_language_flag) + # Declaring function parameters + set(OPTIONS "") + set(ONE_VALUE_ARGS + LANGUAGE_NAME + FLAG_NAME + NEW_FLAG_VALUE) + set(MULTI_VALUE_ARGS "") + + # Parsing function parameters + cmake_parse_arguments(MODIFY + "${OPTIONS}" + "${ONE_VALUE_ARGS}" + "${MULTI_VALUE_ARGS}" + ${ARGN}) + + # Variables + set(FLAG_REGULAR_EXPRESSION "${MODIFY_FLAG_NAME}.*[ ]*") + set(NEW_VALUE "${MODIFY_FLAG_NAME}${MODIFY_NEW_FLAG_VALUE}") + + # Replacing specified flag with new value + string(REGEX REPLACE + ${FLAG_REGULAR_EXPRESSION} ${NEW_VALUE} + NEW_COMPILE_FLAGS + "${CMAKE_${MODIFY_LANGUAGE_NAME}_FLAGS}") + + # Returning the value + set(CMAKE_${MODIFY_LANGUAGE_NAME}_FLAGS ${NEW_COMPILE_FLAGS} PARENT_SCOPE) +endfunction() + +function(get_function_name_with_default_bit_width in_function_name bit_width out_function_name) + + if(in_function_name MATCHES ".*_i") + + string(REPLACE "_i" "" in_function_name ${in_function_name}) + + set(${out_function_name} "${in_function_name}_${bit_width}_i" PARENT_SCOPE) + + else() + + set(${out_function_name} "${in_function_name}_${bit_width}" PARENT_SCOPE) + + endif() + +endfunction() + +macro(get_list_of_supported_optimizations PLATFORMS_LIST) + list(APPEND PLATFORMS_LIST "") + list(APPEND PLATFORMS_LIST "px") + list(APPEND PLATFORMS_LIST "avx512") +endmacro(get_list_of_supported_optimizations) + +function(generate_unpack_kernel_arrays current_directory PLATFORMS_LIST) + list(APPEND UNPACK_POSTFIX_LIST "") + list(APPEND UNPACK_PRLE_POSTFIX_LIST "") + list(APPEND PACK_POSTFIX_LIST "") + list(APPEND PACK_INDEX_POSTFIX_LIST "") + list(APPEND SCAN_POSTFIX_LIST "") + list(APPEND DEFAULT_BIT_WIDTH_FUNCTIONS_LIST "") + list(APPEND DEFAULT_BIT_WIDTH_LIST "") + + #create list of functions that use only 8u 16u 32u postfixes + list(APPEND DEFAULT_BIT_WIDTH_FUNCTIONS_LIST "unpack_prle") + list(APPEND DEFAULT_BIT_WIDTH_FUNCTIONS_LIST "extract") + list(APPEND DEFAULT_BIT_WIDTH_FUNCTIONS_LIST "extract_i") + list(APPEND DEFAULT_BIT_WIDTH_FUNCTIONS_LIST "select") + list(APPEND DEFAULT_BIT_WIDTH_FUNCTIONS_LIST "select_i") + list(APPEND DEFAULT_BIT_WIDTH_FUNCTIONS_LIST "expand") + + #create default bit width list + list(APPEND DEFAULT_BIT_WIDTH_LIST "8u") + list(APPEND DEFAULT_BIT_WIDTH_LIST "16u") + list(APPEND DEFAULT_BIT_WIDTH_LIST "32u") + + #create scan kernel postfixes + list(APPEND SCAN_COMPARATOR_LIST "") + + list(APPEND SCAN_COMPARATOR_LIST "eq") + list(APPEND SCAN_COMPARATOR_LIST "ne") + list(APPEND SCAN_COMPARATOR_LIST "lt") + list(APPEND SCAN_COMPARATOR_LIST "le") + list(APPEND SCAN_COMPARATOR_LIST "gt") + list(APPEND SCAN_COMPARATOR_LIST "ge") + list(APPEND SCAN_COMPARATOR_LIST "range") + list(APPEND SCAN_COMPARATOR_LIST "not_range") + + foreach(SCAN_COMPARATOR IN LISTS SCAN_COMPARATOR_LIST) + list(APPEND SCAN_POSTFIX_LIST "_${SCAN_COMPARATOR}_8u") + list(APPEND SCAN_POSTFIX_LIST "_${SCAN_COMPARATOR}_16u8u") + list(APPEND SCAN_POSTFIX_LIST "_${SCAN_COMPARATOR}_32u8u") + endforeach() + + # create unpack kernel postfixes + foreach(input_width RANGE 1 32 1) + if(input_width LESS 8 OR input_width EQUAL 8) + list(APPEND UNPACK_POSTFIX_LIST "_${input_width}u8u") + + elseif(input_width LESS 16 OR input_width EQUAL 16) + list(APPEND UNPACK_POSTFIX_LIST "_${input_width}u16u") + + else() + list(APPEND UNPACK_POSTFIX_LIST "_${input_width}u32u") + endif() + endforeach() + + # create pack kernel postfixes + foreach(output_width RANGE 1 8 1) + list(APPEND PACK_POSTFIX_LIST "_8u${output_width}u") + endforeach() + + foreach(output_width RANGE 9 16 1) + list(APPEND PACK_POSTFIX_LIST "_16u${output_width}u") + endforeach() + + foreach(output_width RANGE 17 32 1) + list(APPEND PACK_POSTFIX_LIST "_32u${output_width}u") + endforeach() + + list(APPEND PACK_POSTFIX_LIST "_8u16u") + list(APPEND PACK_POSTFIX_LIST "_8u32u") + list(APPEND PACK_POSTFIX_LIST "_16u32u") + + # create pack index kernel postfixes + list(APPEND PACK_INDEX_POSTFIX_LIST "_nu") + list(APPEND PACK_INDEX_POSTFIX_LIST "_8u") + list(APPEND PACK_INDEX_POSTFIX_LIST "_8u16u") + list(APPEND PACK_INDEX_POSTFIX_LIST "_8u32u") + + # write to file + file(MAKE_DIRECTORY ${current_directory}/generated) + + foreach(PLATFORM_VALUE IN LISTS PLATFORMS_LIST) + set(directory "${current_directory}/generated") + set(PLATFORM_PREFIX "${PLATFORM_VALUE}_") + + # + # Write unpack table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}unpack.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}unpack.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}unpack.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}unpack.cpp "unpack_table_t ${PLATFORM_PREFIX}unpack_table = {\n") + + #write LE kernels + foreach(UNPACK_POSTFIX IN LISTS UNPACK_POSTFIX_LIST) + file(APPEND ${directory}/${PLATFORM_PREFIX}unpack.cpp "\t${PLATFORM_PREFIX}qplc_unpack${UNPACK_POSTFIX},\n") + endforeach() + + #write BE kernels + + #get last element of the list + set(LAST_ELEMENT "") + list(GET UNPACK_POSTFIX_LIST -1 LAST_ELEMENT) + + foreach(UNPACK_POSTFIX IN LISTS UNPACK_POSTFIX_LIST) + + if(UNPACK_POSTFIX STREQUAL LAST_ELEMENT) + file(APPEND ${directory}/${PLATFORM_PREFIX}unpack.cpp "\t${PLATFORM_PREFIX}qplc_unpack_be${UNPACK_POSTFIX}};\n") + else() + file(APPEND ${directory}/${PLATFORM_PREFIX}unpack.cpp "\t${PLATFORM_PREFIX}qplc_unpack_be${UNPACK_POSTFIX},\n") + endif() + endforeach() + + file(APPEND ${directory}/${PLATFORM_PREFIX}unpack.cpp "}\n") + + # + # Write pack table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}pack.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack.cpp "pack_table_t ${PLATFORM_PREFIX}pack_table = {\n") + + #write LE kernels + foreach(PACK_POSTFIX IN LISTS PACK_POSTFIX_LIST) + file(APPEND ${directory}/${PLATFORM_PREFIX}pack.cpp "\t${PLATFORM_PREFIX}qplc_pack${PACK_POSTFIX},\n") + endforeach() + + #write BE kernels + + #get last element of the list + set(LAST_ELEMENT "") + list(GET PACK_POSTFIX_LIST -1 LAST_ELEMENT) + + foreach(PACK_POSTFIX IN LISTS PACK_POSTFIX_LIST) + + if(PACK_POSTFIX STREQUAL LAST_ELEMENT) + file(APPEND ${directory}/${PLATFORM_PREFIX}pack.cpp "\t${PLATFORM_PREFIX}qplc_pack_be${PACK_POSTFIX}};\n") + else() + file(APPEND ${directory}/${PLATFORM_PREFIX}pack.cpp "\t${PLATFORM_PREFIX}qplc_pack_be${PACK_POSTFIX},\n") + endif() + endforeach() + + file(APPEND ${directory}/${PLATFORM_PREFIX}pack.cpp "}\n") + + # + # Write scan table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}scan.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}scan.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}scan.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}scan.cpp "scan_table_t ${PLATFORM_PREFIX}scan_table = {\n") + + #get last element of the list + set(LAST_ELEMENT "") + list(GET SCAN_POSTFIX_LIST -1 LAST_ELEMENT) + + foreach(SCAN_POSTFIX IN LISTS SCAN_POSTFIX_LIST) + + if(SCAN_POSTFIX STREQUAL LAST_ELEMENT) + file(APPEND ${directory}/${PLATFORM_PREFIX}scan.cpp "\t${PLATFORM_PREFIX}qplc_scan${SCAN_POSTFIX}};\n") + else() + file(APPEND ${directory}/${PLATFORM_PREFIX}scan.cpp "\t${PLATFORM_PREFIX}qplc_scan${SCAN_POSTFIX},\n") + endif() + endforeach() + + file(APPEND ${directory}/${PLATFORM_PREFIX}scan.cpp "}\n") + + # + # Write scan_i table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}scan_i.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}scan_i.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}scan_i.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}scan_i.cpp "scan_i_table_t ${PLATFORM_PREFIX}scan_i_table = {\n") + + #get last element of the list + set(LAST_ELEMENT "") + list(GET SCAN_POSTFIX_LIST -1 LAST_ELEMENT) + + foreach(SCAN_POSTFIX IN LISTS SCAN_POSTFIX_LIST) + + if(SCAN_POSTFIX STREQUAL LAST_ELEMENT) + file(APPEND ${directory}/${PLATFORM_PREFIX}scan_i.cpp "\t${PLATFORM_PREFIX}qplc_scan${SCAN_POSTFIX}_i};\n") + else() + file(APPEND ${directory}/${PLATFORM_PREFIX}scan_i.cpp "\t${PLATFORM_PREFIX}qplc_scan${SCAN_POSTFIX}_i,\n") + endif() + endforeach() + + file(APPEND ${directory}/${PLATFORM_PREFIX}scan_i.cpp "}\n") + + # + # Write pack_index table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}pack_index.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "pack_index_table_t ${PLATFORM_PREFIX}pack_index_table = {\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "\t${PLATFORM_PREFIX}qplc_pack_bits_nu,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "\t${PLATFORM_PREFIX}qplc_pack_index_8u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "\t${PLATFORM_PREFIX}qplc_pack_index_8u16u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "\t${PLATFORM_PREFIX}qplc_pack_index_8u32u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "\t${PLATFORM_PREFIX}qplc_pack_bits_be_nu,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "\t${PLATFORM_PREFIX}qplc_pack_index_8u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "\t${PLATFORM_PREFIX}qplc_pack_index_be_8u16u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "\t${PLATFORM_PREFIX}qplc_pack_index_be_8u32u};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}pack_index.cpp "}\n") + + # + # Write default bit width functions + # + foreach(DEAULT_BIT_WIDTH_FUNCTION IN LISTS DEFAULT_BIT_WIDTH_FUNCTIONS_LIST) + file(WRITE ${directory}/${PLATFORM_PREFIX}${DEAULT_BIT_WIDTH_FUNCTION}.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}${DEAULT_BIT_WIDTH_FUNCTION}.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}${DEAULT_BIT_WIDTH_FUNCTION}.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}${DEAULT_BIT_WIDTH_FUNCTION}.cpp "${DEAULT_BIT_WIDTH_FUNCTION}_table_t ${PLATFORM_PREFIX}${DEAULT_BIT_WIDTH_FUNCTION}_table = {\n") + + #get last element of the list + set(LAST_ELEMENT "") + list(GET DEFAULT_BIT_WIDTH_LIST -1 LAST_ELEMENT) + + foreach(BIT_WIDTH IN LISTS DEFAULT_BIT_WIDTH_LIST) + + set(FUNCTION_NAME "") + get_function_name_with_default_bit_width(${DEAULT_BIT_WIDTH_FUNCTION} ${BIT_WIDTH} FUNCTION_NAME) + + if(BIT_WIDTH STREQUAL LAST_ELEMENT) + file(APPEND ${directory}/${PLATFORM_PREFIX}${DEAULT_BIT_WIDTH_FUNCTION}.cpp "\t${PLATFORM_PREFIX}qplc_${FUNCTION_NAME}};\n") + else() + file(APPEND ${directory}/${PLATFORM_PREFIX}${DEAULT_BIT_WIDTH_FUNCTION}.cpp "\t${PLATFORM_PREFIX}qplc_${FUNCTION_NAME},\n") + endif() + endforeach() + + file(APPEND ${directory}/${PLATFORM_PREFIX}${DEAULT_BIT_WIDTH_FUNCTION}.cpp "}\n") + endforeach() + + # + # Write aggregates table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}aggregates.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}aggregates.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}aggregates.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}aggregates.cpp "aggregates_table_t ${PLATFORM_PREFIX}aggregates_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}aggregates.cpp "\t${PLATFORM_PREFIX}qplc_bit_aggregates_8u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}aggregates.cpp "\t${PLATFORM_PREFIX}qplc_aggregates_8u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}aggregates.cpp "\t${PLATFORM_PREFIX}qplc_aggregates_16u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}aggregates.cpp "\t${PLATFORM_PREFIX}qplc_aggregates_32u};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}aggregates.cpp "}\n") + + # + # Write mem_copy functions table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}memory_copy.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}memory_copy.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}memory_copy.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}memory_copy.cpp "memory_copy_table_t ${PLATFORM_PREFIX}memory_copy_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}memory_copy.cpp "\t${PLATFORM_PREFIX}qplc_copy_8u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}memory_copy.cpp "\t${PLATFORM_PREFIX}qplc_copy_16u,\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}memory_copy.cpp "\t${PLATFORM_PREFIX}qplc_copy_32u};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}memory_copy.cpp "}\n") + + # + # Write mem_copy functions table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}zero.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}zero.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}zero.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}zero.cpp "zero_table_t ${PLATFORM_PREFIX}zero_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}zero.cpp "\t${PLATFORM_PREFIX}qplc_zero_8u};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}zero.cpp "}\n") + + # + # Write move functions table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}move.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}move.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}move.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}move.cpp "move_table_t ${PLATFORM_PREFIX}move_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}move.cpp "\t${PLATFORM_PREFIX}qplc_move_8u};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}move.cpp "}\n") + + # + # Write crc64 function table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}crc64.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}crc64.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}crc64.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}crc64.cpp "crc64_table_t ${PLATFORM_PREFIX}crc64_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}crc64.cpp "\t${PLATFORM_PREFIX}qplc_crc64};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}crc64.cpp "}\n") + + # + # Write xor_checksum function table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}xor_checksum.cpp "#include \"qplc_api.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}xor_checksum.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}xor_checksum.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}xor_checksum.cpp "xor_checksum_table_t ${PLATFORM_PREFIX}xor_checksum_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}xor_checksum.cpp "\t${PLATFORM_PREFIX}qplc_xor_checksum_8u};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}xor_checksum.cpp "}\n") + + # + # Write deflate functions table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}deflate.cpp "#include \"deflate_slow_icf.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "#include \"deflate_hash_table.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "#include \"deflate_histogram.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "deflate_table_t ${PLATFORM_PREFIX}deflate_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "\t reinterpret_cast(&${PLATFORM_PREFIX}slow_deflate_icf_body),\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "\t reinterpret_cast(&${PLATFORM_PREFIX}deflate_histogram_reset),\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "\t reinterpret_cast(&${PLATFORM_PREFIX}deflate_hash_table_reset)};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate.cpp "}\n") + + # + # Write deflate fix functions table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}deflate_fix.cpp "#include \"deflate_slow.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate_fix.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate_fix.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate_fix.cpp "deflate_fix_table_t ${PLATFORM_PREFIX}deflate_fix_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate_fix.cpp "\t reinterpret_cast(&${PLATFORM_PREFIX}slow_deflate_body)};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}deflate_fix.cpp "}\n") + + # + # Write setup_dictionary functions table + # + file(WRITE ${directory}/${PLATFORM_PREFIX}setup_dictionary.cpp "#include \"deflate_slow_utils.h\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}setup_dictionary.cpp "#include \"dispatcher/dispatcher.hpp\"\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}setup_dictionary.cpp "namespace qpl::core_sw::dispatcher\n{\n") + file(APPEND ${directory}/${PLATFORM_PREFIX}setup_dictionary.cpp "setup_dictionary_table_t ${PLATFORM_PREFIX}setup_dictionary_table = {\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}setup_dictionary.cpp "\t reinterpret_cast(&${PLATFORM_PREFIX}setup_dictionary)};\n") + + file(APPEND ${directory}/${PLATFORM_PREFIX}setup_dictionary.cpp "}\n") + + endforeach() +endfunction() -# check nasm compiler -include(CheckLanguage) -check_language(ASM_NASM) -if(NOT CMAKE_ASM_NASM_COMPILER) - message(FATAL_ERROR "Please install NASM from 'https://www.nasm.us/' because NASM compiler can not be found!") -endif() -# [SUBDIR]isal enable_language(ASM_NASM) set(ISAL_C_SRC ${QPL_SRC_DIR}/isal/igzip/adler32_base.c @@ -106,11 +512,6 @@ set_target_properties(isal PROPERTIES CXX_STANDARD 11 C_STANDARD 99) -target_compile_options(isal PRIVATE - "$<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}>" - "$<$:>" - "$<$:>") - # AS_FEATURE_LEVEL=10 means "Check SIMD capabilities of the target system at runtime and use up to AVX512 if available". # HAVE_KNOWS_AVX512 means rely on AVX512 being available on the target system. target_compile_options(isal_asm PRIVATE "-I${QPL_SRC_DIR}/isal/include/" @@ -163,15 +564,7 @@ foreach(PLATFORM_ID IN LISTS PLATFORMS_LIST) PUBLIC $ PRIVATE $) - set_target_properties(qplcore_${PLATFORM_ID} PROPERTIES - $<$:C_STANDARD 17>) - - target_compile_options(qplcore_${PLATFORM_ID} - PRIVATE ${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS} - PRIVATE "$<$:>" - PRIVATE "$<$:-O3;-D_FORTIFY_SOURCE=2>") - - # Set specific compiler options and/or definitions based on a platform + # Set specific compiler options and/or definitions based on a platform if (${PLATFORM_ID} MATCHES "avx512") target_compile_definitions(qplcore_${PLATFORM_ID} PRIVATE PLATFORM=2) target_compile_options(qplcore_${PLATFORM_ID} PRIVATE -march=skylake-avx512) @@ -220,10 +613,7 @@ set_target_properties(qplcore_sw_dispatcher PROPERTIES CXX_STANDARD 17) target_compile_definitions(qplcore_sw_dispatcher PUBLIC -DQPL_LIB) target_compile_options(qplcore_sw_dispatcher - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; - ${QPL_LINUX_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS}; - $<$:-O3;-D_FORTIFY_SOURCE=2>> - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) + PRIVATE ${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}) # [SUBDIR]core-iaa file(GLOB HW_PATH_SRC ${QPL_SRC_DIR}/core-iaa/sources/aecs/*.c @@ -248,14 +638,6 @@ target_include_directories(core_iaa PRIVATE $ # own_checkers.h PRIVATE $) -set_target_properties(core_iaa PROPERTIES - $<$:C_STANDARD 17> - CXX_STANDARD 17) - -target_compile_options(core_iaa - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; - $<$:-O3;-D_FORTIFY_SOURCE=2>>) - target_compile_features(core_iaa PRIVATE c_std_11) target_compile_definitions(core_iaa PRIVATE QPL_BADARG_CHECK @@ -285,10 +667,7 @@ set_property(GLOBAL APPEND PROPERTY QPL_LIB_DEPS $) target_compile_options(middle_layer_lib - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; - ${QPL_LINUX_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS}; - $<$:-O3;-D_FORTIFY_SOURCE=2>> - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) + PRIVATE ${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}) target_compile_definitions(middle_layer_lib PUBLIC QPL_VERSION="${QPL_VERSION}" @@ -323,15 +702,8 @@ target_include_directories(_qpl PRIVATE $ PRIVATE $) -set_target_properties(_qpl PROPERTIES - $<$:C_STANDARD 17> - CXX_STANDARD 17) - target_compile_options(_qpl - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_REQUIRED_FLAGS}; - ${QPL_LINUX_TOOLCHAIN_DYNAMIC_LIBRARY_FLAGS}; - $<$:-O3;-D_FORTIFY_SOURCE=2>> - PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) + PRIVATE ${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}) target_compile_definitions(_qpl PRIVATE -DQPL_LIB diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index d469b359d1a..42bfb48db70 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -38,7 +38,7 @@ rm -f CMakeCache.txt # To check it, find and delete them. grep -o -P '"contrib/[^"]+"' ../.gitmodules | - grep -v -P 'llvm-project|abseil-cpp|qpl|grpc|corrosion' | + grep -v -P 'llvm-project|abseil-cpp|grpc|corrosion' | xargs -I@ find ../@ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' | xargs rm From 7dda3b2353b22639a0304219c01beed16407c6eb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Nov 2023 10:11:15 +0000 Subject: [PATCH 488/813] Review comments --- src/Backups/BackupCoordinationFileInfos.cpp | 65 ++++++++++++--------- src/Backups/BackupFileInfo.h | 7 ++- src/Backups/BackupImpl.cpp | 29 ++++++--- src/Backups/BackupImpl.h | 7 +-- 4 files changed, 65 insertions(+), 43 deletions(-) diff --git a/src/Backups/BackupCoordinationFileInfos.cpp b/src/Backups/BackupCoordinationFileInfos.cpp index b17b755b966..3280064a5d7 100644 --- a/src/Backups/BackupCoordinationFileInfos.cpp +++ b/src/Backups/BackupCoordinationFileInfos.cpp @@ -50,6 +50,21 @@ BackupFileInfo BackupCoordinationFileInfos::getFileInfoByDataFileIndex(size_t da return *(file_infos_for_all_hosts[data_file_index]); } +namespace +{ + +/// copy all the file infos that are shared between reference target and source +void copyFileInfoToReference(const BackupFileInfo & target, BackupFileInfo & reference) +{ + reference.size = target.size; + reference.checksum = target.checksum; + reference.base_size = target.base_size; + reference.base_checksum = target.base_checksum; + reference.encrypted_by_disk = target.encrypted_by_disk; +} + +} + void BackupCoordinationFileInfos::prepare() const { if (prepared) @@ -78,11 +93,24 @@ void BackupCoordinationFileInfos::prepare() const num_files = 0; total_size_of_files = 0; + std::vector unresolved_references; + std::unordered_map file_name_to_info; + + const auto handle_unresolved_references = [&](const auto & try_resolve_reference) + { + for (auto * reference : unresolved_references) + { + if (!try_resolve_reference(*reference)) + throw DB::Exception( + ErrorCodes::LOGICAL_ERROR, + "Couldn't resolve reference {} with target {}", + reference->file_name, + reference->reference_target); + } + }; + if (plain_backup) { - std::vector unresolved_references; - std::unordered_map file_name_to_info; - const auto try_resolve_reference = [&](BackupFileInfo & reference) { auto it = file_name_to_info.find(reference.reference_target); @@ -91,10 +119,9 @@ void BackupCoordinationFileInfos::prepare() const return false; auto & target_info = it->second; - target_info->reference_sources.push_back(reference.file_name); - reference.size = target_info->size; + target_info->data_file_copies.push_back(reference.file_name); + copyFileInfoToReference(*target_info, reference); total_size_of_files += reference.size; - reference.checksum = target_info->checksum; return true; }; @@ -118,23 +145,12 @@ void BackupCoordinationFileInfos::prepare() const } } - for (auto * reference : unresolved_references) - { - if (!try_resolve_reference(*reference)) - throw DB::Exception( - ErrorCodes::LOGICAL_ERROR, - "Couldn't resolve reference {} with target {}", - reference->file_name, - reference->reference_target); - } + handle_unresolved_references(try_resolve_reference); num_files = file_infos_for_all_hosts.size(); } else { - std::vector unresolved_references; - std::unordered_map file_name_to_info; - const auto try_resolve_reference = [&](BackupFileInfo & reference) { auto it = file_name_to_info.find(reference.reference_target); @@ -143,8 +159,7 @@ void BackupCoordinationFileInfos::prepare() const return false; auto & target_info = it->second; - reference.size = target_info->size; - reference.checksum = target_info->checksum; + copyFileInfoToReference(*target_info, reference); reference.data_file_name = target_info->data_file_name; reference.data_file_index = target_info->data_file_index; return true; @@ -195,15 +210,7 @@ void BackupCoordinationFileInfos::prepare() const file_name_to_info.emplace(info.file_name, &info); } - for (auto * reference : unresolved_references) - { - if (!try_resolve_reference(*reference)) - throw DB::Exception( - ErrorCodes::LOGICAL_ERROR, - "Couldn't resolve reference {} with target {}", - reference->file_name, - reference->reference_target); - } + handle_unresolved_references(try_resolve_reference); num_files = file_infos_for_all_hosts.size(); } diff --git a/src/Backups/BackupFileInfo.h b/src/Backups/BackupFileInfo.h index 42bda3aa6ed..009fee091e0 100644 --- a/src/Backups/BackupFileInfo.h +++ b/src/Backups/BackupFileInfo.h @@ -42,9 +42,10 @@ struct BackupFileInfo /// Set if this file is just a reference to another file String reference_target; - /// List of files that are referencing this file - /// Used for plain backup which needs to resolve all references - Strings reference_sources; + /// (While writing a backup) if this list is not empty then after writing + /// `data_file_name` it should be copied to this list of destinations too. + /// This is used for plain backups. + Strings data_file_copies; struct LessByFileName { diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 56c30fab5c2..61984d58889 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -24,6 +24,8 @@ #include #include +#include + namespace ProfileEvents { @@ -452,7 +454,6 @@ void BackupImpl::readBackupMetadata() size_of_entries = 0; const auto * contents = config_root->getNodeByPath("contents"); - std::vector> reference_files; for (const Poco::XML::Node * child = contents->firstChild(); child; child = child->nextSibling()) { if (child->nodeName() == "file") @@ -913,15 +914,20 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) /// NOTE: `mutex` must be unlocked during copying otherwise writing will be in one thread maximum and hence slow. - if (use_archive) + const auto write_info_to_archive = [&](const auto & file_name) { - LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}, adding to archive", info.data_file_name, src_file_desc, info.data_file_index); - auto out = archive_writer->writeFile(info.data_file_name); + auto out = archive_writer->writeFile(file_name); auto read_buffer = entry->getReadBuffer(writer->getReadSettings()); if (info.base_size != 0) read_buffer->seek(info.base_size, SEEK_SET); copyData(*read_buffer, *out); out->finalize(); + }; + + if (use_archive) + { + LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}, adding to archive", info.data_file_name, src_file_desc, info.data_file_index); + write_info_to_archive(info.data_file_name); } else if (src_disk && from_immutable_file) { @@ -935,11 +941,20 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) writer->copyDataToFile(info.data_file_name, create_read_buffer, info.base_size, info.size - info.base_size); } - if (!deduplicate_files) + std::function copy_file_inside_backup; + if (use_archive) { - for (const auto & reference : info.reference_sources) - writer->copyFile(reference, info.data_file_name, info.size - info.base_size); + copy_file_inside_backup = write_info_to_archive; } + else + { + copy_file_inside_backup = [&](const auto & data_file_copy) + { + writer->copyFile(data_file_copy, info.data_file_name, info.size - info.base_size); + }; + } + + std::ranges::for_each(info.data_file_copies, copy_file_inside_backup); { std::lock_guard lock{mutex}; diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index a4ab3d84d0c..6070db79aa6 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -72,11 +72,14 @@ public: Strings listFiles(const String & directory, bool recursive) const override; bool hasFiles(const String & directory) const override; bool fileExists(const String & file_name) const override; + bool fileExists(const SizeAndChecksum & size_and_checksum) const override; UInt64 getFileSize(const String & file_name) const override; UInt128 getFileChecksum(const String & file_name) const override; SizeAndChecksum getFileSizeAndChecksum(const String & file_name) const override; std::unique_ptr readFile(const String & file_name) const override; + std::unique_ptr readFile(const SizeAndChecksum & size_and_checksum) const override; size_t copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const override; + size_t copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const override; void writeFile(const BackupFileInfo & info, BackupEntryPtr entry) override; void finalizeWriting() override; bool supportsWritingInMultipleThreads() const override { return !use_archive; } @@ -111,10 +114,6 @@ private: std::unique_ptr readFileImpl(const SizeAndChecksum & size_and_checksum, bool read_encrypted) const; - bool fileExists(const SizeAndChecksum & size_and_checksum) const override; - std::unique_ptr readFile(const SizeAndChecksum & size_and_checksum) const override; - size_t copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const override; - BackupInfo backup_info; const String backup_name_for_logging; const bool use_archive; From 9bcedf376436bb066db639a7ab7325cd71fb3b73 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Nov 2023 10:27:19 +0000 Subject: [PATCH 489/813] Cleanup --- src/Backups/BackupIO_Default.cpp | 2 -- src/Common/ZooKeeper/ZooKeeper.cpp | 1 - src/Storages/StorageKeeperMap.cpp | 40 ++++++++++++++++-------------- 3 files changed, 21 insertions(+), 22 deletions(-) diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp index 95f2c66b6b9..5ac522695ce 100644 --- a/src/Backups/BackupIO_Default.cpp +++ b/src/Backups/BackupIO_Default.cpp @@ -91,6 +91,4 @@ void BackupWriterDefault::copyFileFromDisk(const String & path_in_backup, DiskPt copyDataToFile(path_in_backup, create_read_buffer, start_pos, length); } - - } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 8a97362aa96..436a4e14f14 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -385,7 +385,6 @@ void ZooKeeper::createAncestors(const std::string & path) size_t last_pos = path.rfind('/'); if (last_pos == std::string::npos || last_pos == 0) return; - std::string current_node = path.substr(0, last_pos); while (true) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 237b65c6a72..e3c960529de 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -763,28 +763,29 @@ void StorageKeeperMap::backupData(BackupEntriesCollector & backup_entries_collec auto post_collecting_task = [my_table_id = std::move(table_id), coordination, &backup_entries_collector, my_data_path_in_backup = data_path_in_backup, this] { auto path_with_data = coordination->getKeeperMapDataPath(zk_root_path); - if (path_with_data == my_data_path_in_backup) + if (path_with_data != my_data_path_in_backup) { - auto temp_disk = backup_entries_collector.getContext()->getGlobalTemporaryVolume()->getDisk(0); - auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size; - - auto with_retries = std::make_shared - ( - &Poco::Logger::get(fmt::format("StorageKeeperMapBackup ({})", getStorageID().getNameForLogs())), - [&] { return getClient(); }, - WithRetries::KeeperSettings::fromContext(backup_entries_collector.getContext()), - [](WithRetries::FaultyKeeper &) {} - ); - - backup_entries_collector.addBackupEntries( - std::make_shared(this->zk_data_path, path_with_data, temp_disk, max_compress_block_size, std::move(with_retries)) - ->getBackupEntries()); + std::string source_path = fs::path(my_data_path_in_backup) / backup_data_filename; + std::string target_path = fs::path(path_with_data) / backup_data_filename; + backup_entries_collector.addBackupEntries({{source_path, std::make_shared(std::move(target_path))}}); return; } - std::string source_path = fs::path(my_data_path_in_backup) / backup_data_filename; - std::string target_path = fs::path(path_with_data) / backup_data_filename; - backup_entries_collector.addBackupEntries({{source_path, std::make_shared(std::move(target_path))}}); + auto temp_disk = backup_entries_collector.getContext()->getGlobalTemporaryVolume()->getDisk(0); + auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size; + + auto with_retries = std::make_shared + ( + &Poco::Logger::get(fmt::format("StorageKeeperMapBackup ({})", getStorageID().getNameForLogs())), + [&] { return getClient(); }, + WithRetries::KeeperSettings::fromContext(backup_entries_collector.getContext()), + [](WithRetries::FaultyKeeper &) {} + ); + + backup_entries_collector.addBackupEntries( + std::make_shared( + this->zk_data_path, path_with_data, temp_disk, max_compress_block_size, std::move(with_retries)) + ->getBackupEntries()); }; backup_entries_collector.addPostTask(post_collecting_task); @@ -796,7 +797,8 @@ void StorageKeeperMap::restoreDataFromBackup(RestorerFromBackup & restorer, cons if (!backup->hasFiles(data_path_in_backup)) return; - auto table_id = toString(getStorageID().uuid); if (!restorer.getRestoreCoordination()->acquireInsertingDataForKeeperMap(zk_root_path, table_id)) + auto table_id = toString(getStorageID().uuid); + if (!restorer.getRestoreCoordination()->acquireInsertingDataForKeeperMap(zk_root_path, table_id)) { /// Other table is already restoring the data for this Keeper path. /// Tables defined on the same path share data From 4f441ec1319ac3f80a70f773f98ea99a80dfc407 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 11:32:27 +0100 Subject: [PATCH 490/813] Own CMake for Abseil --- contrib/abseil-cpp-cmake/CMakeLists.txt | 3451 ++++++++++++++++++++++- contrib/re2-cmake/CMakeLists.txt | 13 +- docker/packager/binary/build.sh | 2 +- 3 files changed, 3436 insertions(+), 30 deletions(-) diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index 2901daf32db..e84b4d46c4a 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -1,33 +1,3428 @@ set(ABSL_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") +set(ABSL_COMMON_INCLUDE_DIRS "${ABSL_ROOT_DIR}") + +# +# Copyright 2017 The Abseil Authors. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# https://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# 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" + "NAME" + "HDRS;SRCS;COPTS;DEFINES;LINKOPTS;DEPS" + ${ARGN} + ) + + set(_NAME "absl_${ABSL_CC_LIB_NAME}") + + # Check if this is a header-only library + set(ABSL_CC_SRCS "${ABSL_CC_LIB_SRCS}") + foreach(src_file IN LISTS ABSL_CC_SRCS) + if(${src_file} MATCHES ".*\\.(h|inc)") + list(REMOVE_ITEM ABSL_CC_SRCS "${src_file}") + endif() + endforeach() + + if(ABSL_CC_SRCS STREQUAL "") + set(ABSL_CC_LIB_IS_INTERFACE 1) + else() + set(ABSL_CC_LIB_IS_INTERFACE 0) + endif() + + if(NOT ABSL_CC_LIB_IS_INTERFACE) + add_library(${_NAME} "") + target_sources(${_NAME} PRIVATE ${ABSL_CC_LIB_SRCS} ${ABSL_CC_LIB_HDRS}) + target_link_libraries(${_NAME} + PUBLIC ${ABSL_CC_LIB_DEPS} + PRIVATE + ${ABSL_CC_LIB_LINKOPTS} + ${ABSL_DEFAULT_LINKOPTS} + ) + + target_include_directories(${_NAME} + PUBLIC "${ABSL_COMMON_INCLUDE_DIRS}") + target_compile_options(${_NAME} + PRIVATE ${ABSL_CC_LIB_COPTS}) + target_compile_definitions(${_NAME} PUBLIC ${ABSL_CC_LIB_DEFINES}) + + else() + # Generating header-only library + add_library(${_NAME} INTERFACE) + target_include_directories(${_NAME} + INTERFACE "${ABSL_COMMON_INCLUDE_DIRS}") + + target_link_libraries(${_NAME} + INTERFACE + ${ABSL_CC_LIB_DEPS} + ${ABSL_CC_LIB_LINKOPTS} + ${ABSL_DEFAULT_LINKOPTS} + ) + target_compile_definitions(${_NAME} INTERFACE ${ABSL_CC_LIB_DEFINES}) + + endif() + + add_library(absl::${ABSL_CC_LIB_NAME} ALIAS ${_NAME}) +endfunction() + + +set(DIR ${ABSL_ROOT_DIR}/absl/algorithm) + +absl_cc_library( + NAME + algorithm + HDRS + "${DIR}/algorithm.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) + +absl_cc_library( + NAME + algorithm_container + HDRS + "${DIR}/container.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::algorithm + absl::core_headers + absl::meta + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/base) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + atomic_hook + HDRS + "${DIR}/internal/atomic_hook.h" + DEPS + absl::config + absl::core_headers + COPTS + ${ABSL_DEFAULT_COPTS} +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + errno_saver + HDRS + "${DIR}/internal/errno_saver.h" + DEPS + absl::config + COPTS + ${ABSL_DEFAULT_COPTS} +) + +absl_cc_library( + NAME + log_severity + HDRS + "${DIR}/log_severity.h" + SRCS + "${DIR}/log_severity.cc" + DEPS + absl::config + absl::core_headers + COPTS + ${ABSL_DEFAULT_COPTS} +) + +absl_cc_library( + NAME + nullability + HDRS + "${DIR}/nullability.h" + SRCS + "${DIR}/internal/nullability_impl.h" + DEPS + absl::core_headers + absl::type_traits + COPTS + ${ABSL_DEFAULT_COPTS} +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + raw_logging_internal + HDRS + "${DIR}/internal/raw_logging.h" + SRCS + "${DIR}/internal/raw_logging.cc" + DEPS + absl::atomic_hook + absl::config + absl::core_headers + absl::errno_saver + absl::log_severity + COPTS + ${ABSL_DEFAULT_COPTS} +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + spinlock_wait + HDRS + "${DIR}/internal/spinlock_wait.h" + SRCS + "${DIR}/internal/spinlock_akaros.inc" + "${DIR}/internal/spinlock_linux.inc" + "${DIR}/internal/spinlock_posix.inc" + "${DIR}/internal/spinlock_wait.cc" + "${DIR}/internal/spinlock_win32.inc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base_internal + absl::core_headers + absl::errno_saver +) + +absl_cc_library( + NAME + config + HDRS + "${DIR}/config.h" + "${DIR}/options.h" + "${DIR}/policy_checks.h" + COPTS + ${ABSL_DEFAULT_COPTS} + PUBLIC +) + +absl_cc_library( + NAME + dynamic_annotations + HDRS + "${DIR}/dynamic_annotations.h" + SRCS + "${DIR}/internal/dynamic_annotations.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) + +absl_cc_library( + NAME + core_headers + HDRS + "${DIR}/attributes.h" + "${DIR}/const_init.h" + "${DIR}/macros.h" + "${DIR}/optimization.h" + "${DIR}/port.h" + "${DIR}/thread_annotations.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + malloc_internal + HDRS + "${DIR}/internal/direct_mmap.h" + "${DIR}/internal/low_level_alloc.h" + SRCS + "${DIR}/internal/low_level_alloc.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::base_internal + absl::config + absl::core_headers + absl::dynamic_annotations + absl::raw_logging_internal + Threads::Threads +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + base_internal + HDRS + "${DIR}/internal/hide_ptr.h" + "${DIR}/internal/identity.h" + "${DIR}/internal/inline_variable.h" + "${DIR}/internal/invoke.h" + "${DIR}/internal/scheduling_mode.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::type_traits +) + +absl_cc_library( + NAME + base + HDRS + "${DIR}/call_once.h" + "${DIR}/casts.h" + "${DIR}/internal/cycleclock.h" + "${DIR}/internal/cycleclock_config.h" + "${DIR}/internal/low_level_scheduling.h" + "${DIR}/internal/per_thread_tls.h" + "${DIR}/internal/spinlock.h" + "${DIR}/internal/sysinfo.h" + "${DIR}/internal/thread_identity.h" + "${DIR}/internal/tsan_mutex_interface.h" + "${DIR}/internal/unscaledcycleclock.h" + "${DIR}/internal/unscaledcycleclock_config.h" + SRCS + "${DIR}/internal/cycleclock.cc" + "${DIR}/internal/spinlock.cc" + "${DIR}/internal/sysinfo.cc" + "${DIR}/internal/thread_identity.cc" + "${DIR}/internal/unscaledcycleclock.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::atomic_hook + absl::base_internal + absl::config + absl::core_headers + absl::dynamic_annotations + absl::log_severity + absl::raw_logging_internal + absl::spinlock_wait + absl::type_traits + Threads::Threads + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + throw_delegate + HDRS + "${DIR}/internal/throw_delegate.h" + SRCS + "${DIR}/internal/throw_delegate.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::raw_logging_internal +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + pretty_function + HDRS + "${DIR}/internal/pretty_function.h" + COPTS + ${ABSL_DEFAULT_COPTS} +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + endian + HDRS + "${DIR}/internal/endian.h" + "${DIR}/internal/unaligned_access.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config + absl::core_headers + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + scoped_set_env + SRCS + "${DIR}/internal/scoped_set_env.cc" + HDRS + "${DIR}/internal/scoped_set_env.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::raw_logging_internal +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + strerror + SRCS + "${DIR}/internal/strerror.cc" + HDRS + "${DIR}/internal/strerror.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::errno_saver +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + fast_type_id + HDRS + "${DIR}/internal/fast_type_id.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config +) + +absl_cc_library( + NAME + prefetch + HDRS + "${DIR}/prefetch.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers +) + +set(DIR ${ABSL_ROOT_DIR}/absl/cleanup) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cleanup_internal + HDRS + "${DIR}/internal/cleanup.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base_internal + absl::core_headers + absl::utility + PUBLIC +) + +absl_cc_library( + NAME + cleanup + HDRS + "${DIR}/cleanup.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::cleanup_internal + absl::config + absl::core_headers + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/container) + +absl_cc_library( + NAME + btree + HDRS + "${DIR}/btree_map.h" + "${DIR}/btree_set.h" + "${DIR}/internal/btree.h" + "${DIR}/internal/btree_container.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::container_common + absl::common_policy_traits + absl::compare + absl::compressed_tuple + absl::container_memory + absl::cord + absl::core_headers + absl::layout + absl::memory + absl::raw_logging_internal + absl::strings + absl::throw_delegate + absl::type_traits + absl::utility +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + compressed_tuple + HDRS + "${DIR}/internal/compressed_tuple.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::utility + PUBLIC +) + +absl_cc_library( + NAME + fixed_array + HDRS + "${DIR}/fixed_array.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::compressed_tuple + absl::algorithm + absl::config + absl::core_headers + absl::dynamic_annotations + absl::throw_delegate + absl::memory + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + inlined_vector_internal + HDRS + "${DIR}/internal/inlined_vector.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::compressed_tuple + absl::core_headers + absl::memory + absl::span + absl::type_traits + PUBLIC +) + +absl_cc_library( + NAME + inlined_vector + HDRS + "${DIR}/inlined_vector.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::algorithm + absl::core_headers + absl::inlined_vector_internal + absl::throw_delegate + absl::memory + absl::type_traits + 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 + HDRS + "${DIR}/flat_hash_map.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::container_memory + absl::core_headers + absl::hash_function_defaults + absl::raw_hash_map + absl::algorithm_container + absl::memory + PUBLIC +) + +absl_cc_library( + NAME + flat_hash_set + HDRS + "${DIR}/flat_hash_set.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::container_memory + absl::hash_function_defaults + absl::raw_hash_set + absl::algorithm_container + absl::core_headers + absl::memory + PUBLIC +) + +absl_cc_library( + NAME + node_hash_map + HDRS + "${DIR}/node_hash_map.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::container_memory + absl::core_headers + absl::hash_function_defaults + absl::node_slot_policy + absl::raw_hash_map + absl::algorithm_container + absl::memory + PUBLIC +) + +absl_cc_library( + NAME + node_hash_set + HDRS + "${DIR}/node_hash_set.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::core_headers + absl::hash_function_defaults + absl::node_slot_policy + absl::raw_hash_set + absl::algorithm_container + absl::memory + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + container_memory + HDRS + "${DIR}/internal/container_memory.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::memory + absl::type_traits + absl::utility + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + hash_function_defaults + HDRS + "${DIR}/internal/hash_function_defaults.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::cord + absl::hash + absl::strings + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + hash_policy_traits + HDRS + "${DIR}/internal/hash_policy_traits.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::common_policy_traits + absl::meta + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + common_policy_traits + HDRS + "${DIR}/internal/common_policy_traits.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::meta + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + hashtablez_sampler + HDRS + "${DIR}/internal/hashtablez_sampler.h" + SRCS + "${DIR}/internal/hashtablez_sampler.cc" + "${DIR}/internal/hashtablez_sampler_force_weak_definition.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config + absl::exponential_biased + absl::raw_logging_internal + absl::sample_recorder + absl::synchronization + absl::time +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + hashtable_debug + HDRS + "${DIR}/internal/hashtable_debug.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::hashtable_debug_hooks +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + hashtable_debug_hooks + HDRS + "${DIR}/internal/hashtable_debug_hooks.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + node_slot_policy + HDRS + "${DIR}/internal/node_slot_policy.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + raw_hash_map + HDRS + "${DIR}/internal/raw_hash_map.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::container_memory + absl::raw_hash_set + absl::throw_delegate + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + container_common + HDRS + "${DIR}/internal/common.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + raw_hash_set + HDRS + "${DIR}/internal/raw_hash_set.h" + SRCS + "${DIR}/internal/raw_hash_set.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::bits + absl::compressed_tuple + absl::config + absl::container_common + absl::container_memory + absl::core_headers + absl::dynamic_annotations + absl::endian + absl::hash + absl::hash_policy_traits + absl::hashtable_debug_hooks + absl::hashtablez_sampler + absl::memory + absl::meta + absl::optional + absl::prefetch + absl::raw_logging_internal + absl::utility + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + layout + HDRS + "${DIR}/internal/layout.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers + absl::meta + absl::strings + absl::span + absl::utility + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/crc) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + crc_cpu_detect + HDRS + "${DIR}/internal/cpu_detect.h" + SRCS + "${DIR}/internal/cpu_detect.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + crc_internal + HDRS + "${DIR}/internal/crc.h" + "${DIR}/internal/crc32_x86_arm_combined_simd.h" + SRCS + "${DIR}/internal/crc.cc" + "${DIR}/internal/crc_internal.h" + "${DIR}/internal/crc_x86_arm_combined.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::crc_cpu_detect + absl::config + absl::core_headers + absl::endian + absl::prefetch + absl::raw_logging_internal + absl::memory + absl::bits +) + +absl_cc_library( + NAME + crc32c + HDRS + "${DIR}/crc32c.h" + "${DIR}/internal/crc32c.h" + "${DIR}/internal/crc_memcpy.h" + SRCS + "${DIR}/crc32c.cc" + "${DIR}/internal/crc32c_inline.h" + "${DIR}/internal/crc_memcpy_fallback.cc" + "${DIR}/internal/crc_memcpy_x86_arm_combined.cc" + "${DIR}/internal/crc_non_temporal_memcpy.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::crc_cpu_detect + absl::crc_internal + absl::non_temporal_memcpy + absl::config + absl::core_headers + absl::endian + absl::prefetch + absl::str_format + absl::strings +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + non_temporal_arm_intrinsics + HDRS + "${DIR}/internal/non_temporal_arm_intrinsics.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + non_temporal_memcpy + HDRS + "${DIR}/internal/non_temporal_memcpy.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::non_temporal_arm_intrinsics + absl::config + absl::core_headers +) + +absl_cc_library( + NAME + crc_cord_state + HDRS + "${DIR}/internal/crc_cord_state.h" + SRCS + "${DIR}/internal/crc_cord_state.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::crc32c + absl::config + absl::strings +) + +set(DIR ${ABSL_ROOT_DIR}/absl/debugging) + +absl_cc_library( + NAME + stacktrace + HDRS + "${DIR}/stacktrace.h" + "${DIR}/internal/stacktrace_aarch64-inl.inc" + "${DIR}/internal/stacktrace_arm-inl.inc" + "${DIR}/internal/stacktrace_config.h" + "${DIR}/internal/stacktrace_emscripten-inl.inc" + "${DIR}/internal/stacktrace_generic-inl.inc" + "${DIR}/internal/stacktrace_powerpc-inl.inc" + "${DIR}/internal/stacktrace_riscv-inl.inc" + "${DIR}/internal/stacktrace_unimplemented-inl.inc" + "${DIR}/internal/stacktrace_win32-inl.inc" + "${DIR}/internal/stacktrace_x86-inl.inc" + SRCS + "${DIR}/stacktrace.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::debugging_internal + absl::config + absl::core_headers + absl::dynamic_annotations + absl::raw_logging_internal + PUBLIC +) + +absl_cc_library( + NAME + symbolize + HDRS + "${DIR}/symbolize.h" + "${DIR}/internal/symbolize.h" + SRCS + "${DIR}/symbolize.cc" + "${DIR}/symbolize_darwin.inc" + "${DIR}/symbolize_elf.inc" + "${DIR}/symbolize_emscripten.inc" + "${DIR}/symbolize_unimplemented.inc" + "${DIR}/symbolize_win32.inc" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::debugging_internal + absl::demangle_internal + absl::base + absl::config + absl::core_headers + absl::dynamic_annotations + absl::malloc_internal + absl::raw_logging_internal + absl::strings + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + examine_stack + HDRS + "${DIR}/internal/examine_stack.h" + SRCS + "${DIR}/internal/examine_stack.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::stacktrace + absl::symbolize + absl::config + absl::core_headers + absl::raw_logging_internal +) + +absl_cc_library( + NAME + failure_signal_handler + HDRS + "${DIR}/failure_signal_handler.h" + SRCS + "${DIR}/failure_signal_handler.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::examine_stack + absl::stacktrace + absl::base + absl::config + absl::core_headers + absl::raw_logging_internal + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + debugging_internal + HDRS + "${DIR}/internal/address_is_readable.h" + "${DIR}/internal/elf_mem_image.h" + "${DIR}/internal/vdso_support.h" + SRCS + "${DIR}/internal/address_is_readable.cc" + "${DIR}/internal/elf_mem_image.cc" + "${DIR}/internal/vdso_support.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::core_headers + absl::config + absl::dynamic_annotations + absl::errno_saver + absl::raw_logging_internal +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + demangle_internal + HDRS + "${DIR}/internal/demangle.h" + SRCS + "${DIR}/internal/demangle.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::core_headers + PUBLIC +) + +absl_cc_library( + NAME + leak_check + HDRS + "${DIR}/leak_check.h" + SRCS + "${DIR}/leak_check.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers + PUBLIC +) + +# component target +absl_cc_library( + NAME + debugging + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::stacktrace + absl::leak_check + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/flags) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + flags_path_util + HDRS + "${DIR}/internal/path_util.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::strings + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + flags_program_name + SRCS + "${DIR}/internal/program_name.cc" + HDRS + "${DIR}/internal/program_name.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::flags_path_util + absl::strings + absl::synchronization + PUBLIC +) + +absl_cc_library( + NAME + flags_config + SRCS + "${DIR}/usage_config.cc" + HDRS + "${DIR}/config.h" + "${DIR}/usage_config.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::flags_path_util + absl::flags_program_name + absl::core_headers + absl::strings + absl::synchronization +) + +absl_cc_library( + NAME + flags_marshalling + SRCS + "${DIR}/marshalling.cc" + HDRS + "${DIR}/marshalling.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::log_severity + absl::int128 + absl::optional + absl::strings + absl::str_format +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + flags_commandlineflag_internal + SRCS + "${DIR}/internal/commandlineflag.cc" + HDRS + "${DIR}/internal/commandlineflag.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::dynamic_annotations + absl::fast_type_id +) + +absl_cc_library( + NAME + flags_commandlineflag + SRCS + "${DIR}/commandlineflag.cc" + HDRS + "${DIR}/commandlineflag.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::fast_type_id + absl::flags_commandlineflag_internal + absl::optional + absl::strings +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + flags_private_handle_accessor + SRCS + "${DIR}/internal/private_handle_accessor.cc" + HDRS + "${DIR}/internal/private_handle_accessor.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::flags_commandlineflag + absl::flags_commandlineflag_internal + absl::strings +) + +absl_cc_library( + NAME + flags_reflection + SRCS + "${DIR}/reflection.cc" + HDRS + "${DIR}/reflection.h" + "${DIR}/internal/registry.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::flags_commandlineflag + absl::flags_private_handle_accessor + absl::flags_config + absl::strings + absl::synchronization + absl::flat_hash_map +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + flags_internal + SRCS + "${DIR}/internal/flag.cc" + HDRS + "${DIR}/internal/flag.h" + "${DIR}/internal/sequence_lock.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::base + absl::config + absl::flags_commandlineflag + absl::flags_commandlineflag_internal + absl::flags_config + absl::flags_marshalling + absl::synchronization + absl::meta + absl::utility + PUBLIC +) + +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 + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::flags_commandlineflag + absl::flags_config + absl::flags_internal + absl::flags_reflection + absl::base + absl::core_headers + absl::strings +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + flags_usage_internal + SRCS + "${DIR}/internal/usage.cc" + HDRS + "${DIR}/internal/usage.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::flags_config + absl::flags + absl::flags_commandlineflag + absl::flags_internal + absl::flags_path_util + absl::flags_private_handle_accessor + absl::flags_program_name + absl::flags_reflection + absl::strings + absl::synchronization +) + +absl_cc_library( + NAME + flags_usage + SRCS + "${DIR}/usage.cc" + HDRS + "${DIR}/usage.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::flags_usage_internal + absl::raw_logging_internal + absl::strings + absl::synchronization +) + +absl_cc_library( + NAME + flags_parse + SRCS + "${DIR}/parse.cc" + HDRS + "${DIR}/internal/parse.h" + "${DIR}/parse.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::algorithm_container + absl::config + absl::core_headers + absl::flags_config + absl::flags + absl::flags_commandlineflag + absl::flags_commandlineflag_internal + absl::flags_internal + absl::flags_private_handle_accessor + absl::flags_program_name + absl::flags_reflection + absl::flags_usage + absl::strings + absl::synchronization +) + +set(DIR ${ABSL_ROOT_DIR}/absl/functional) + +absl_cc_library( + NAME + any_invocable + SRCS + "${DIR}/internal/any_invocable.h" + HDRS + "${DIR}/any_invocable.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base_internal + absl::config + absl::core_headers + absl::type_traits + absl::utility + PUBLIC +) + +absl_cc_library( + NAME + bind_front + SRCS + "${DIR}/internal/front_binder.h" + HDRS + "${DIR}/bind_front.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base_internal + absl::compressed_tuple + PUBLIC +) + +absl_cc_library( + NAME + function_ref + SRCS + "${DIR}/internal/function_ref.h" + HDRS + "${DIR}/function_ref.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base_internal + absl::core_headers + absl::any_invocable + absl::meta + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/hash) + +absl_cc_library( + NAME + hash + HDRS + "${DIR}/hash.h" + SRCS + "${DIR}/internal/hash.cc" + "${DIR}/internal/hash.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::bits + absl::city + absl::config + absl::core_headers + absl::endian + absl::fixed_array + absl::function_ref + absl::meta + absl::int128 + absl::strings + absl::optional + absl::variant + absl::utility + absl::low_level_hash + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + city + HDRS + "${DIR}/internal/city.h" + SRCS + "${DIR}/internal/city.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers + absl::endian +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + low_level_hash + HDRS + "${DIR}/internal/low_level_hash.h" + SRCS + "${DIR}/internal/low_level_hash.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::endian + absl::int128 + absl::prefetch +) + +set(DIR ${ABSL_ROOT_DIR}/absl/log) + +# Internal targets +absl_cc_library( + NAME + log_internal_check_impl + SRCS + HDRS + "${DIR}/internal/check_impl.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::core_headers + absl::log_internal_check_op + absl::log_internal_conditions + absl::log_internal_message + absl::log_internal_strip +) + +absl_cc_library( + NAME + log_internal_check_op + SRCS + "${DIR}/internal/check_op.cc" + HDRS + "${DIR}/internal/check_op.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::log_internal_nullguard + absl::log_internal_nullstream + absl::log_internal_strip + absl::strings +) + +absl_cc_library( + NAME + log_internal_conditions + SRCS + "${DIR}/internal/conditions.cc" + HDRS + "${DIR}/internal/conditions.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::log_internal_voidify +) + +absl_cc_library( + NAME + log_internal_config + SRCS + HDRS + "${DIR}/internal/config.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers +) + +absl_cc_library( + NAME + log_internal_flags + SRCS + HDRS + "${DIR}/internal/flags.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::flags +) + +absl_cc_library( + NAME + log_internal_format + SRCS + "${DIR}/internal/log_format.cc" + HDRS + "${DIR}/internal/log_format.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::log_internal_append_truncated + absl::log_internal_config + absl::log_internal_globals + absl::log_severity + absl::strings + absl::str_format + absl::time + absl::span +) + +absl_cc_library( + NAME + log_internal_globals + SRCS + "${DIR}/internal/globals.cc" + HDRS + "${DIR}/internal/globals.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::log_severity + absl::raw_logging_internal + absl::strings + absl::time +) + +absl_cc_library( + NAME + log_internal_log_impl + SRCS + HDRS + "${DIR}/internal/log_impl.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::log_internal_conditions + absl::log_internal_message + absl::log_internal_strip +) + +absl_cc_library( + NAME + log_internal_proto + SRCS + "${DIR}/internal/proto.cc" + HDRS + "${DIR}/internal/proto.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::strings + absl::span +) + +absl_cc_library( + NAME + log_internal_message + SRCS + "${DIR}/internal/log_message.cc" + HDRS + "${DIR}/internal/log_message.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::errno_saver + absl::inlined_vector + absl::examine_stack + absl::log_internal_append_truncated + absl::log_internal_format + absl::log_internal_globals + absl::log_internal_proto + absl::log_internal_log_sink_set + absl::log_internal_nullguard + absl::log_globals + absl::log_entry + absl::log_severity + absl::log_sink + absl::log_sink_registry + absl::memory + absl::raw_logging_internal + absl::strings + absl::strerror + absl::time + absl::span +) + +absl_cc_library( + NAME + log_internal_log_sink_set + SRCS + "${DIR}/internal/log_sink_set.cc" + HDRS + "${DIR}/internal/log_sink_set.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + $<$:-llog> + DEPS + absl::base + absl::cleanup + absl::config + absl::core_headers + absl::log_internal_config + absl::log_internal_globals + absl::log_globals + absl::log_entry + absl::log_severity + absl::log_sink + absl::raw_logging_internal + absl::synchronization + absl::span + absl::strings +) + +absl_cc_library( + NAME + log_internal_nullguard + SRCS + "${DIR}/internal/nullguard.cc" + HDRS + "${DIR}/internal/nullguard.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers +) + +absl_cc_library( + NAME + log_internal_nullstream + SRCS + HDRS + "${DIR}/internal/nullstream.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::log_severity + absl::strings +) + +absl_cc_library( + NAME + log_internal_strip + SRCS + HDRS + "${DIR}/internal/strip.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::log_internal_message + absl::log_internal_nullstream + absl::log_severity +) + +absl_cc_library( + NAME + log_internal_voidify + SRCS + HDRS + "${DIR}/internal/voidify.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config +) + +absl_cc_library( + NAME + log_internal_append_truncated + SRCS + HDRS + "${DIR}/internal/append_truncated.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::strings + absl::span +) + +# Public targets +absl_cc_library( + NAME + absl_check + SRCS + HDRS + "${DIR}/absl_check.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::log_internal_check_impl + PUBLIC +) + +absl_cc_library( + NAME + absl_log + SRCS + HDRS + "${DIR}/absl_log.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::log_internal_log_impl + PUBLIC +) + +absl_cc_library( + NAME + check + SRCS + HDRS + "${DIR}/check.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::log_internal_check_impl + absl::core_headers + absl::log_internal_check_op + absl::log_internal_conditions + absl::log_internal_message + absl::log_internal_strip + PUBLIC +) + +absl_cc_library( + NAME + die_if_null + SRCS + "${DIR}/die_if_null.cc" + HDRS + "${DIR}/die_if_null.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::log + absl::strings + PUBLIC +) + +absl_cc_library( + NAME + log_flags + SRCS + "${DIR}/flags.cc" + HDRS + "${DIR}/flags.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::log_globals + absl::log_severity + absl::log_internal_config + absl::log_internal_flags + absl::flags + absl::flags_marshalling + absl::strings + PUBLIC +) + +absl_cc_library( + NAME + log_globals + SRCS + "${DIR}/globals.cc" + HDRS + "${DIR}/globals.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::atomic_hook + absl::config + absl::core_headers + absl::hash + absl::log_severity + absl::raw_logging_internal + absl::strings +) + +absl_cc_library( + NAME + log_initialize + SRCS + "${DIR}/initialize.cc" + HDRS + "${DIR}/initialize.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::log_globals + absl::log_internal_globals + absl::time + PUBLIC +) + +absl_cc_library( + NAME + log + SRCS + HDRS + "${DIR}/log.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::log_internal_log_impl + PUBLIC +) + +absl_cc_library( + NAME + log_entry + SRCS + "${DIR}/log_entry.cc" + HDRS + "${DIR}/log_entry.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::log_internal_config + absl::log_severity + absl::span + absl::strings + absl::time + PUBLIC +) + +absl_cc_library( + NAME + log_sink + SRCS + "${DIR}/log_sink.cc" + HDRS + "${DIR}/log_sink.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::log_entry + PUBLIC +) + +absl_cc_library( + NAME + log_sink_registry + SRCS + HDRS + "${DIR}/log_sink_registry.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::log_sink + absl::log_internal_log_sink_set + PUBLIC +) + +absl_cc_library( + NAME + log_streamer + SRCS + HDRS + "${DIR}/log_streamer.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::absl_log + absl::log_severity + absl::optional + absl::strings + absl::strings_internal + absl::utility + PUBLIC +) + +absl_cc_library( + NAME + log_internal_structured + HDRS + "${DIR}/internal/structured.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::log_internal_message + absl::strings +) + +absl_cc_library( + NAME + log_structured + HDRS + "${DIR}/structured.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::log_internal_structured + absl::strings + PUBLIC +) + +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 +) + +set(DIR ${ABSL_ROOT_DIR}/absl/memory) + +absl_cc_library( + NAME + memory + HDRS + "${DIR}/memory.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::core_headers + absl::meta + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/meta) + +absl_cc_library( + NAME + type_traits + HDRS + "${DIR}/type_traits.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers + PUBLIC +) + +# component target +absl_cc_library( + NAME + meta + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::type_traits + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/numeric) + +absl_cc_library( + NAME + bits + HDRS + "${DIR}/bits.h" + "${DIR}/internal/bits.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::core_headers + PUBLIC +) + +absl_cc_library( + NAME + int128 + HDRS + "${DIR}/int128.h" + SRCS + "${DIR}/int128.cc" + "${DIR}/int128_have_intrinsic.inc" + "${DIR}/int128_no_intrinsic.inc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers + absl::bits + PUBLIC +) + +# component target +absl_cc_library( + NAME + numeric + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::int128 + PUBLIC +) + +absl_cc_library( + NAME + numeric_representation + HDRS + "${DIR}/internal/representation.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) + +absl_cc_library( + NAME + sample_recorder + HDRS + "${DIR}/internal/sample_recorder.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::synchronization +) + +set(DIR ${ABSL_ROOT_DIR}/absl/profiling) + +absl_cc_library( + NAME + exponential_biased + SRCS + "${DIR}/internal/exponential_biased.cc" + HDRS + "${DIR}/internal/exponential_biased.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers +) + +absl_cc_library( + NAME + periodic_sampler + SRCS + "${DIR}/internal/periodic_sampler.cc" + HDRS + "${DIR}/internal/periodic_sampler.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::core_headers + absl::exponential_biased +) + +set(DIR ${ABSL_ROOT_DIR}/absl/random) + +absl_cc_library( + NAME + random_random + HDRS + "${DIR}/random.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::random_distributions + absl::random_internal_nonsecure_base + absl::random_internal_pcg_engine + absl::random_internal_pool_urbg + absl::random_internal_randen_engine + absl::random_seed_sequences +) + +absl_cc_library( + NAME + random_bit_gen_ref + HDRS + "${DIR}/bit_gen_ref.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::core_headers + absl::random_internal_distribution_caller + absl::random_internal_fast_uniform_bits + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_mock_helpers + HDRS + "${DIR}/internal/mock_helpers.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::fast_type_id + absl::optional +) + +absl_cc_library( + NAME + random_distributions + SRCS + "${DIR}/discrete_distribution.cc" + "${DIR}/gaussian_distribution.cc" + HDRS + "${DIR}/bernoulli_distribution.h" + "${DIR}/beta_distribution.h" + "${DIR}/discrete_distribution.h" + "${DIR}/distributions.h" + "${DIR}/exponential_distribution.h" + "${DIR}/gaussian_distribution.h" + "${DIR}/log_uniform_int_distribution.h" + "${DIR}/poisson_distribution.h" + "${DIR}/uniform_int_distribution.h" + "${DIR}/uniform_real_distribution.h" + "${DIR}/zipf_distribution.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::base_internal + absl::config + absl::core_headers + absl::random_internal_generate_real + absl::random_internal_distribution_caller + absl::random_internal_fast_uniform_bits + absl::random_internal_fastmath + absl::random_internal_iostream_state_saver + absl::random_internal_traits + absl::random_internal_uniform_helper + absl::random_internal_wide_multiply + absl::strings + absl::type_traits +) + +absl_cc_library( + NAME + random_seed_gen_exception + SRCS + "${DIR}/seed_gen_exception.cc" + HDRS + "${DIR}/seed_gen_exception.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config +) + +absl_cc_library( + NAME + random_seed_sequences + SRCS + "${DIR}/seed_sequences.cc" + HDRS + "${DIR}/seed_sequences.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::inlined_vector + absl::random_internal_pool_urbg + absl::random_internal_salted_seed_seq + absl::random_internal_seed_material + absl::random_seed_gen_exception + absl::span +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_traits + HDRS + "${DIR}/internal/traits.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_distribution_caller + HDRS + "${DIR}/internal/distribution_caller.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::utility + absl::fast_type_id +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_fast_uniform_bits + HDRS + "${DIR}/internal/fast_uniform_bits.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_seed_material + SRCS + "${DIR}/internal/seed_material.cc" + HDRS + "${DIR}/internal/seed_material.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::core_headers + absl::optional + absl::random_internal_fast_uniform_bits + absl::raw_logging_internal + absl::span + absl::strings +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_pool_urbg + SRCS + "${DIR}/internal/pool_urbg.cc" + HDRS + "${DIR}/internal/pool_urbg.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::endian + absl::random_internal_randen + absl::random_internal_seed_material + absl::random_internal_traits + absl::random_seed_gen_exception + absl::raw_logging_internal + absl::span +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_salted_seed_seq + HDRS + "${DIR}/internal/salted_seed_seq.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::inlined_vector + absl::optional + absl::span + absl::random_internal_seed_material + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_iostream_state_saver + HDRS + "${DIR}/internal/iostream_state_saver.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::int128 + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_generate_real + HDRS + "${DIR}/internal/generate_real.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::bits + absl::random_internal_fastmath + absl::random_internal_traits + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_wide_multiply + HDRS + "${DIR}/internal/wide_multiply.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::bits + absl::config + absl::int128 +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_fastmath + HDRS + "${DIR}/internal/fastmath.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::bits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_nonsecure_base + HDRS + "${DIR}/internal/nonsecure_base.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::core_headers + absl::inlined_vector + absl::random_internal_pool_urbg + absl::random_internal_salted_seed_seq + absl::random_internal_seed_material + absl::span + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_pcg_engine + HDRS + "${DIR}/internal/pcg_engine.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::int128 + absl::random_internal_fastmath + absl::random_internal_iostream_state_saver + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_randen_engine + HDRS + "${DIR}/internal/randen_engine.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::endian + absl::random_internal_iostream_state_saver + absl::random_internal_randen + absl::raw_logging_internal + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_platform + HDRS + "${DIR}/internal/randen_traits.h" + "${DIR}/internal/platform.h" + SRCS + "${DIR}/internal/randen_round_keys.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_randen + SRCS + "${DIR}/internal/randen.cc" + HDRS + "${DIR}/internal/randen.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::random_internal_platform + absl::random_internal_randen_hwaes + absl::random_internal_randen_slow +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_randen_slow + SRCS + "${DIR}/internal/randen_slow.cc" + HDRS + "${DIR}/internal/randen_slow.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::random_internal_platform + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_randen_hwaes + SRCS + "${DIR}/internal/randen_detect.cc" + HDRS + "${DIR}/internal/randen_detect.h" + "${DIR}/internal/randen_hwaes.h" + COPTS + ${ABSL_DEFAULT_COPTS} + ${ABSL_RANDOM_RANDEN_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::random_internal_platform + absl::random_internal_randen_hwaes_impl + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_randen_hwaes_impl + SRCS + "${DIR}/internal/randen_hwaes.cc" + "${DIR}/internal/randen_hwaes.h" + COPTS + ${ABSL_DEFAULT_COPTS} + ${ABSL_RANDOM_RANDEN_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::random_internal_platform + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_uniform_helper + HDRS + "${DIR}/internal/uniform_helper.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::random_internal_traits + absl::type_traits +) + +set(DIR ${ABSL_ROOT_DIR}/absl/status) + +absl_cc_library( + NAME + status + HDRS + "${DIR}/status.h" + SRCS + "${DIR}/internal/status_internal.h" + "${DIR}/status.cc" + "${DIR}/status_payload_printer.h" + "${DIR}/status_payload_printer.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEFINES + "$<$:_LINUX_SOURCE_COMPAT>" + DEPS + absl::atomic_hook + absl::config + absl::cord + absl::core_headers + absl::function_ref + absl::inlined_vector + absl::memory + absl::optional + absl::raw_logging_internal + absl::span + absl::stacktrace + absl::strerror + absl::str_format + absl::strings + absl::symbolize + PUBLIC +) + +absl_cc_library( + NAME + statusor + HDRS + "${DIR}/statusor.h" + SRCS + "${DIR}/statusor.cc" + "${DIR}/internal/statusor_internal.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::raw_logging_internal + absl::status + absl::strings + absl::type_traits + absl::utility + absl::variant + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/strings) + +absl_cc_library( + NAME + string_view + HDRS + "${DIR}/string_view.h" + SRCS + "${DIR}/string_view.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::throw_delegate + PUBLIC +) + +absl_cc_library( + NAME + strings + HDRS + "${DIR}/ascii.h" + "${DIR}/charconv.h" + "${DIR}/escaping.h" + "${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" + "${DIR}/str_join.h" + "${DIR}/str_replace.h" + "${DIR}/str_split.h" + "${DIR}/strip.h" + "${DIR}/substitute.h" + SRCS + "${DIR}/ascii.cc" + "${DIR}/charconv.cc" + "${DIR}/escaping.cc" + "${DIR}/internal/charconv_bigint.cc" + "${DIR}/internal/charconv_bigint.h" + "${DIR}/internal/charconv_parse.cc" + "${DIR}/internal/charconv_parse.h" + "${DIR}/internal/damerau_levenshtein_distance.cc" + "${DIR}/internal/memutil.cc" + "${DIR}/internal/memutil.h" + "${DIR}/internal/stringify_sink.h" + "${DIR}/internal/stringify_sink.cc" + "${DIR}/internal/stl_type_traits.h" + "${DIR}/internal/str_join_internal.h" + "${DIR}/internal/str_split_internal.h" + "${DIR}/match.cc" + "${DIR}/numbers.cc" + "${DIR}/str_cat.cc" + "${DIR}/str_replace.cc" + "${DIR}/str_split.cc" + "${DIR}/substitute.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::string_view + absl::strings_internal + absl::base + absl::bits + absl::charset + absl::config + absl::core_headers + absl::endian + absl::int128 + absl::memory + absl::raw_logging_internal + absl::throw_delegate + absl::type_traits + PUBLIC +) + +absl_cc_library( + NAME + charset + HDRS + charset.h + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::core_headers + absl::string_view + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + strings_internal + HDRS + "${DIR}/internal/escaping.cc" + "${DIR}/internal/escaping.h" + "${DIR}/internal/ostringstream.h" + "${DIR}/internal/resize_uninitialized.h" + "${DIR}/internal/utf8.h" + SRCS + "${DIR}/internal/ostringstream.cc" + "${DIR}/internal/utf8.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers + absl::endian + absl::raw_logging_internal + absl::type_traits +) + +absl_cc_library( + NAME + str_format + HDRS + "${DIR}/str_format.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::str_format_internal + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + str_format_internal + HDRS + "${DIR}/internal/str_format/arg.h" + "${DIR}/internal/str_format/bind.h" + "${DIR}/internal/str_format/checker.h" + "${DIR}/internal/str_format/constexpr_parser.h" + "${DIR}/internal/str_format/extension.h" + "${DIR}/internal/str_format/float_conversion.h" + "${DIR}/internal/str_format/output.h" + "${DIR}/internal/str_format/parser.h" + SRCS + "${DIR}/internal/str_format/arg.cc" + "${DIR}/internal/str_format/bind.cc" + "${DIR}/internal/str_format/extension.cc" + "${DIR}/internal/str_format/float_conversion.cc" + "${DIR}/internal/str_format/output.cc" + "${DIR}/internal/str_format/parser.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::bits + absl::strings + absl::config + absl::core_headers + absl::inlined_vector + absl::numeric_representation + absl::type_traits + absl::utility + absl::int128 + absl::span +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cord_internal + HDRS + "${DIR}/internal/cord_data_edge.h" + "${DIR}/internal/cord_internal.h" + "${DIR}/internal/cord_rep_btree.h" + "${DIR}/internal/cord_rep_btree_navigator.h" + "${DIR}/internal/cord_rep_btree_reader.h" + "${DIR}/internal/cord_rep_crc.h" + "${DIR}/internal/cord_rep_consume.h" + "${DIR}/internal/cord_rep_flat.h" + SRCS + "${DIR}/internal/cord_internal.cc" + "${DIR}/internal/cord_rep_btree.cc" + "${DIR}/internal/cord_rep_btree_navigator.cc" + "${DIR}/internal/cord_rep_btree_reader.cc" + "${DIR}/internal/cord_rep_crc.cc" + "${DIR}/internal/cord_rep_consume.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base_internal + absl::compressed_tuple + absl::config + absl::container_memory + absl::core_headers + absl::crc_cord_state + absl::endian + absl::inlined_vector + absl::layout + absl::raw_logging_internal + absl::strings + absl::throw_delegate + absl::type_traits +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cordz_update_tracker + HDRS + "${DIR}/internal/cordz_update_tracker.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cordz_functions + HDRS + "${DIR}/internal/cordz_functions.h" + SRCS + "${DIR}/internal/cordz_functions.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers + absl::exponential_biased + absl::raw_logging_internal +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cordz_statistics + HDRS + "${DIR}/internal/cordz_statistics.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::core_headers + absl::cordz_update_tracker + absl::synchronization +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cordz_handle + HDRS + "${DIR}/internal/cordz_handle.h" + SRCS + "${DIR}/internal/cordz_handle.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config + absl::raw_logging_internal + absl::synchronization +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cordz_info + HDRS + "${DIR}/internal/cordz_info.h" + SRCS + "${DIR}/internal/cordz_info.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config + absl::cord_internal + absl::cordz_functions + absl::cordz_handle + absl::cordz_statistics + absl::cordz_update_tracker + absl::core_headers + absl::inlined_vector + absl::span + absl::raw_logging_internal + absl::stacktrace + absl::synchronization + absl::time +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cordz_sample_token + HDRS + "${DIR}/internal/cordz_sample_token.h" + SRCS + "${DIR}/internal/cordz_sample_token.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::cordz_handle + absl::cordz_info +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + cordz_update_scope + HDRS + "${DIR}/internal/cordz_update_scope.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::cord_internal + absl::cordz_info + absl::cordz_update_tracker + absl::core_headers +) + +absl_cc_library( + NAME + cord + HDRS + "${DIR}/cord.h" + "${DIR}/cord_buffer.h" + SRCS + "${DIR}/cord.cc" + "${DIR}/cord_analysis.cc" + "${DIR}/cord_analysis.h" + "${DIR}/cord_buffer.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config + absl::cord_internal + absl::cordz_functions + absl::cordz_info + absl::cordz_update_scope + absl::cordz_update_tracker + absl::core_headers + absl::crc32c + absl::crc_cord_state + absl::endian + absl::function_ref + absl::inlined_vector + absl::optional + absl::raw_logging_internal + absl::span + absl::strings + absl::type_traits + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/synchronization) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + graphcycles_internal + HDRS + "${DIR}/internal/graphcycles.h" + SRCS + "${DIR}/internal/graphcycles.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::base_internal + absl::config + absl::core_headers + absl::malloc_internal + absl::raw_logging_internal +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + kernel_timeout_internal + HDRS + "${DIR}/internal/kernel_timeout.h" + SRCS + "${DIR}/internal/kernel_timeout.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::raw_logging_internal + absl::time +) + +absl_cc_library( + NAME + synchronization + HDRS + "${DIR}/barrier.h" + "${DIR}/blocking_counter.h" + "${DIR}/internal/create_thread_identity.h" + "${DIR}/internal/futex.h" + "${DIR}/internal/futex_waiter.h" + "${DIR}/internal/per_thread_sem.h" + "${DIR}/internal/pthread_waiter.h" + "${DIR}/internal/sem_waiter.h" + "${DIR}/internal/stdcpp_waiter.h" + "${DIR}/internal/waiter.h" + "${DIR}/internal/waiter_base.h" + "${DIR}/internal/win32_waiter.h" + "${DIR}/mutex.h" + "${DIR}/notification.h" + SRCS + "${DIR}/barrier.cc" + "${DIR}/blocking_counter.cc" + "${DIR}/internal/create_thread_identity.cc" + "${DIR}/internal/futex_waiter.cc" + "${DIR}/internal/per_thread_sem.cc" + "${DIR}/internal/pthread_waiter.cc" + "${DIR}/internal/sem_waiter.cc" + "${DIR}/internal/stdcpp_waiter.cc" + "${DIR}/internal/waiter_base.cc" + "${DIR}/internal/win32_waiter.cc" + "${DIR}/notification.cc" + "${DIR}/mutex.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::graphcycles_internal + absl::kernel_timeout_internal + absl::atomic_hook + absl::base + absl::base_internal + absl::config + absl::core_headers + absl::dynamic_annotations + absl::malloc_internal + absl::raw_logging_internal + absl::stacktrace + absl::symbolize + absl::time + Threads::Threads + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/time) + +absl_cc_library( + NAME + time + HDRS + "${DIR}/civil_time.h" + "${DIR}/clock.h" + "${DIR}/time.h" + SRCS + "${DIR}/civil_time.cc" + "${DIR}/clock.cc" + "${DIR}/duration.cc" + "${DIR}/format.cc" + "${DIR}/internal/get_current_time_chrono.inc" + "${DIR}/internal/get_current_time_posix.inc" + "${DIR}/time.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base + absl::civil_time + absl::core_headers + absl::int128 + absl::raw_logging_internal + absl::strings + absl::time_zone + PUBLIC +) + +absl_cc_library( + NAME + civil_time + HDRS + "${DIR}/internal/cctz/include/cctz/civil_time.h" + "${DIR}/internal/cctz/include/cctz/civil_time_detail.h" + SRCS + "${DIR}/internal/cctz/src/civil_time_detail.cc" + COPTS + ${ABSL_DEFAULT_COPTS} +) + +absl_cc_library( + NAME + time_zone + HDRS + "${DIR}/internal/cctz/include/cctz/time_zone.h" + "${DIR}/internal/cctz/include/cctz/zone_info_source.h" + SRCS + "${DIR}/internal/cctz/src/time_zone_fixed.cc" + "${DIR}/internal/cctz/src/time_zone_fixed.h" + "${DIR}/internal/cctz/src/time_zone_format.cc" + "${DIR}/internal/cctz/src/time_zone_if.cc" + "${DIR}/internal/cctz/src/time_zone_if.h" + "${DIR}/internal/cctz/src/time_zone_impl.cc" + "${DIR}/internal/cctz/src/time_zone_impl.h" + "${DIR}/internal/cctz/src/time_zone_info.cc" + "${DIR}/internal/cctz/src/time_zone_info.h" + "${DIR}/internal/cctz/src/time_zone_libc.cc" + "${DIR}/internal/cctz/src/time_zone_libc.h" + "${DIR}/internal/cctz/src/time_zone_lookup.cc" + "${DIR}/internal/cctz/src/time_zone_posix.cc" + "${DIR}/internal/cctz/src/time_zone_posix.h" + "${DIR}/internal/cctz/src/tzfile.h" + "${DIR}/internal/cctz/src/zone_info_source.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + Threads::Threads + $<$:-Wl,-framework,CoreFoundation> +) + +set(DIR ${ABSL_ROOT_DIR}/absl/types) + +absl_cc_library( + NAME + any + HDRS + "${DIR}/any.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::bad_any_cast + absl::config + absl::core_headers + absl::fast_type_id + absl::type_traits + absl::utility + PUBLIC +) + +absl_cc_library( + NAME + bad_any_cast + HDRS + "${DIR}/bad_any_cast.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::bad_any_cast_impl + absl::config + PUBLIC +) + +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + bad_any_cast_impl + SRCS + "${DIR}/bad_any_cast.h" + "${DIR}/bad_any_cast.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::raw_logging_internal +) + +absl_cc_library( + NAME + span + HDRS + "${DIR}/span.h" + SRCS + "${DIR}/internal/span.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::algorithm + absl::core_headers + absl::throw_delegate + absl::type_traits + PUBLIC +) + +absl_cc_library( + NAME + optional + HDRS + "${DIR}/optional.h" + SRCS + "${DIR}/internal/optional.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::bad_optional_access + absl::base_internal + absl::config + absl::core_headers + absl::memory + absl::type_traits + absl::utility + PUBLIC +) + +absl_cc_library( + NAME + bad_optional_access + HDRS + "${DIR}/bad_optional_access.h" + SRCS + "${DIR}/bad_optional_access.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::raw_logging_internal + PUBLIC +) + +absl_cc_library( + NAME + bad_variant_access + HDRS + "${DIR}/bad_variant_access.h" + SRCS + "${DIR}/bad_variant_access.cc" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::raw_logging_internal + PUBLIC +) + +absl_cc_library( + NAME + variant + HDRS + "${DIR}/variant.h" + SRCS + "${DIR}/internal/variant.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::bad_variant_access + absl::base_internal + absl::config + absl::core_headers + absl::type_traits + absl::utility + PUBLIC +) + +absl_cc_library( + NAME + compare + HDRS + "${DIR}/compare.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::core_headers + absl::type_traits + PUBLIC +) + +set(DIR ${ABSL_ROOT_DIR}/absl/utility) + +absl_cc_library( + NAME + utility + HDRS + "${DIR}/utility.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::base_internal + absl::config + absl::type_traits + PUBLIC +) + +absl_cc_library( + NAME + if_constexpr + HDRS + "${DIR}/internal/if_constexpr.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) -set(ABSL_PROPAGATE_CXX_STD ON) -add_subdirectory("${ABSL_ROOT_DIR}" "${ClickHouse_BINARY_DIR}/contrib/abseil-cpp") add_library(_abseil_swiss_tables INTERFACE) - -target_link_libraries(_abseil_swiss_tables INTERFACE - absl::flat_hash_map - absl::flat_hash_set -) - -get_target_property(FLAT_HASH_MAP_INCLUDE_DIR absl::flat_hash_map INTERFACE_INCLUDE_DIRECTORIES) -target_include_directories (_abseil_swiss_tables SYSTEM BEFORE INTERFACE ${FLAT_HASH_MAP_INCLUDE_DIR}) - -get_target_property(FLAT_HASH_SET_INCLUDE_DIR absl::flat_hash_set INTERFACE_INCLUDE_DIRECTORIES) -target_include_directories (_abseil_swiss_tables SYSTEM BEFORE INTERFACE ${FLAT_HASH_SET_INCLUDE_DIR}) - +target_include_directories (_abseil_swiss_tables SYSTEM BEFORE INTERFACE ${ABSL_ROOT_DIR}) add_library(ch_contrib::abseil_swiss_tables ALIAS _abseil_swiss_tables) - -set(ABSL_FORMAT_SRC - ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/arg.cc - ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/bind.cc - ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/extension.cc - ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/float_conversion.cc - ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/output.cc - ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/parser.cc -) - -add_library(_abseil_str_format ${ABSL_FORMAT_SRC}) -target_include_directories(_abseil_str_format PUBLIC ${ABSL_ROOT_DIR}) - -add_library(ch_contrib::abseil_str_format ALIAS _abseil_str_format) diff --git a/contrib/re2-cmake/CMakeLists.txt b/contrib/re2-cmake/CMakeLists.txt index e72b5e1fca8..f773bc65a69 100644 --- a/contrib/re2-cmake/CMakeLists.txt +++ b/contrib/re2-cmake/CMakeLists.txt @@ -27,6 +27,17 @@ set(RE2_SOURCES add_library(_re2 ${RE2_SOURCES}) target_include_directories(_re2 PUBLIC "${SRC_DIR}") -target_link_libraries(_re2 ch_contrib::abseil_str_format) +target_link_libraries(_re2 PRIVATE + absl::base + absl::core_headers + absl::fixed_array + absl::flat_hash_map + absl::flat_hash_set + absl::inlined_vector + absl::strings + absl::str_format + absl::synchronization + absl::optional + absl::span) add_library(ch_contrib::re2 ALIAS _re2) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 42bfb48db70..c764a4dd8c1 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -38,7 +38,7 @@ rm -f CMakeCache.txt # To check it, find and delete them. grep -o -P '"contrib/[^"]+"' ../.gitmodules | - grep -v -P 'llvm-project|abseil-cpp|grpc|corrosion' | + grep -v -P 'llvm-project|grpc|corrosion' | xargs -I@ find ../@ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' | xargs rm From 0548eefbb784de32ec7e409ca9150482f3a171a9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 11:35:48 +0100 Subject: [PATCH 491/813] Simpler CMake --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index c764a4dd8c1..37440fe8202 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -38,7 +38,7 @@ rm -f CMakeCache.txt # To check it, find and delete them. grep -o -P '"contrib/[^"]+"' ../.gitmodules | - grep -v -P 'llvm-project|grpc|corrosion' | + grep -v -P 'llvm-project|google-protobuf|grpc|corrosion' | xargs -I@ find ../@ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' | xargs rm From 74a8f3191dc96f3ac46187b556b5d127b7ae6030 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 17 Nov 2023 11:38:56 +0100 Subject: [PATCH 492/813] Update HTTPSession.cpp --- base/poco/Net/src/HTTPSession.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index d303a4c654b..8f951b3102c 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -113,13 +113,12 @@ void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco } catch (NetException &) { - #ifndef NDEBUG + throw; +#else // mute exceptions in release // just in case when changing settings on socket is not allowed // however it should be OK for timeouts -#else - throw; #endif } } From d035e5f44fc8820a0aef96b30106ed9e78bb6408 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 16 Nov 2023 17:41:05 +0100 Subject: [PATCH 493/813] Improve diagnostics in test 02908_many_requests_to_system_replicas --- .../02908_many_requests_to_system_replicas.sh | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) 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 70dc5f4d8c4..c620fcf4bea 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 @@ -14,12 +14,13 @@ echo "Creating $NUM_TABLES tables" function init_table() { + set -e i=$1 - curl $CLICKHOUSE_URL --silent --fail --data "CREATE TABLE test_02908_r1_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r1') ORDER BY tuple()" - curl $CLICKHOUSE_URL --silent --fail --data "CREATE TABLE test_02908_r2_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r2') ORDER BY tuple()" - curl $CLICKHOUSE_URL --silent --fail --data "CREATE TABLE test_02908_r3_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r3') ORDER BY tuple()" + curl $CLICKHOUSE_URL --silent --fail --show-error --data "CREATE TABLE test_02908_r1_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r1') ORDER BY tuple()" 2>&1 + curl $CLICKHOUSE_URL --silent --fail --show-error --data "CREATE TABLE test_02908_r2_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r2') ORDER BY tuple()" 2>&1 + curl $CLICKHOUSE_URL --silent --fail --show-error --data "CREATE TABLE test_02908_r3_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r3') ORDER BY tuple()" 2>&1 - curl $CLICKHOUSE_URL --silent --fail --data "INSERT INTO test_02908_r1_$i SELECT rand64() FROM numbers(5);" + curl $CLICKHOUSE_URL --silent --fail --show-error --data "INSERT INTO test_02908_r1_$i SELECT rand64() FROM numbers(5);" 2>&1 } export init_table; @@ -36,13 +37,13 @@ echo "Making making $CONCURRENCY requests to system.replicas" for i in `seq 1 $CONCURRENCY`; do - curl $CLICKHOUSE_URL --silent --fail --data "SELECT * FROM system.replicas WHERE database=currentDatabase() FORMAT Null;" & + curl $CLICKHOUSE_URL --silent --fail --show-error --data "SELECT * FROM system.replicas WHERE database=currentDatabase() FORMAT Null;" 2>&1 || echo "query $i failed" & done echo "Query system.replicas while waiting for other concurrent requests to finish" # lost_part_count column is read from ZooKeeper -curl $CLICKHOUSE_URL --silent --fail --data "SELECT sum(lost_part_count) FROM system.replicas WHERE database=currentDatabase();"; +curl $CLICKHOUSE_URL --silent --fail --show-error --data "SELECT sum(lost_part_count) FROM system.replicas WHERE database=currentDatabase();" 2>&1; # is_leader column is filled without ZooKeeper -curl $CLICKHOUSE_URL --silent --fail --data "SELECT sum(is_leader) FROM system.replicas WHERE database=currentDatabase();"; +curl $CLICKHOUSE_URL --silent --fail --show-error --data "SELECT sum(is_leader) FROM system.replicas WHERE database=currentDatabase();" 2>&1; wait; From 6c3793acb0162454a6327cf1c4e9c097d924fc78 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 17 Nov 2023 12:03:00 +0100 Subject: [PATCH 494/813] Allow delegate disk to handle retries for createDirectories --- src/Disks/DiskEncrypted.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 8b4461a8dee..eddf08f4e91 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -60,9 +60,9 @@ public: void createDirectories(const String & path) override { - auto tx = createEncryptedTransaction(); - tx->createDirectories(path); - tx->commit(); + auto wrapped_path = wrappedPath(path); + /// Delegate disk can have retry logic for recursive directory creation. Let it handle it. + delegate->createDirectories(wrapped_path); } void clearDirectory(const String & path) override From cdfc0e07e5161dc2d381c0ad74205c0514e94850 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 17 Nov 2023 11:06:03 +0000 Subject: [PATCH 495/813] Update version_date.tsv and changelogs after v23.10.4.25-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.10.4.25-stable.md | 28 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 5 files changed, 33 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.10.4.25-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 1f4fd39bc26..63de9f6c462 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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 41be7e611a3..d26bb344fef 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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 0ff6ae2e227..53a36818121 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.10.4.25-stable.md b/docs/changelogs/v23.10.4.25-stable.md new file mode 100644 index 00000000000..2d7d2a38e04 --- /dev/null +++ b/docs/changelogs/v23.10.4.25-stable.md @@ -0,0 +1,28 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.10.4.25-stable (330fd687d41) FIXME as compared to v23.10.3.5-stable (b2ba7637a41) + +#### Build/Testing/Packaging Improvement +* Backported in [#56633](https://github.com/ClickHouse/ClickHouse/issues/56633): In [#54043](https://github.com/ClickHouse/ClickHouse/issues/54043) the setup plan started to appear in the logs. It should be only in the `runner_get_all_tests.log` only. As well, send the failed infrastructure event to CI db. [#56214](https://github.com/ClickHouse/ClickHouse/pull/56214) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#56737](https://github.com/ClickHouse/ClickHouse/issues/56737): Do not fetch changed submodules in the builder container. [#56689](https://github.com/ClickHouse/ClickHouse/pull/56689) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Select from system tables when table based on table function. [#55540](https://github.com/ClickHouse/ClickHouse/pull/55540) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix restore from backup with `flatten_nested` and `data_type_default_nullable` [#56306](https://github.com/ClickHouse/ClickHouse/pull/56306) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix segfault during Kerberos initialization [#56401](https://github.com/ClickHouse/ClickHouse/pull/56401) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix: RabbitMQ OpenSSL dynamic loading issue [#56703](https://github.com/ClickHouse/ClickHouse/pull/56703) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix crash in GCD codec in case when zeros present in data [#56704](https://github.com/ClickHouse/ClickHouse/pull/56704) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix crash in FPC codec [#56795](https://github.com/ClickHouse/ClickHouse/pull/56795) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Rewrite jobs to use callable workflow [#56385](https://github.com/ClickHouse/ClickHouse/pull/56385) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Continue rewriting workflows to reusable tests [#56501](https://github.com/ClickHouse/ClickHouse/pull/56501) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Better exception messages [#56854](https://github.com/ClickHouse/ClickHouse/pull/56854) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0f2684cd91d..ace5546aadb 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 v23.10.2.13-stable 2023-11-08 v23.10.1.1976-stable 2023-11-02 @@ -31,6 +32,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.17.13-lts 2023-11-17 v23.3.16.7-lts 2023-11-08 v23.3.15.29-lts 2023-10-31 v23.3.14.78-lts 2023-10-18 From c695405c85454e5f0266342fdbe02ab000a670cb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 17 Nov 2023 11:08:05 +0000 Subject: [PATCH 496/813] Update version_date.tsv and changelogs after v23.3.17.13-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.3.17.13-lts.md | 23 +++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 5 files changed, 28 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.3.17.13-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 1f4fd39bc26..63de9f6c462 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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 41be7e611a3..d26bb344fef 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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 0ff6ae2e227..53a36818121 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.3.17.13-lts.md b/docs/changelogs/v23.3.17.13-lts.md new file mode 100644 index 00000000000..a18ced70d46 --- /dev/null +++ b/docs/changelogs/v23.3.17.13-lts.md @@ -0,0 +1,23 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.17.13-lts (e867d59020f) FIXME as compared to v23.3.16.7-lts (fb4125cc92a) + +#### Build/Testing/Packaging Improvement +* Backported in [#56731](https://github.com/ClickHouse/ClickHouse/issues/56731): Do not fetch changed submodules in the builder container. [#56689](https://github.com/ClickHouse/ClickHouse/pull/56689) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix segfault during Kerberos initialization [#56401](https://github.com/ClickHouse/ClickHouse/pull/56401) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix crash in FPC codec [#56795](https://github.com/ClickHouse/ClickHouse/pull/56795) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Rewrite jobs to use callable workflow [#56385](https://github.com/ClickHouse/ClickHouse/pull/56385) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Continue rewriting workflows to reusable tests [#56501](https://github.com/ClickHouse/ClickHouse/pull/56501) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Better exception messages [#56854](https://github.com/ClickHouse/ClickHouse/pull/56854) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0f2684cd91d..ace5546aadb 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 v23.10.2.13-stable 2023-11-08 v23.10.1.1976-stable 2023-11-02 @@ -31,6 +32,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.17.13-lts 2023-11-17 v23.3.16.7-lts 2023-11-08 v23.3.15.29-lts 2023-10-31 v23.3.14.78-lts 2023-10-18 From 09b3f5e541935a1dd15ae1cab056ae680cec46ed Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 17 Nov 2023 11:10:10 +0000 Subject: [PATCH 497/813] Update version_date.tsv and changelogs after v23.8.7.24-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.8.7.24-lts.md | 31 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 +++ 5 files changed, 37 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.8.7.24-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 1f4fd39bc26..63de9f6c462 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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 41be7e611a3..d26bb344fef 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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 0ff6ae2e227..53a36818121 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.8.7.24-lts.md b/docs/changelogs/v23.8.7.24-lts.md new file mode 100644 index 00000000000..37862c17315 --- /dev/null +++ b/docs/changelogs/v23.8.7.24-lts.md @@ -0,0 +1,31 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.8.7.24-lts (812b95e14ba) FIXME as compared to v23.8.6.16-lts (077df679bed) + +#### Build/Testing/Packaging Improvement +* Backported in [#56733](https://github.com/ClickHouse/ClickHouse/issues/56733): Do not fetch changed submodules in the builder container. [#56689](https://github.com/ClickHouse/ClickHouse/pull/56689) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Select from system tables when table based on table function. [#55540](https://github.com/ClickHouse/ClickHouse/pull/55540) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix incomplete query result for UNION in view() function. [#56274](https://github.com/ClickHouse/ClickHouse/pull/56274) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash in case of adding a column with type Object(JSON) [#56307](https://github.com/ClickHouse/ClickHouse/pull/56307) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix segfault during Kerberos initialization [#56401](https://github.com/ClickHouse/ClickHouse/pull/56401) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix: RabbitMQ OpenSSL dynamic loading issue [#56703](https://github.com/ClickHouse/ClickHouse/pull/56703) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix crash in FPC codec [#56795](https://github.com/ClickHouse/ClickHouse/pull/56795) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NO CL CATEGORY + +* Backported in [#56601](https://github.com/ClickHouse/ClickHouse/issues/56601):. [#56598](https://github.com/ClickHouse/ClickHouse/pull/56598) ([Maksim Kita](https://github.com/kitaisreal)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Rewrite jobs to use callable workflow [#56385](https://github.com/ClickHouse/ClickHouse/pull/56385) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Continue rewriting workflows to reusable tests [#56501](https://github.com/ClickHouse/ClickHouse/pull/56501) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Better exception messages [#56854](https://github.com/ClickHouse/ClickHouse/pull/56854) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0f2684cd91d..c0b9fd22964 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 v23.10.2.13-stable 2023-11-08 v23.10.1.1976-stable 2023-11-02 @@ -5,6 +6,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.7.24-lts 2023-11-17 v23.8.6.16-lts 2023-11-08 v23.8.5.16-lts 2023-10-31 v23.8.4.69-lts 2023-10-19 @@ -31,6 +33,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.17.13-lts 2023-11-17 v23.3.16.7-lts 2023-11-08 v23.3.15.29-lts 2023-10-31 v23.3.14.78-lts 2023-10-18 From af2f06db5282fa801b82d1b9def56284f49ce77c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 17 Nov 2023 11:12:16 +0000 Subject: [PATCH 498/813] Update version_date.tsv and changelogs after v23.9.5.29-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.9.5.29-stable.md | 34 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 5 files changed, 41 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.9.5.29-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 1f4fd39bc26..63de9f6c462 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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 41be7e611a3..d26bb344fef 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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 0ff6ae2e227..53a36818121 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,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="23.10.3.5" +ARG VERSION="23.10.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.9.5.29-stable.md b/docs/changelogs/v23.9.5.29-stable.md new file mode 100644 index 00000000000..02572d0e562 --- /dev/null +++ b/docs/changelogs/v23.9.5.29-stable.md @@ -0,0 +1,34 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.9.5.29-stable (f8554c1a1ff) FIXME as compared to v23.9.4.11-stable (74c1f49dd6a) + +#### Build/Testing/Packaging Improvement +* Backported in [#56631](https://github.com/ClickHouse/ClickHouse/issues/56631): In [#54043](https://github.com/ClickHouse/ClickHouse/issues/54043) the setup plan started to appear in the logs. It should be only in the `runner_get_all_tests.log` only. As well, send the failed infrastructure event to CI db. [#56214](https://github.com/ClickHouse/ClickHouse/pull/56214) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#56735](https://github.com/ClickHouse/ClickHouse/issues/56735): Do not fetch changed submodules in the builder container. [#56689](https://github.com/ClickHouse/ClickHouse/pull/56689) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Select from system tables when table based on table function. [#55540](https://github.com/ClickHouse/ClickHouse/pull/55540) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix incomplete query result for UNION in view() function. [#56274](https://github.com/ClickHouse/ClickHouse/pull/56274) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash in case of adding a column with type Object(JSON) [#56307](https://github.com/ClickHouse/ClickHouse/pull/56307) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix segfault during Kerberos initialization [#56401](https://github.com/ClickHouse/ClickHouse/pull/56401) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix: RabbitMQ OpenSSL dynamic loading issue [#56703](https://github.com/ClickHouse/ClickHouse/pull/56703) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix crash in GCD codec in case when zeros present in data [#56704](https://github.com/ClickHouse/ClickHouse/pull/56704) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix crash in FPC codec [#56795](https://github.com/ClickHouse/ClickHouse/pull/56795) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NO CL CATEGORY + +* Backported in [#56603](https://github.com/ClickHouse/ClickHouse/issues/56603):. [#56598](https://github.com/ClickHouse/ClickHouse/pull/56598) ([Maksim Kita](https://github.com/kitaisreal)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Improve enrich image [#55793](https://github.com/ClickHouse/ClickHouse/pull/55793) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Rewrite jobs to use callable workflow [#56385](https://github.com/ClickHouse/ClickHouse/pull/56385) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Continue rewriting workflows to reusable tests [#56501](https://github.com/ClickHouse/ClickHouse/pull/56501) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Better exception messages [#56854](https://github.com/ClickHouse/ClickHouse/pull/56854) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0f2684cd91d..014ee5e9a17 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,10 +1,13 @@ +v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 v23.10.2.13-stable 2023-11-08 v23.10.1.1976-stable 2023-11-02 +v23.9.5.29-stable 2023-11-17 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.7.24-lts 2023-11-17 v23.8.6.16-lts 2023-11-08 v23.8.5.16-lts 2023-10-31 v23.8.4.69-lts 2023-10-19 @@ -31,6 +34,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.17.13-lts 2023-11-17 v23.3.16.7-lts 2023-11-08 v23.3.15.29-lts 2023-10-31 v23.3.14.78-lts 2023-10-18 From d6fdfdd45f590860a12023c25b2af735f2853796 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 12:16:54 +0100 Subject: [PATCH 499/813] Remove outdated instructions --- tests/instructions/coverity.txt | 28 ----------- tests/instructions/cppcheck.txt | 22 --------- tests/instructions/heap-profiler.txt | 14 ------ tests/instructions/kafka.txt | 45 ----------------- tests/instructions/sanitizers.md | 72 ---------------------------- tests/instructions/syntax.txt | 5 -- tests/instructions/tscancode.txt | 26 ---------- 7 files changed, 212 deletions(-) delete mode 100644 tests/instructions/coverity.txt delete mode 100644 tests/instructions/cppcheck.txt delete mode 100644 tests/instructions/heap-profiler.txt delete mode 100644 tests/instructions/kafka.txt delete mode 100644 tests/instructions/sanitizers.md delete mode 100644 tests/instructions/syntax.txt delete mode 100644 tests/instructions/tscancode.txt diff --git a/tests/instructions/coverity.txt b/tests/instructions/coverity.txt deleted file mode 100644 index f8d6d68d326..00000000000 --- a/tests/instructions/coverity.txt +++ /dev/null @@ -1,28 +0,0 @@ -# Download tool at https://scan.coverity.com/download?tab=cxx - -tar xf cov-analysis-linux64-2017.07.tar.gz -export PATH=$PATH:/home/milovidov/cov-analysis-linux64-2017.07/bin - -mkdir ClickHouse_coverity -cd ClickHouse_coverity -git clone --recursive git@github.com:yandex/ClickHouse.git . - -mkdir build -cd build - -# "Debug" is for faster build -CC=gcc-7 CXX=g++-7 cmake -D CMAKE_BUILD_TYPE=Debug -D CCACHE_FOUND=0 .. - -# Build all targets that we don't want to analyze. -cd contrib && make -j24 && cd .. - -cov-configure --comptype gcc --compiler gcc-7 --template - -cov-build --dir cov-int make -j24 - -# Build is painful slow. Some targets compile in about one hour. Total time is about 4..5 hours. - -tar czvf clickhouse.tgz cov-int - -# tarball is 1.2 GB. -# Upload result at https://scan.coverity.com/projects/yandex-clickhouse/builds/new diff --git a/tests/instructions/cppcheck.txt b/tests/instructions/cppcheck.txt deleted file mode 100644 index 1bc6d1f6c09..00000000000 --- a/tests/instructions/cppcheck.txt +++ /dev/null @@ -1,22 +0,0 @@ -# Install cppcheck - -mkdir cppcheck && cd cppcheck -git clone git@github.com:danmar/cppcheck.git . -mkdir build && cd build -CC=gcc-7 CXX=g++-7 cmake -D CMAKE_BUILD_TYPE=Release .. -make -j24 -sudo make install - -# Perform analysis - -cd ClickHouse_clean/build -cppcheck -j24 --project=compile_commands.json --enable=all 2> cppcheck-errors.txt - -# or (from directory with sources) -# cppcheck -i contrib -i build --enable=all . 2> cppcheck-errors.txt - -# Check is pretty fast. -# It gives many false positives. -# But the result is worth looking and at least few real errors found. - -grep -v -F 'contrib/' cppcheck-errors.txt diff --git a/tests/instructions/heap-profiler.txt b/tests/instructions/heap-profiler.txt deleted file mode 100644 index 3c35e9cf518..00000000000 --- a/tests/instructions/heap-profiler.txt +++ /dev/null @@ -1,14 +0,0 @@ -Build clickhouse without tcmalloc. cmake -D ENABLE_TCMALLOC=0 - -Copy clickhouse binary to your server. -scp programs/clickhouse server:~ - -ssh to your server - -Stop clickhouse: -sudo service clickhouse-server stop - -Run clickhouse with heap profiler from the terminal: -sudo -u clickhouse LD_PRELOAD=/usr/lib/libtcmalloc.so HEAPPROFILE=/var/log/clickhouse-server/heap.hprof ./clickhouse server --config /etc/clickhouse-server/config.xml - -Profiles will appear in /var/log/clickhouse-server/ diff --git a/tests/instructions/kafka.txt b/tests/instructions/kafka.txt deleted file mode 100644 index 69e87f38b24..00000000000 --- a/tests/instructions/kafka.txt +++ /dev/null @@ -1,45 +0,0 @@ -Use this config for docker-compose: - - version: '3' - - services: - - kafka: - depends_on: - - zookeeper - hostname: kafka - image: wurstmeister/kafka - environment: - KAFKA_LISTENERS: INSIDE://:9092,OUTSIDE://:9094 - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 - ports: - - "9092:9092" - - "9094:9094" - - security_opt: - - label:disable - - zookeeper: - hostname: zookeeper - image: zookeeper - - security_opt: - - label:disable - -Start containers with `docker-compose up`. - -In clickhouse-client create table like: - - CREATE TABLE kafka ( a UInt8, b String) ENGINE = Kafka('localhost:9092', 'topic', 'group1', 'CSV') SETTINGS kafka_row_delimiter = '\n'; - -Login inside Kafka container and stream some data: - - docker exec -it bash --login - vi data.csv - cat data.csv | /opt/kafka/bin/kafka-console-producer.sh --topic topic --broker-list localhost:9092 - -Read data in clickhouse: - - SELECT * FROM kafka; diff --git a/tests/instructions/sanitizers.md b/tests/instructions/sanitizers.md deleted file mode 100644 index 3c50f6cbab7..00000000000 --- a/tests/instructions/sanitizers.md +++ /dev/null @@ -1,72 +0,0 @@ -# How to use Address Sanitizer - -Note: We use Address Sanitizer to run functional tests for every commit automatically. - -``` -mkdir build_asan && cd build_asan -``` - -Note: using clang instead of gcc is strongly recommended. Make sure you have installed required packages (`clang`, `lld`). It may be required to specify non-standard `lld` binary using `LINKER_NAME` option (e.g. `-D LINKER_NAME=lld-8`). - -``` -CC=clang CXX=clang++ cmake -D SANITIZE=address .. -ninja -``` - -## Copy binary to your server - -``` -scp ./programs/clickhouse yourserver:~/clickhouse-asan -``` - -## Start ClickHouse and run tests - -``` -sudo -u clickhouse ./clickhouse-asan server --config /etc/clickhouse-server/config.xml -``` - - -# How to use Thread Sanitizer - -``` -mkdir build_tsan && cd build_tsan -``` - -``` -CC=clang CXX=clang++ cmake -D SANITIZE=thread .. -ninja -``` - -## Start ClickHouse and run tests - -``` -sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml -``` - - -# How to use Undefined Behaviour Sanitizer - -``` -mkdir build_ubsan && cd build_ubsan -``` - -Note: clang is mandatory, because gcc (in version 8) has false positives due to devirtualization and it has less amount of checks. - -``` -CC=clang CXX=clang++ cmake -D SANITIZE=undefined .. -ninja -``` - -## Start ClickHouse and run tests - -``` -sudo -u clickhouse UBSAN_OPTIONS='print_stacktrace=1' ./clickhouse-ubsan server --config /etc/clickhouse-server/config.xml -``` - - -# How to use Memory Sanitizer - -``` -CC=clang CXX=clang++ cmake -D SANITIZE=memory .. -ninja -``` diff --git a/tests/instructions/syntax.txt b/tests/instructions/syntax.txt deleted file mode 100644 index 228b0eb6045..00000000000 --- a/tests/instructions/syntax.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Relatively quick syntax check (20 minutes on 16-core server) - -mkdir build && cd build -cmake -D CMAKE_BUILD_TYPE=Debug .. -time jq --raw-output '.[] | .command' compile_commands.json | grep -P -- ' -o [^ ]+\.o' | grep -v -P -- '-c .+/contrib/' | grep -vP '\.(s|asm)$' | sed -r -e 's/ -o [^ ]+\.o/ -fsyntax-only/' | sort -R | xargs -I{} -P$(nproc) sh -c '{}' diff --git a/tests/instructions/tscancode.txt b/tests/instructions/tscancode.txt deleted file mode 100644 index 33a4eb34f35..00000000000 --- a/tests/instructions/tscancode.txt +++ /dev/null @@ -1,26 +0,0 @@ -# TScanCode is a static analyzer from Tencent -# It looks like to be based on CppCheck - -git clone git@github.com:Tencent/TscanCode.git -cd TscanCode/trunk -make -j4 - -# It looks weird that TScanCode itself compiles with multiple warnings like 'unused-but-set-variable' and 'misleading-indentation' - -# Run analysis: - -./tscancode -j4 --enable=all ~/work/ClickHouse 2> result.txt - -# It has no way to remove specific directories. We have to checkout ClickHouse to separate directory and manually remove "contrib". -# Otherwise it segfaults when analysing llvm submodule. - -# It works quite fast: - -real 0m17.174s -user 0m45.498s -sys 0m0.496s - -wc -l result.txt -61 result.txt - -# It gives almost all false positives. From 9523bd0ad8d0841c2b76a00e18e09c20f50e46d7 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 17 Nov 2023 12:17:01 +0100 Subject: [PATCH 500/813] Fix config --- tests/clickhouse-test | 2 +- tests/config/users.d/s3_cache_new.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 36ac409a4cb..1fad1583fa5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -577,7 +577,7 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), - "filesystem_cache_getorset_batch_size": lambda: random.choice([0, 3, 10, 50]), + "filesystem_cache_segments_batch_size": lambda: random.choice([0, 3, 10, 50]), "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint( 0, 1 ), diff --git a/tests/config/users.d/s3_cache_new.xml b/tests/config/users.d/s3_cache_new.xml index 638b7267960..0afa3d68fc6 100644 --- a/tests/config/users.d/s3_cache_new.xml +++ b/tests/config/users.d/s3_cache_new.xml @@ -1,7 +1,7 @@ - 10 + 10 From 216450e789f62b509bbef9f451f852e8527ac034 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 12:23:06 +0100 Subject: [PATCH 501/813] Some tasks are done --- tests/instructions/easy_tasks_sorted_ru.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/instructions/easy_tasks_sorted_ru.md b/tests/instructions/easy_tasks_sorted_ru.md index 17e9708eef5..bc95e6b1c37 100644 --- a/tests/instructions/easy_tasks_sorted_ru.md +++ b/tests/instructions/easy_tasks_sorted_ru.md @@ -201,9 +201,9 @@ https://clickhouse.com/docs/en/operations/table_engines/external_data/ ## Возможность ATTACH партиции с меньшим или большим количеством столбцов. -## Поддержка неконстантного аргумента с тайм-зоной у некоторых функций для работы с датой и временем. +## + Поддержка неконстантного аргумента с тайм-зоной у некоторых функций для работы с датой и временем. -## Возможность задавать параметры соединений для табличных функций, движков таблиц и для реплик из отдельных разделов конфигурации. +## + Возможность задавать параметры соединений для табличных функций, движков таблиц и для реплик из отдельных разделов конфигурации. ## + Настройка rollup_use_nulls. From 86119dbc3f56a78234eaa67ea40760bb0e30e7fa Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 17 Nov 2023 12:26:50 +0100 Subject: [PATCH 502/813] fix data race --- src/Storages/MergeTree/DataPartsExchange.cpp | 10 ++++++++-- .../02916_replication_protocol_wait_for_part.sql | 4 ++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index c39263a0b73..0192fb1868b 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -349,7 +349,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( return data_checksums; } -bool wait_loop(UInt32 wait_timeout_ms, std::function pred) +bool wait_loop(UInt32 wait_timeout_ms, const std::function & pred) { static const UInt32 loop_delay_ms = 5; @@ -360,6 +360,7 @@ bool wait_loop(UInt32 wait_timeout_ms, std::function pred) return true; Stopwatch timer; + sleepForMilliseconds(loop_delay_ms); while (!pred() && timer.elapsedMilliseconds() < wait_timeout_ms) { sleepForMilliseconds(loop_delay_ms); @@ -387,8 +388,13 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) /// do not expose PreActive parts for zero-copy static const UInt32 wait_timeout_ms = 1000; - bool pred_result = wait_loop(wait_timeout_ms, [&] () { return part->getState() != MergeTreeDataPartState::PreActive; }); + auto pred = [&] () + { + auto lock = data.lockParts(); + return part->getState() != MergeTreeDataPartState::PreActive; + }; + bool pred_result = wait_loop(wait_timeout_ms, pred); if (!pred_result) throw Exception( ErrorCodes::ABORTED, diff --git a/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql index 97ef33f96e8..010e29a34e8 100644 --- a/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql +++ b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql @@ -22,5 +22,5 @@ insert into tableIn values(2); system sync replica tableOut; select count() from tableOut; -drop table tableIn -drop table tableOut +drop table tableIn; +drop table tableOut; From b2dc5ada6e1702332d15fbd515c728e5d06cb7d2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 17 Nov 2023 11:31:52 +0000 Subject: [PATCH 503/813] Fix tryDecodeBase64() with invalid input --- src/Functions/FunctionBase64Conversion.h | 10 ++++----- .../00732_base64_functions.reference | 6 ++--- .../0_stateless/00732_base64_functions.sql | 22 +++++++++++++------ 3 files changed, 22 insertions(+), 16 deletions(-) diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index f52dec0eaf7..de922747ccd 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -76,12 +76,10 @@ struct TryBase64Decode static size_t perform(const std::span src, UInt8 * dst) { size_t outlen = 0; - base64_decode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); + int rc = base64_decode(reinterpret_cast(src.data()), src.size(), reinterpret_cast(dst), &outlen, 0); - // during decoding character array can be partially polluted - // if fail, revert back and clean - if (!outlen) - *dst = 0; + if (rc != 1) + outlen = 0; return outlen; } @@ -147,7 +145,7 @@ private: for (size_t row = 0; row < src_row_count; ++row) { const size_t src_length = src_offsets[row] - src_offset_prev - 1; - const auto outlen = Func::perform({src, src_length}, dst_pos); + 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. diff --git a/tests/queries/0_stateless/00732_base64_functions.reference b/tests/queries/0_stateless/00732_base64_functions.reference index f97c19427e7..8f91ffa74ab 100644 --- a/tests/queries/0_stateless/00732_base64_functions.reference +++ b/tests/queries/0_stateless/00732_base64_functions.reference @@ -21,9 +21,9 @@ fooba foobar 1 1 1 1 -fooba -~ + + + Zm9v foo foo -TEcgT3B0aW11cw== diff --git a/tests/queries/0_stateless/00732_base64_functions.sql b/tests/queries/0_stateless/00732_base64_functions.sql index 99268004003..3c60bf939fe 100644 --- a/tests/queries/0_stateless/00732_base64_functions.sql +++ b/tests/queries/0_stateless/00732_base64_functions.sql @@ -2,17 +2,23 @@ SET send_logs_level = 'fatal'; -SELECT base64Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val); +SELECT base64Encode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64Decode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryBase64Decode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64Encode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64Decode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryBase64Decode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- test with valid inputs + +SELECT base64Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val); SELECT base64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val); SELECT tryBase64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val); SELECT base64Decode(base64Encode('foo')) = 'foo', base64Encode(base64Decode('Zm9v')) == 'Zm9v'; SELECT tryBase64Decode(base64Encode('foo')) = 'foo', base64Encode(tryBase64Decode('Zm9v')) == 'Zm9v'; -SELECT base64Encode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT base64Decode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT tryBase64Decode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- test with invalid inputs SELECT base64Decode('Zm9vYmF=Zm9v'); -- { serverError INCORRECT_DATA } SELECT tryBase64Decode('Zm9vYmF=Zm9v'); @@ -20,9 +26,11 @@ SELECT tryBase64Decode('Zm9vYmF=Zm9v'); SELECT base64Decode('foo'); -- { serverError INCORRECT_DATA } SELECT tryBase64Decode('foo'); +SELECT base64Decode('aoeo054640eu='); -- { serverError INCORRECT_DATA } +SELECT tryBase64Decode('aoeo054640eu='); + +-- test FixedString arguments + select base64Encode(toFixedString('foo', 3)); select base64Decode(toFixedString('Zm9v', 4)); select tryBase64Decode(toFixedString('Zm9v', 4)); - --- This query reproduces a bug in TurboBase64 library (which we no longer use) -select distinct base64Encode(materialize('LG Optimus')) from numbers(100); From 318c7a06f92db53d1cf7389a37387193da187ff6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 12:39:00 +0100 Subject: [PATCH 504/813] Avoid dependencies with no fixed versions --- docker/packager/binary/Dockerfile | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index fb033e28959..65f79da44a0 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -6,29 +6,27 @@ FROM clickhouse/test-util:latest AS cctools ENV CC=clang-${LLVM_VERSION} ENV CXX=clang++-${LLVM_VERSION} # !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!! -# DO NOT PUT ANYTHING BEFORE THREE NEXT `RUN` DIRECTIVES +# DO NOT PUT ANYTHING BEFORE THE NEXT TWO `RUN` DIRECTIVES # THE MOST HEAVY OPERATION MUST BE THE FIRST IN THE CACHE # !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!! # libtapi is required to support .tbh format from recent MacOS SDKs RUN git clone --depth 1 https://github.com/tpoechtrager/apple-libtapi.git \ && cd apple-libtapi \ + && git checkout 15dfc2a8c9a2a89d06ff227560a69f5265b692f9 \ && INSTALLPREFIX=/cctools ./build.sh \ && ./install.sh \ && cd .. \ && rm -rf apple-libtapi # Build and install tools for cross-linking to Darwin (x86-64) -RUN git clone --depth 1 https://github.com/tpoechtrager/cctools-port.git \ - && cd cctools-port/cctools \ - && ./configure --prefix=/cctools --with-libtapi=/cctools \ - --target=x86_64-apple-darwin \ - && make install -j$(nproc) \ - && cd ../.. \ - && rm -rf cctools-port - # Build and install tools for cross-linking to Darwin (aarch64) RUN git clone --depth 1 https://github.com/tpoechtrager/cctools-port.git \ && cd cctools-port/cctools \ + && git checkout 59f5fb87a3d2c6fd2ba3df6533015cd6172001c6 \ + && ./configure --prefix=/cctools --with-libtapi=/cctools \ + --target=x86_64-apple-darwin \ + && make install -j$(nproc) \ + && make clean \ && ./configure --prefix=/cctools --with-libtapi=/cctools \ --target=aarch64-apple-darwin \ && make install -j$(nproc) \ From 45b69566012ece56f9e284f2521c4bb13fb43c7f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 17 Nov 2023 12:42:06 +0100 Subject: [PATCH 505/813] Fix race on zk_log --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index fd845016f8a..4335ea4655f 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1147,7 +1147,8 @@ void ZooKeeper::pushRequest(RequestInfo && info) { checkSessionDeadline(); info.time = clock::now(); - if (zk_log) + auto maybe_zk_log = std::atomic_load(&zk_log); + if (maybe_zk_log) { info.request->thread_id = getThreadId(); info.request->query_id = String(CurrentThread::getQueryId()); From 1d5bc13e2aa18dd30553eede612df350973425c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 12:50:51 +0100 Subject: [PATCH 506/813] Check what will happen if I remove some lines --- docker/packager/binary/Dockerfile | 7 ------- 1 file changed, 7 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index fb033e28959..2d3b83814b8 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -62,19 +62,12 @@ RUN curl https://sh.rustup.rs -sSf | bash -s -- -y && \ rustup target add aarch64-unknown-linux-musl && \ rustup target add riscv64gc-unknown-linux-gnu -# NOTE: Seems like gcc-11 is too new for ubuntu20 repository # A cross-linker for RISC-V 64 (we need it, because LLVM's LLD does not work): RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ && apt-get update \ && apt-get install --yes \ binutils-riscv64-linux-gnu \ build-essential \ - g++-11 \ - gcc-11 \ - gcc-aarch64-linux-gnu \ - libc6 \ - libc6-dev \ - libc6-dev-arm64-cross \ python3-boto3 \ yasm \ zstd \ From 2a9d05e24541d098dfe375567d5d369698a859bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 12:59:04 +0100 Subject: [PATCH 507/813] Remove more lines --- docker/packager/packager | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index e63a4912e7c..b5bcbada1da 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -236,16 +236,14 @@ def parse_env_variables( cc = compiler result.append("DEB_ARCH=amd64") - cxx = cc.replace("gcc", "g++").replace("clang", "clang++") + cxx = cc.replace("clang", "clang++") if package_type == "deb": - # NOTE: This are the env for packages/build script + # NOTE: This is the env for packages/build script result.append("MAKE_DEB=true") cmake_flags.append("-DENABLE_TESTS=0") cmake_flags.append("-DENABLE_UTILS=0") - cmake_flags.append("-DCMAKE_EXPORT_NO_PACKAGE_REGISTRY=ON") cmake_flags.append("-DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON") - cmake_flags.append("-DCMAKE_AUTOGEN_VERBOSE=ON") cmake_flags.append("-DCMAKE_INSTALL_PREFIX=/usr") cmake_flags.append("-DCMAKE_INSTALL_SYSCONFDIR=/etc") cmake_flags.append("-DCMAKE_INSTALL_LOCALSTATEDIR=/var") @@ -265,12 +263,7 @@ def parse_env_variables( elif package_type == "fuzzers": cmake_flags.append("-DENABLE_FUZZING=1") cmake_flags.append("-DENABLE_PROTOBUF=1") - cmake_flags.append("-DUSE_INTERNAL_PROTOBUF_LIBRARY=1") cmake_flags.append("-DWITH_COVERAGE=1") - cmake_flags.append("-DCMAKE_AUTOGEN_VERBOSE=ON") - # cmake_flags.append("-DCMAKE_INSTALL_PREFIX=/usr") - # cmake_flags.append("-DCMAKE_INSTALL_SYSCONFDIR=/etc") - # cmake_flags.append("-DCMAKE_INSTALL_LOCALSTATEDIR=/var") # Reduce linking and building time by avoid *install/all dependencies cmake_flags.append("-DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON") From 354fb5182bfc15c37687da9918680430e6478eb8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 13:01:33 +0100 Subject: [PATCH 508/813] Remove some code that I don't understand --- cmake/darwin/toolchain-aarch64.cmake | 6 ------ cmake/darwin/toolchain-x86_64.cmake | 6 ------ cmake/freebsd/toolchain-aarch64.cmake | 6 ------ cmake/freebsd/toolchain-ppc64le.cmake | 6 ------ cmake/freebsd/toolchain-x86_64.cmake | 6 ------ cmake/linux/toolchain-aarch64.cmake | 6 ------ cmake/linux/toolchain-ppc64le.cmake | 6 ------ cmake/linux/toolchain-riscv64.cmake | 6 ------ cmake/linux/toolchain-s390x.cmake | 6 ------ cmake/linux/toolchain-x86_64-musl.cmake | 6 ------ cmake/linux/toolchain-x86_64.cmake | 6 ------ 11 files changed, 66 deletions(-) diff --git a/cmake/darwin/toolchain-aarch64.cmake b/cmake/darwin/toolchain-aarch64.cmake index 569b02bb642..178153c1098 100644 --- a/cmake/darwin/toolchain-aarch64.cmake +++ b/cmake/darwin/toolchain-aarch64.cmake @@ -9,9 +9,3 @@ set (CMAKE_ASM_COMPILER_TARGET "aarch64-apple-darwin") set (CMAKE_OSX_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../toolchain/darwin-aarch64") set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it doesn't work in CMake - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/darwin/toolchain-x86_64.cmake b/cmake/darwin/toolchain-x86_64.cmake index c4527d2fc0d..b9cbe72a2b6 100644 --- a/cmake/darwin/toolchain-x86_64.cmake +++ b/cmake/darwin/toolchain-x86_64.cmake @@ -9,9 +9,3 @@ set (CMAKE_ASM_COMPILER_TARGET "x86_64-apple-darwin") set (CMAKE_OSX_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../toolchain/darwin-x86_64") set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it doesn't work in CMake - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/freebsd/toolchain-aarch64.cmake b/cmake/freebsd/toolchain-aarch64.cmake index 8a8da00f3be..0d7eba7c198 100644 --- a/cmake/freebsd/toolchain-aarch64.cmake +++ b/cmake/freebsd/toolchain-aarch64.cmake @@ -13,9 +13,3 @@ set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it # Will be changed later, but somehow needed to be set here. set (CMAKE_AR "ar") set (CMAKE_RANLIB "ranlib") - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/freebsd/toolchain-ppc64le.cmake b/cmake/freebsd/toolchain-ppc64le.cmake index c3f6594204d..f9878bb47be 100644 --- a/cmake/freebsd/toolchain-ppc64le.cmake +++ b/cmake/freebsd/toolchain-ppc64le.cmake @@ -13,9 +13,3 @@ set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it # Will be changed later, but somehow needed to be set here. set (CMAKE_AR "ar") set (CMAKE_RANLIB "ranlib") - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/freebsd/toolchain-x86_64.cmake b/cmake/freebsd/toolchain-x86_64.cmake index 460de6a7d39..60489da1d65 100644 --- a/cmake/freebsd/toolchain-x86_64.cmake +++ b/cmake/freebsd/toolchain-x86_64.cmake @@ -13,9 +13,3 @@ set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it # Will be changed later, but somehow needed to be set here. set (CMAKE_AR "ar") set (CMAKE_RANLIB "ranlib") - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/linux/toolchain-aarch64.cmake b/cmake/linux/toolchain-aarch64.cmake index 2dedef8859f..954f3da4331 100644 --- a/cmake/linux/toolchain-aarch64.cmake +++ b/cmake/linux/toolchain-aarch64.cmake @@ -20,9 +20,3 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/aarch64-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/linux/toolchain-ppc64le.cmake b/cmake/linux/toolchain-ppc64le.cmake index c46ea954b71..ae10cac9a55 100644 --- a/cmake/linux/toolchain-ppc64le.cmake +++ b/cmake/linux/toolchain-ppc64le.cmake @@ -20,9 +20,3 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/powerpc64le-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/linux/toolchain-riscv64.cmake b/cmake/linux/toolchain-riscv64.cmake index 7f876f88d72..7f0e30869fc 100644 --- a/cmake/linux/toolchain-riscv64.cmake +++ b/cmake/linux/toolchain-riscv64.cmake @@ -27,9 +27,3 @@ set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=bfd") # ld.lld: error: section size decrease is too large # But GNU BinUtils work. set (LINKER_NAME "riscv64-linux-gnu-ld.bfd" CACHE STRING "Linker name" FORCE) - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 945eb9affa4..b89275d5812 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -23,9 +23,3 @@ set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/linux/toolchain-x86_64-musl.cmake b/cmake/linux/toolchain-x86_64-musl.cmake index bc327e5ac25..250e52d5f58 100644 --- a/cmake/linux/toolchain-x86_64-musl.cmake +++ b/cmake/linux/toolchain-x86_64-musl.cmake @@ -21,11 +21,5 @@ set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - set (USE_MUSL 1) add_definitions(-DUSE_MUSL=1) diff --git a/cmake/linux/toolchain-x86_64.cmake b/cmake/linux/toolchain-x86_64.cmake index 55b9df79f70..8f54cbb0b48 100644 --- a/cmake/linux/toolchain-x86_64.cmake +++ b/cmake/linux/toolchain-x86_64.cmake @@ -32,9 +32,3 @@ set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") - -set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) - -set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) -set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From a3083f305ba50de44e33bcd97a089cb41e24ce22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 13:06:38 +0100 Subject: [PATCH 509/813] Remove more code that I don't understand --- cmake/freebsd/toolchain-aarch64.cmake | 4 ---- cmake/freebsd/toolchain-ppc64le.cmake | 4 ---- cmake/freebsd/toolchain-x86_64.cmake | 4 ---- cmake/linux/toolchain-aarch64.cmake | 4 ---- cmake/linux/toolchain-ppc64le.cmake | 4 ---- cmake/linux/toolchain-riscv64.cmake | 4 ---- cmake/linux/toolchain-s390x.cmake | 4 ---- cmake/linux/toolchain-x86_64-musl.cmake | 4 ---- cmake/linux/toolchain-x86_64.cmake | 4 ---- 9 files changed, 36 deletions(-) diff --git a/cmake/freebsd/toolchain-aarch64.cmake b/cmake/freebsd/toolchain-aarch64.cmake index 0d7eba7c198..53b7856ed03 100644 --- a/cmake/freebsd/toolchain-aarch64.cmake +++ b/cmake/freebsd/toolchain-aarch64.cmake @@ -9,7 +9,3 @@ set (CMAKE_ASM_COMPILER_TARGET "aarch64-unknown-freebsd12") set (CMAKE_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/freebsd-aarch64") set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it doesn't work in CMake - -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") diff --git a/cmake/freebsd/toolchain-ppc64le.cmake b/cmake/freebsd/toolchain-ppc64le.cmake index f9878bb47be..bb23f0fbafc 100644 --- a/cmake/freebsd/toolchain-ppc64le.cmake +++ b/cmake/freebsd/toolchain-ppc64le.cmake @@ -9,7 +9,3 @@ set (CMAKE_ASM_COMPILER_TARGET "powerpc64le-unknown-freebsd13") set (CMAKE_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/freebsd-ppc64le") set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it doesn't work in CMake - -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") diff --git a/cmake/freebsd/toolchain-x86_64.cmake b/cmake/freebsd/toolchain-x86_64.cmake index 60489da1d65..4635880b4a6 100644 --- a/cmake/freebsd/toolchain-x86_64.cmake +++ b/cmake/freebsd/toolchain-x86_64.cmake @@ -9,7 +9,3 @@ set (CMAKE_ASM_COMPILER_TARGET "x86_64-pc-freebsd11") set (CMAKE_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/freebsd-x86_64") set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it doesn't work in CMake - -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") diff --git a/cmake/linux/toolchain-aarch64.cmake b/cmake/linux/toolchain-aarch64.cmake index 954f3da4331..b80cc01296d 100644 --- a/cmake/linux/toolchain-aarch64.cmake +++ b/cmake/linux/toolchain-aarch64.cmake @@ -9,10 +9,6 @@ set (CMAKE_C_COMPILER_TARGET "aarch64-linux-gnu") set (CMAKE_CXX_COMPILER_TARGET "aarch64-linux-gnu") set (CMAKE_ASM_COMPILER_TARGET "aarch64-linux-gnu") -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") - set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-aarch64") set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/aarch64-linux-gnu/libc") diff --git a/cmake/linux/toolchain-ppc64le.cmake b/cmake/linux/toolchain-ppc64le.cmake index ae10cac9a55..98e8f7e8489 100644 --- a/cmake/linux/toolchain-ppc64le.cmake +++ b/cmake/linux/toolchain-ppc64le.cmake @@ -9,10 +9,6 @@ set (CMAKE_C_COMPILER_TARGET "powerpc64le-linux-gnu") set (CMAKE_CXX_COMPILER_TARGET "powerpc64le-linux-gnu") set (CMAKE_ASM_COMPILER_TARGET "powerpc64le-linux-gnu") -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") - set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-powerpc64le") set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/powerpc64le-linux-gnu/libc") diff --git a/cmake/linux/toolchain-riscv64.cmake b/cmake/linux/toolchain-riscv64.cmake index 7f0e30869fc..ae5a38f08eb 100644 --- a/cmake/linux/toolchain-riscv64.cmake +++ b/cmake/linux/toolchain-riscv64.cmake @@ -9,10 +9,6 @@ set (CMAKE_C_COMPILER_TARGET "riscv64-linux-gnu") set (CMAKE_CXX_COMPILER_TARGET "riscv64-linux-gnu") set (CMAKE_ASM_COMPILER_TARGET "riscv64-linux-gnu") -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") - set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-riscv64") set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}") diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index b89275d5812..d34329fb3bb 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -9,10 +9,6 @@ set (CMAKE_C_COMPILER_TARGET "s390x-linux-gnu") set (CMAKE_CXX_COMPILER_TARGET "s390x-linux-gnu") set (CMAKE_ASM_COMPILER_TARGET "s390x-linux-gnu") -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") - set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-s390x") set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") diff --git a/cmake/linux/toolchain-x86_64-musl.cmake b/cmake/linux/toolchain-x86_64-musl.cmake index 250e52d5f58..fa7b3eaf0d1 100644 --- a/cmake/linux/toolchain-x86_64-musl.cmake +++ b/cmake/linux/toolchain-x86_64-musl.cmake @@ -9,10 +9,6 @@ set (CMAKE_C_COMPILER_TARGET "x86_64-linux-musl") set (CMAKE_CXX_COMPILER_TARGET "x86_64-linux-musl") set (CMAKE_ASM_COMPILER_TARGET "x86_64-linux-musl") -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") - set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64-musl") set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}") diff --git a/cmake/linux/toolchain-x86_64.cmake b/cmake/linux/toolchain-x86_64.cmake index 8f54cbb0b48..e341219a7e5 100644 --- a/cmake/linux/toolchain-x86_64.cmake +++ b/cmake/linux/toolchain-x86_64.cmake @@ -19,10 +19,6 @@ set (CMAKE_C_COMPILER_TARGET "x86_64-linux-gnu") set (CMAKE_CXX_COMPILER_TARGET "x86_64-linux-gnu") set (CMAKE_ASM_COMPILER_TARGET "x86_64-linux-gnu") -# Will be changed later, but somehow needed to be set here. -set (CMAKE_AR "ar") -set (CMAKE_RANLIB "ranlib") - set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64") set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-gnu/libc") From 215cd7b9f285bbfa5d07882dfd64820e3428c402 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 15:13:16 +0300 Subject: [PATCH 510/813] Update build.sh --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 37440fe8202..f943011df9d 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -38,7 +38,7 @@ rm -f CMakeCache.txt # To check it, find and delete them. grep -o -P '"contrib/[^"]+"' ../.gitmodules | - grep -v -P 'llvm-project|google-protobuf|grpc|corrosion' | + grep -v -P 'llvm-project|google-protobuf|grpc|abseil-cpp|corrosion' | xargs -I@ find ../@ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' | xargs rm From 19dd29e8af5f79323a8f86c6a72f68ae2b45fa6e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 15:19:00 +0300 Subject: [PATCH 511/813] Update Dockerfile --- docker/packager/binary/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 65f79da44a0..d7864a11672 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -22,7 +22,7 @@ RUN git clone --depth 1 https://github.com/tpoechtrager/apple-libtapi.git \ # Build and install tools for cross-linking to Darwin (aarch64) RUN git clone --depth 1 https://github.com/tpoechtrager/cctools-port.git \ && cd cctools-port/cctools \ - && git checkout 59f5fb87a3d2c6fd2ba3df6533015cd6172001c6 \ + && git checkout 2ea20c36c10fa1ec70ada3d5aeb2c205d4aa591e \ && ./configure --prefix=/cctools --with-libtapi=/cctools \ --target=x86_64-apple-darwin \ && make install -j$(nproc) \ From ea3cd71225794724ed63213d3567a9167133eade Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 15:42:10 +0300 Subject: [PATCH 512/813] Update Dockerfile --- docker/packager/binary/Dockerfile | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index d7864a11672..8b5049c0fc7 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -10,7 +10,7 @@ ENV CXX=clang++-${LLVM_VERSION} # THE MOST HEAVY OPERATION MUST BE THE FIRST IN THE CACHE # !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!! # libtapi is required to support .tbh format from recent MacOS SDKs -RUN git clone --depth 1 https://github.com/tpoechtrager/apple-libtapi.git \ +RUN git clone https://github.com/tpoechtrager/apple-libtapi.git \ && cd apple-libtapi \ && git checkout 15dfc2a8c9a2a89d06ff227560a69f5265b692f9 \ && INSTALLPREFIX=/cctools ./build.sh \ @@ -20,9 +20,9 @@ RUN git clone --depth 1 https://github.com/tpoechtrager/apple-libtapi.git \ # Build and install tools for cross-linking to Darwin (x86-64) # Build and install tools for cross-linking to Darwin (aarch64) -RUN git clone --depth 1 https://github.com/tpoechtrager/cctools-port.git \ +RUN git clone https://github.com/tpoechtrager/cctools-port.git \ && cd cctools-port/cctools \ - && git checkout 2ea20c36c10fa1ec70ada3d5aeb2c205d4aa591e \ + && git checkout 319ef50ea51a73acfc3d691396c05005e48647da \ && ./configure --prefix=/cctools --with-libtapi=/cctools \ --target=x86_64-apple-darwin \ && make install -j$(nproc) \ From b42db2ec298d67ad0c4cb5f190fc7c1f1815e781 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 13:50:13 +0100 Subject: [PATCH 513/813] Update fasttest --- docker/test/fasttest/run.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 1b72dab5e3c..d3695ba2613 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -206,7 +206,7 @@ function build ( cd "$FASTTEST_BUILD" TIMEFORMAT=$'\nreal\t%3R\nuser\t%3U\nsys\t%3S' - ( time ninja clickhouse-bundle) |& ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt" + ( time ninja clickhouse-bundle clickhouse-stripped) |& ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt" BUILD_SECONDS_ELAPSED=$(awk '/^....-..-.. ..:..:.. real\t[0-9]/ {print $4}' < "$FASTTEST_OUTPUT/build_log.txt") echo "build_clickhouse_fasttest_binary: [ OK ] $BUILD_SECONDS_ELAPSED sec." \ | ts '%Y-%m-%d %H:%M:%S' \ @@ -215,7 +215,6 @@ function build mkdir -p "$FASTTEST_OUTPUT/binaries/" cp programs/clickhouse "$FASTTEST_OUTPUT/binaries/clickhouse" - strip programs/clickhouse -o programs/clickhouse-stripped zstd --threads=0 programs/clickhouse-stripped -o "$FASTTEST_OUTPUT/binaries/clickhouse-stripped.zst" fi ccache_status From 9eb0b74167b6b72343fba25bbc31fd42f919ef1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 15:54:32 +0300 Subject: [PATCH 514/813] Update Dockerfile --- docker/packager/binary/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 8b5049c0fc7..5e630ea5a3f 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -22,7 +22,7 @@ RUN git clone https://github.com/tpoechtrager/apple-libtapi.git \ # Build and install tools for cross-linking to Darwin (aarch64) RUN git clone https://github.com/tpoechtrager/cctools-port.git \ && cd cctools-port/cctools \ - && git checkout 319ef50ea51a73acfc3d691396c05005e48647da \ + && git checkout 2a3e1c2a6ff54a30f898b70cfb9ba1692a55fad7 \ && ./configure --prefix=/cctools --with-libtapi=/cctools \ --target=x86_64-apple-darwin \ && make install -j$(nproc) \ From 4e6f265e3e864ffdbd2cb2e7c39d3da095019352 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 17 Nov 2023 14:36:48 +0100 Subject: [PATCH 515/813] Fix tests --- tests/integration/test_storage_iceberg/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index f5b2733eeb8..d5f8d04e258 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -525,7 +525,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata_selection_" + format_version + TABLE_NAME = "test_metadata_selection_with_uuid_" + format_version spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" From 332e7f565e613ab3a519749534472d14b30845dd Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 17 Nov 2023 14:41:26 +0100 Subject: [PATCH 516/813] Fix concat tests --- tests/queries/0_stateless/00727_concat.reference | 2 +- tests/queries/0_stateless/00727_concat.sql | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00727_concat.reference b/tests/queries/0_stateless/00727_concat.reference index 7c48ba97c2b..1e102051fd0 100644 --- a/tests/queries/0_stateless/00727_concat.reference +++ b/tests/queries/0_stateless/00727_concat.reference @@ -46,7 +46,7 @@ With [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30 With 42 With 4 -- Nested -With [(\'foo\',\'qaz\'),(\'bar\',\'qux\')] +With [\'foo\',\'bar\'][\'qaz\',\'qux\'] -- NULL arguments \N \N diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index 7d901514aea..edeaf9340dd 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -50,15 +50,19 @@ SELECT concat('With ', materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [( SELECT concat('With ', materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]] :: MultiPolygon)); SELECT '-- SimpleAggregateFunction'; -CREATE OR REPLACE TABLE concat_saf_test(x SimpleAggregateFunction(max, Int32)) ENGINE=MergeTree ORDER BY tuple(); +DROP TABLE IF EXISTS concat_saf_test; +CREATE TABLE concat_saf_test(x SimpleAggregateFunction(max, Int32)) ENGINE=MergeTree ORDER BY tuple(); INSERT INTO concat_saf_test VALUES (42); INSERT INTO concat_saf_test SELECT max(number) FROM numbers(5); SELECT concat('With ', x) FROM concat_saf_test ORDER BY x DESC; +DROP TABLE concat_saf_test; SELECT '-- Nested'; -CREATE OR REPLACE TABLE concat_nested_test(kv Nested(k String, v String)) ENGINE = MergeTree ORDER BY tuple(); +DROP TABLE IF EXISTS concat_nested_test; +CREATE TABLE concat_nested_test(attrs Nested(k String, v String)) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO concat_nested_test VALUES (['foo', 'bar'], ['qaz', 'qux']); -SELECT concat('With ', kv) FROM concat_nested_test; +SELECT concat('With ', attrs.k, attrs.v) FROM concat_nested_test; +DROP TABLE concat_nested_test; SELECT '-- NULL arguments'; SELECT concat(NULL, NULL); From 571a35c84d70fc54ce6852b1fc2084ab2e5adf47 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 17 Nov 2023 14:45:19 +0100 Subject: [PATCH 517/813] Make some tests independent on macro settings --- ...licated_minimalistic_part_header_zookeeper.sh | 11 +++++++---- .../00953_zookeeper_suetin_deduplication_bug.sh | 14 +++++++------- ...6_inactive_replica_cleanup_nodes_zookeeper.sh | 11 +++++++---- ...1586_replicated_mutations_empty_partition.sql | 4 ++-- ...nt_alter_mutations_kill_many_replicas_long.sh | 7 +++++-- .../01700_system_zookeeper_path_in.reference | 2 -- .../01700_system_zookeeper_path_in.sql | 16 ++++++++-------- .../02439_merge_selecting_partitions.sql | 2 +- 8 files changed, 37 insertions(+), 30 deletions(-) diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index bab2304cec2..12d889a7137 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -7,6 +7,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +SHARD=$($CLICKHOUSE_CLIENT --query "Select getMacro('shard')") +REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") + $CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS part_header_r1; @@ -54,8 +57,8 @@ elapsed=1 until [ $elapsed -eq 5 ]; do sleep $(( elapsed++ )) - count1=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/s1/replicas/1r1/parts'") - count2=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/s1/replicas/2r1/parts'") + count1=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/$SHARD/replicas/1$REPLICA/parts'") + count2=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/$SHARD/replicas/2$REPLICA/parts'") [[ $count1 == 1 && $count2 == 1 ]] && break done @@ -64,10 +67,10 @@ $CLICKHOUSE_CLIENT -nm -q " SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/s1/replicas/1r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/$SHARD/replicas/1$REPLICA/parts'; SELECT '*** replica 2 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/s1/replicas/2r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/$SHARD/replicas/2$REPLICA/parts'; SELECT '*** Test ALTER ***'; ALTER TABLE part_header_r1 MODIFY COLUMN y String; diff --git a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh index ad0146b9d99..57a41526900 100755 --- a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh +++ b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh CLICKHOUSE_TEST_ZOOKEEPER_PREFIX="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}" - +SHARD=$($CLICKHOUSE_CLIENT --query "Select getMacro('shard')") $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS elog;" @@ -30,33 +30,33 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 3, 'h $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 3 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/s1/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/$SHARD/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/s1/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/$SHARD/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 1, 'hello')" $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 4 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/s1/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/$SHARD/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/s1/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/$SHARD/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')" $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 5 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/s1/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/$SHARD/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/s1/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/$SHARD/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')" diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index 2d761df998e..67a2a70b509 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -5,6 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +SHARD=$($CLICKHOUSE_CLIENT --query "Select getMacro('shard')") +REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") + # Check that if we have one inactive replica and a huge number of INSERTs to active replicas, # the number of nodes in ZooKeeper does not grow unbounded. @@ -32,16 +35,16 @@ for _ in {1..60}; do done -$CLICKHOUSE_CLIENT --query "SELECT numChildren < $((SCALE / 4)) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1' AND name = 'log'"; +$CLICKHOUSE_CLIENT --query "SELECT numChildren < $((SCALE / 4)) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/$SHARD' AND name = 'log'"; echo -e '\n---\n'; -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/1r1' AND name = 'is_lost'"; -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/2r1' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/$SHARD/replicas/1$REPLICA' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/$SHARD/replicas/2$REPLICA' AND name = 'is_lost'"; echo -e '\n---\n'; $CLICKHOUSE_CLIENT --query "ATTACH TABLE r2" $CLICKHOUSE_CLIENT --receive_timeout 600 --query "SYSTEM SYNC REPLICA r2" # Need to increase timeout, otherwise it timed out in debug build -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/2r1' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/$SHARD/replicas/2$REPLICA' AND name = 'is_lost'"; $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS r1; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index b5ad6c06e96..c4a3c939c26 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -16,7 +16,7 @@ INSERT INTO replicated_mutations_empty_partitions SETTINGS insert_keeper_fault_i SELECT count(distinct value) FROM replicated_mutations_empty_partitions; -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/s1/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/'||getMacro('shard')||'/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '3'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '4'; @@ -24,7 +24,7 @@ ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '5'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '9'; -- still ten records -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/s1/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/'||getMacro('shard')||'/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh index f8f3ccd6dd6..2762f918d72 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh @@ -7,6 +7,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./replication.lib . "$CURDIR"/replication.lib +SHARD=$($CLICKHOUSE_CLIENT --query "Select getMacro('shard')") +REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") + REPLICAS=5 for i in $(seq $REPLICAS); do @@ -79,9 +82,9 @@ while true; do done -metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/r11/' and name = 'metadata_version'") +metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/$SHARD/replicas/${REPLICA}1/' and name = 'metadata_version'") for i in $(seq $REPLICAS); do - replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/r1$i/' and name = 'metadata_version'") + replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/$SHARD/replicas/${REPLICA}$i/' and name = 'metadata_version'") if [ "$metadata_version" != "$replica_metadata_version" ]; then echo "Metadata version on replica $i differs from the first replica, FAIL" diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference index 664d8e84f27..b4eaf226106 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference @@ -14,5 +14,3 @@ abandonable_lock-other failed_parts last_part parallel -shared -shared diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql index cf4bc7650e7..3b321d3cea5 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql @@ -8,17 +8,17 @@ CREATE TABLE sample_table ( ENGINE ReplicatedMergeTree('/clickhouse/{database}/01700_system_zookeeper_path_in/{shard}', '{replica}') ORDER BY tuple(); -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/' || getMacro('shard') AND name like 'block%' ORDER BY name; +SELECT 'r1' FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/' || getMacro('shard') || '/replicas' AND name LIKE '%'|| getMacro('replica') ||'%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/' || getMacro('shard')) AND name LIKE 'block%' ORDER BY name; +SELECT 'r1' FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/' || getMacro('shard') || '/replicas') AND name LIKE '%' || getMacro('replica') || '%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1', - '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/' || getMacro('shard'), + '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/' || getMacro('shard') || '/replicas') AND name LIKE 'block%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1/', name) - FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/s1')) ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/' || getMacro('shard') || '/', name) + FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND name NOT LIKE 'zero_copy_%' AND path = '/clickhouse/' || currentDatabase() || '/01700_system_zookeeper_path_in/' || getMacro('shard'))) ORDER BY name; DROP TABLE IF EXISTS sample_table; diff --git a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql index 1d01fde56d6..0142afba7f2 100644 --- a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql +++ b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql @@ -21,7 +21,7 @@ select sleepEachRow(3) as higher_probability_of_reproducing_the_issue format Nul system flush logs; -- it should not list unneeded partitions where we cannot merge anything -select * from system.zookeeper_log where path like '/test/02439/s1/' || currentDatabase() || '/block_numbers/%' +select * from system.zookeeper_log where path like '/test/02439/' || getMacro('shard') || '/' || currentDatabase() || '/block_numbers/%' and op_num in ('List', 'SimpleList', 'FilteredList') and path not like '%/block_numbers/1' and path not like '%/block_numbers/123' and event_time >= now() - interval 1 minute From 01000e8b9edddbbe337d5e6287c20b9b88a64cc2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Nov 2023 14:57:02 +0100 Subject: [PATCH 518/813] Merge with master --- contrib/abseil-cpp-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index e84b4d46c4a..e6c3268c57a 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -2683,6 +2683,7 @@ absl_cc_library( "${DIR}/status.h" SRCS "${DIR}/internal/status_internal.h" + "${DIR}/internal/status_internal.cc" "${DIR}/status.cc" "${DIR}/status_payload_printer.h" "${DIR}/status_payload_printer.cc" @@ -2761,7 +2762,6 @@ 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" From 4d5becb4deff91f4e9675624175f2207ea2d9ccf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 17 Nov 2023 15:01:54 +0100 Subject: [PATCH 519/813] Adapt test_storage_s3/test.py::test_predefined_connection_configuration --- tests/integration/test_storage_s3/test.py | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 01ade1acc4d..3dd3c9e39d0 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -944,13 +944,6 @@ def test_predefined_connection_configuration(started_cluster): instance.query("GRANT SELECT ON *.* TO user") instance.query(f"drop table if exists {name}", user="user") - error = instance.query_and_get_error( - f"CREATE TABLE {name} (id UInt32) ENGINE = S3(s3_conf1, format='CSV')" - ) - assert ( - "To execute this query, it's necessary to have the grant NAMED COLLECTION ON s3_conf1" - in error - ) error = instance.query_and_get_error( f"CREATE TABLE {name} (id UInt32) ENGINE = S3(s3_conf1, format='CSV')", user="user", @@ -975,11 +968,6 @@ def test_predefined_connection_configuration(started_cluster): ) assert result == instance.query("SELECT number FROM numbers(10)") - error = instance.query_and_get_error("SELECT * FROM s3(no_collection)") - assert ( - "To execute this query, it's necessary to have the grant NAMED COLLECTION ON no_collection" - in error - ) error = instance.query_and_get_error("SELECT * FROM s3(no_collection)", user="user") assert ( "To execute this query, it's necessary to have the grant NAMED COLLECTION ON no_collection" From a530d8c80db5cd03e567f7eb6962824cfc12b9f2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 17 Nov 2023 15:25:43 +0000 Subject: [PATCH 520/813] Fix flaky test #56926 --- .../02494_query_cache_events.reference | 3 --- .../0_stateless/02494_query_cache_events.sql | 15 ++------------- 2 files changed, 2 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_events.reference b/tests/queries/0_stateless/02494_query_cache_events.reference index 9bcd2820f27..00510f3a0c6 100644 --- a/tests/queries/0_stateless/02494_query_cache_events.reference +++ b/tests/queries/0_stateless/02494_query_cache_events.reference @@ -1,7 +1,4 @@ ---- 1 -0 1 ---- 1 0 1 1 0 diff --git a/tests/queries/0_stateless/02494_query_cache_events.sql b/tests/queries/0_stateless/02494_query_cache_events.sql index 05c0acad4b8..f92e71cb50f 100644 --- a/tests/queries/0_stateless/02494_query_cache_events.sql +++ b/tests/queries/0_stateless/02494_query_cache_events.sql @@ -4,20 +4,7 @@ -- Start with empty query cache QC SYSTEM DROP QUERY CACHE; --- Run a query with QC on. The first execution is a QC miss. -SELECT '---'; SELECT 1 SETTINGS use_query_cache = true; - -SYSTEM FLUSH LOGS; -SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses'] -FROM system.query_log -WHERE type = 'QueryFinish' - AND current_database = currentDatabase() - AND query = 'SELECT 1 SETTINGS use_query_cache = true;'; - - --- Run previous query again with query cache on -SELECT '---'; SELECT 1 SETTINGS use_query_cache = true; SYSTEM FLUSH LOGS; @@ -28,4 +15,6 @@ WHERE type = 'QueryFinish' AND query = 'SELECT 1 SETTINGS use_query_cache = true;' ORDER BY event_time_microseconds; +-- (The 1st execution was a cache miss, the 2nd execution was a cache hit) + SYSTEM DROP QUERY CACHE; From dd626d51cbc3b7307b7d48279d31f02dca8ec302 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 17 Nov 2023 16:36:19 +0100 Subject: [PATCH 521/813] Fix perf tests report when there are no tests (#56881) * fix perf tests report when there are no tests * Automatic style fix * Update docker/test/performance-comparison/compare.sh --------- Co-authored-by: robot-clickhouse --- docker/test/performance-comparison/compare.sh | 10 +++++ docker/test/performance-comparison/report.py | 42 ++++++++++++++----- 2 files changed, 42 insertions(+), 10 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 7d6de732489..f10236b7135 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -189,6 +189,8 @@ function run_tests test_prefix=right/performance fi + run_only_changed_tests=0 + # Determine which tests to run. if [ -v CHPC_TEST_GREP ] then @@ -203,6 +205,7 @@ function run_tests # tests. The lists of changed files are prepared in entrypoint.sh because # it has the repository. test_files=($(sed "s/tests\/performance/${test_prefix//\//\\/}/" changed-test-definitions.txt)) + run_only_changed_tests=1 else # The default -- run all tests found in the test dir. test_files=($(ls "$test_prefix"/*.xml)) @@ -226,6 +229,13 @@ function run_tests test_files=("${test_files[@]}") fi + if [ "$run_only_changed_tests" -ne 0 ]; then + if [ ${#test_files[@]} -eq 0 ]; then + time "$script_dir/report.py" --no-tests-run > report.html + exit 0 + fi + fi + # For PRs w/o changes in test definitons, test only a subset of queries, # and run them less times. If the corresponding environment variables are # already set, keep those values. diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 7da30ba7a08..c2bc773bd54 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -19,6 +19,7 @@ parser.add_argument( choices=["main", "all-queries"], help="Which report to build", ) +parser.add_argument("--no-tests-run", action="store_true", default=False) args = parser.parse_args() tables = [] @@ -354,6 +355,36 @@ if args.report == "main": add_tested_commits() + def print_status(status, message): + print( + ( + """ + + + """.format( + status=status, message=message + ) + ) + ) + + if args.no_tests_run: + for t in tables: + print(t) + print( + "

No tests to run. Only changed tests were run, but all changed tests are from another batch.

" + ) + print( + f""" +
+ {os.getenv("CHPC_ADD_REPORT_LINKS") or ''} + + + """ + ) + # Why failure? Because otherwise we will not notice if we have a bug that leads to 0 tests being run + print_status("failure", "No tests changed, nothing to run") + exit(0) + run_error_rows = tsvRows("run-errors.tsv") error_tests += len(run_error_rows) addSimpleTable("Run Errors", ["Test", "Error"], run_error_rows) @@ -646,16 +677,7 @@ if args.report == "main": status = "failure" message = "Errors while building the report." - print( - ( - """ - - - """.format( - status=status, message=message - ) - ) - ) + print_status(status, message) elif args.report == "all-queries": print((header_template.format())) From 6d5a5f9fcd07bc87d558060495f9b071b2535abf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 17 Nov 2023 17:26:53 +0100 Subject: [PATCH 522/813] buffer result if out copacity is not enough --- src/IO/Lz4DeflatingWriteBuffer.cpp | 190 ++++++++++-------- src/IO/Lz4DeflatingWriteBuffer.h | 5 +- .../test_checking_s3_blobs_paranoid/test.py | 6 +- 3 files changed, 110 insertions(+), 91 deletions(-) diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index aab8dacef38..076b8c44f91 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -2,6 +2,59 @@ #include +namespace +{ + using namespace DB; + + class SinkToOut + { + public: + SinkToOut(WriteBuffer * out_, Memory<> & mem_, size_t guaranteed_capacity) + : sink(out_) + , tmp_out(mem_) + , cur_out(sink) + { + chassert(sink); + + if (sink->available() < guaranteed_capacity) + { + mem_.resize(guaranteed_capacity); + cur_out = &tmp_out; + } + } + + size_t getCapacity() + { + return cur_out->available(); + } + + BufferBase::Position getPosition() + { + return cur_out->position(); + } + + void advancePosition(size_t size) + { + chassert(size <= cur_out->available()); + cur_out->position() += size; + } + + ~SinkToOut() noexcept(false) + { + if (cur_out == sink) + return; + + sink->write(tmp_out.buffer().begin(), tmp_out.count()); + } + + private: + WriteBuffer * sink; + BufferWithOutsideMemory tmp_out; + WriteBuffer * cur_out; + }; +} + + namespace DB { namespace ErrorCodes @@ -13,9 +66,9 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) : WriteBufferWithOwnMemoryDecorator(std::move(out_), buf_size, existing_memory, alignment) , in_data(nullptr) - , out_data(nullptr) , in_capacity(0) - , out_capacity(0) + , tmp_memory(buf_size) + { kPrefs = { {LZ4F_max256KB, @@ -36,8 +89,8 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( if (LZ4F_isError(ret)) throw Exception( ErrorCodes::LZ4_ENCODER_FAILED, - "creation of LZ4 compression context failed. LZ4F version: {}", - LZ4F_VERSION); + "creation of LZ4 compression context failed. LZ4F version: {}, error: {}", + LZ4F_VERSION, LZ4F_getErrorName(ret)); } Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() @@ -54,107 +107,76 @@ void Lz4DeflatingWriteBuffer::nextImpl() in_data = reinterpret_cast(working_buffer.begin()); in_capacity = offset(); - out_capacity = out->buffer().end() - out->position(); - out_data = reinterpret_cast(out->position()); - - try + if (first_time) { - if (first_time) + auto sink = SinkToOut(out.get(), tmp_memory, LZ4F_HEADER_SIZE_MAX); + chassert(sink.getCapacity() >= LZ4F_HEADER_SIZE_MAX); + + /// write frame header and check for errors + size_t header_size = LZ4F_compressBegin( + ctx, sink.getPosition(), sink.getCapacity(), &kPrefs); + + if (LZ4F_isError(header_size)) + throw Exception( + ErrorCodes::LZ4_ENCODER_FAILED, + "LZ4 failed to start stream encoding. LZ4F version: {}, error: {}", + LZ4F_VERSION, LZ4F_getErrorName(header_size)); + + sink.advancePosition(header_size); + first_time = false; + } + + do + { + /// Ensure that there is enough space for compressed block of minimal size + size_t min_compressed_block_size = LZ4F_compressBound(1, &kPrefs); + + auto sink = SinkToOut(out.get(), tmp_memory, min_compressed_block_size); + chassert(sink.getCapacity() >= min_compressed_block_size); + + /// LZ4F_compressUpdate compresses whole input buffer at once so we need to shink it manually + size_t cur_buffer_size = in_capacity; + if (sink.getCapacity() >= min_compressed_block_size) /// We cannot shrink the input buffer if it's already too small. { - if (out_capacity < LZ4F_HEADER_SIZE_MAX) - { - out->next(); - out_capacity = out->buffer().end() - out->position(); - out_data = reinterpret_cast(out->position()); - } - - /// write frame header and check for errors - size_t header_size = LZ4F_compressBegin(ctx, out_data, out_capacity, &kPrefs); - - if (LZ4F_isError(header_size)) - throw Exception( - ErrorCodes::LZ4_ENCODER_FAILED, - "LZ4 failed to start stream encoding. LZ4F version: {}", - LZ4F_VERSION); - - out_capacity -= header_size; - out->position() = out->buffer().end() - out_capacity; - out_data = reinterpret_cast(out->position()); - - first_time = false; + while (sink.getCapacity() < LZ4F_compressBound(cur_buffer_size, &kPrefs)) + cur_buffer_size /= 2; } - do - { - /// Ensure that there is enough space for compressed block of minimal size - size_t min_compressed_block_size = LZ4F_compressBound(1, &kPrefs); - if (out_capacity < min_compressed_block_size) - { - out->next(); - out_capacity = out->buffer().end() - out->position(); - out_data = reinterpret_cast(out->position()); - } + size_t compressed_size = LZ4F_compressUpdate( + ctx, sink.getPosition(), sink.getCapacity(), in_data, cur_buffer_size, nullptr); - /// LZ4F_compressUpdate compresses whole input buffer at once so we need to shink it manually - size_t cur_buffer_size = in_capacity; - if (out_capacity >= min_compressed_block_size) /// We cannot shrink the input buffer if it's already too small. - { - while (out_capacity < LZ4F_compressBound(cur_buffer_size, &kPrefs)) - cur_buffer_size /= 2; - } + if (LZ4F_isError(compressed_size)) + throw Exception( + ErrorCodes::LZ4_ENCODER_FAILED, + "LZ4 failed to encode stream. LZ4F version: {}, error {}, out_capacity {}", + LZ4F_VERSION, LZ4F_getErrorName(compressed_size), sink.getCapacity()); - size_t compressed_size = LZ4F_compressUpdate(ctx, out_data, out_capacity, in_data, cur_buffer_size, nullptr); + in_capacity -= cur_buffer_size; + in_data = reinterpret_cast(working_buffer.end() - in_capacity); - if (LZ4F_isError(compressed_size)) - throw Exception( - ErrorCodes::LZ4_ENCODER_FAILED, - "LZ4 failed to encode stream. LZ4F version: {}", - LZ4F_VERSION); - - in_capacity -= cur_buffer_size; - in_data = reinterpret_cast(working_buffer.end() - in_capacity); - - out_capacity -= compressed_size; - out->position() = out->buffer().end() - out_capacity; - out_data = reinterpret_cast(out->position()); - } - while (in_capacity > 0); + sink.advancePosition(compressed_size); } - catch (...) - { - out->position() = out->buffer().begin(); - throw; - } - out->next(); - out_capacity = out->buffer().end() - out->position(); + while (in_capacity > 0); } void Lz4DeflatingWriteBuffer::finalizeBefore() { next(); - out_capacity = out->buffer().end() - out->position(); - out_data = reinterpret_cast(out->position()); - - if (out_capacity < LZ4F_compressBound(0, &kPrefs)) - { - out->next(); - out_capacity = out->buffer().end() - out->position(); - out_data = reinterpret_cast(out->position()); - } + auto suffix_size = LZ4F_compressBound(0, &kPrefs); + auto sink = SinkToOut(out.get(), tmp_memory, suffix_size); + chassert(sink.getCapacity() >= suffix_size); /// compression end - size_t end_size = LZ4F_compressEnd(ctx, out_data, out_capacity, nullptr); + size_t end_size = LZ4F_compressEnd(ctx, sink.getPosition(), sink.getCapacity(), nullptr); if (LZ4F_isError(end_size)) throw Exception( ErrorCodes::LZ4_ENCODER_FAILED, - "LZ4 failed to end stream encoding. LZ4F version: {}", - LZ4F_VERSION); + "LZ4 failed to end stream encoding. LZ4F version: {}, error {}, out_capacity {}", + LZ4F_VERSION, LZ4F_getErrorName(end_size), sink.getCapacity()); - out_capacity -= end_size; - out->position() = out->buffer().end() - out_capacity; - out_data = reinterpret_cast(out->position()); + sink.advancePosition(end_size); } void Lz4DeflatingWriteBuffer::finalizeAfter() diff --git a/src/IO/Lz4DeflatingWriteBuffer.h b/src/IO/Lz4DeflatingWriteBuffer.h index 68873b5f8ee..65f4f0c7349 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.h +++ b/src/IO/Lz4DeflatingWriteBuffer.h @@ -33,10 +33,9 @@ private: LZ4F_compressionContext_t ctx; void * in_data; - void * out_data; - size_t in_capacity; - size_t out_capacity; + + Memory<> tmp_memory; bool first_time = true; }; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index d6bcb3fb8f4..60642379366 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -97,9 +97,8 @@ def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): ] -# Add "lz4" compression method in the list after https://github.com/ClickHouse/ClickHouse/issues/50975 is fixed @pytest.mark.parametrize( - "compression", ["none", "gzip", "br", "xz", "zstd", "bz2", "deflate"] + "compression", ["none", "gzip", "br", "xz", "zstd", "bz2", "deflate", "lz4"] ) def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression): node = cluster.instances["node"] @@ -137,9 +136,8 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression assert count_s3_errors == 1 -# Add "lz4" compression method in the list after https://github.com/ClickHouse/ClickHouse/issues/50975 is fixed @pytest.mark.parametrize( - "compression", ["none", "gzip", "br", "xz", "zstd", "bz2", "deflate"] + "compression", ["none", "gzip", "br", "xz", "zstd", "bz2", "deflate", "lz4"] ) def test_upload_s3_fail_upload_part_when_multi_part_upload( cluster, broken_s3, compression From 29e64347b939638b70d4b9397b9f2d1972f75008 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 17 Nov 2023 17:37:14 +0100 Subject: [PATCH 523/813] improve exception message --- src/Storages/MergeTree/DataPartsExchange.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 0192fb1868b..43642b3ba79 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -398,7 +398,8 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) if (!pred_result) throw Exception( ErrorCodes::ABORTED, - "Part {} is in PreActive state for {} ms. Another host has to be asked.", + "Could not exchange part {} as it's in preActive state ({} ms) and it uses zero copy replication. " + "This is expected behaviour and the client will retry fetching the part automatically.", name, wait_timeout_ms); return part; From 6366819f120f5fd108acd60079921ff6aa595116 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 17 Nov 2023 16:52:20 +0000 Subject: [PATCH 524/813] Fix generating deep nested columns in CapnProto/Protobuf schemas --- src/Formats/StructureToFormatSchemaUtils.cpp | 4 ++ ...apnp_protobuf_auto_schema_nested.reference | 52 +++++++++++++++++++ ...02920_capnp_protobuf_auto_schema_nested.sh | 21 ++++++++ 3 files changed, 77 insertions(+) create mode 100644 tests/queries/0_stateless/02920_capnp_protobuf_auto_schema_nested.reference create mode 100755 tests/queries/0_stateless/02920_capnp_protobuf_auto_schema_nested.sh diff --git a/src/Formats/StructureToFormatSchemaUtils.cpp b/src/Formats/StructureToFormatSchemaUtils.cpp index 47701fa4f81..c56ff821a4a 100644 --- a/src/Formats/StructureToFormatSchemaUtils.cpp +++ b/src/Formats/StructureToFormatSchemaUtils.cpp @@ -96,6 +96,10 @@ NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types, bool nested[field_name].emplace_back(nested_name, type); } + /// Collect nested recursively. + for (auto & [field_name, elements] : nested) + elements = collectNested(elements, allow_split_by_underscore, format_name); + for (const auto & [field_name, elements]: nested) result.emplace_back(field_name, std::make_shared(elements.getTypes(), elements.getNames())); diff --git a/tests/queries/0_stateless/02920_capnp_protobuf_auto_schema_nested.reference b/tests/queries/0_stateless/02920_capnp_protobuf_auto_schema_nested.reference new file mode 100644 index 00000000000..9874bc57142 --- /dev/null +++ b/tests/queries/0_stateless/02920_capnp_protobuf_auto_schema_nested.reference @@ -0,0 +1,52 @@ + +message Message +{ + message H + { + uint32 k = 1; + } + H h = 1; + message A + { + uint32 g = 1; + message B + { + uint32 c = 1; + uint32 f = 2; + message D + { + uint32 e = 1; + } + D d = 3; + } + B b = 2; + } + A a = 2; +} +46 (45,(42,44,43)) + +struct Message +{ + struct H + { + k @0 : UInt8; + } + h @0 : H; + struct A + { + g @0 : UInt8; + struct B + { + c @0 : UInt8; + f @1 : UInt8; + struct D + { + e @0 : UInt8; + } + d @2 : D; + } + b @1 : B; + } + a @1 : A; +} +(46) (45,(42,44,(43))) diff --git a/tests/queries/0_stateless/02920_capnp_protobuf_auto_schema_nested.sh b/tests/queries/0_stateless/02920_capnp_protobuf_auto_schema_nested.sh new file mode 100755 index 00000000000..aee6b866719 --- /dev/null +++ b/tests/queries/0_stateless/02920_capnp_protobuf_auto_schema_nested.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SCHEMA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME-schema +FILE=$CLICKHOUSE_TEST_UNIQUE_NAME + +$CLICKHOUSE_LOCAL -q "select 42 as \`a.b.c\`, 43 as \`a.b.d.e\`, 44 as \`a.b.f\`, 45 as \`a.g\`, 46 as \`h.k\` format Protobuf settings output_format_schema='$SCHEMA_FILE.proto'" > $FILE.pb +tail -n +2 $SCHEMA_FILE.proto +$CLICKHOUSE_LOCAL -q "select * from file('$FILE.pb') settings format_schema='$SCHEMA_FILE:Message'" + +$CLICKHOUSE_LOCAL -q "select 42 as a_b_c, 43 as a_b_d_e, 44 as a_b_f, 45 as a_g, 46 as h_k format CapnProto settings output_format_schema='$SCHEMA_FILE.capnp'" > $FILE.capnp +tail -n +2 $SCHEMA_FILE.capnp +$CLICKHOUSE_LOCAL -q "select * from file('$FILE.capnp') settings format_schema='$SCHEMA_FILE:Message'" + +rm $SCHEMA_FILE* +rm $FILE.* + From fcce5409f47c4db15ffedded1789e34f9e528010 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Nov 2023 15:17:19 +0000 Subject: [PATCH 525/813] Simplify --- cmake/cpu_features.cmake | 50 ++++++++++++++-------------- contrib/aws-cmake/AwsSIMD.cmake | 2 +- contrib/fastops-cmake/CMakeLists.txt | 4 +-- contrib/rocksdb-cmake/CMakeLists.txt | 6 ++-- 4 files changed, 29 insertions(+), 33 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 765e36403ad..484c139f1a7 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -134,60 +134,60 @@ elseif (ARCH_AMD64) # ClickHouse can be cross-compiled (e.g. on an ARM host for x86) but it is also possible to build ClickHouse on x86 w/o AVX for x86 w/ # AVX. We only assume that the compiler can emit certain SIMD instructions, we don't care if the host system is able to run the binary. - SET (HAVE_SSSE3 1) - SET (HAVE_SSE41 1) - SET (HAVE_SSE42 1) - SET (HAVE_PCLMULQDQ 1) - SET (HAVE_POPCNT 1) - SET (HAVE_AVX 1) - SET (HAVE_AVX2 1) - SET (HAVE_AVX512 1) - SET (HAVE_AVX512_VBMI 1) - SET (HAVE_BMI 1) - SET (HAVE_BMI2 1) - - if (HAVE_SSSE3 AND ENABLE_SSSE3) + if (ENABLE_SSSE3) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mssse3") endif () - if (HAVE_SSE41 AND ENABLE_SSE41) + + if (ENABLE_SSE41) set (COMPILER_FLAGS "${COMPILER_FLAGS} -msse4.1") endif () - if (HAVE_SSE42 AND ENABLE_SSE42) + + if (ENABLE_SSE42) set (COMPILER_FLAGS "${COMPILER_FLAGS} -msse4.2") endif () - if (HAVE_PCLMULQDQ AND ENABLE_PCLMULQDQ) + + if (ENABLE_PCLMULQDQ) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mpclmul") endif () - if (HAVE_POPCNT AND ENABLE_POPCNT) + + if (ENABLE_POPCNT) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mpopcnt") endif () - if (HAVE_AVX AND ENABLE_AVX) + + if (ENABLE_AVX) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx") endif () - if (HAVE_AVX2 AND ENABLE_AVX2) + + if (ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx2") endif () - if (HAVE_AVX512 AND ENABLE_AVX512) + + if (ENABLE_AVX512) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx512f -mavx512bw -mavx512vl") endif () - if (HAVE_AVX512 AND ENABLE_AVX512 AND HAVE_AVX512_VBMI AND ENABLE_AVX512_VBMI) + + if (ENABLE_AVX512 AND ENABLE_AVX512_VBMI) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx512vbmi") endif () - if (HAVE_BMI AND ENABLE_BMI) + + if (ENABLE_BMI) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mbmi") endif () - if (HAVE_BMI2 AND HAVE_AVX2 AND ENABLE_AVX2 AND ENABLE_BMI2) + + if (ENABLE_AVX2 AND ENABLE_BMI2) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mbmi2") endif () + if (ENABLE_AVX512_FOR_SPEC_OP) set (X86_INTRINSICS_FLAGS "") - if (HAVE_BMI) + if (1) set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") endif () - if (HAVE_AVX512) + if (1) set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx512f -mavx512bw -mavx512vl -mprefer-vector-width=256") endif () endif () + else () # RISC-V + exotic platforms endif () diff --git a/contrib/aws-cmake/AwsSIMD.cmake b/contrib/aws-cmake/AwsSIMD.cmake index a2f50f27d4e..24f7628e86f 100644 --- a/contrib/aws-cmake/AwsSIMD.cmake +++ b/contrib/aws-cmake/AwsSIMD.cmake @@ -2,7 +2,7 @@ # SPDX-License-Identifier: Apache-2.0. if (USE_CPU_EXTENSIONS) - if (HAVE_AVX2) + if (ENABLE_AVX2) set (AVX2_CFLAGS "-mavx -mavx2") set (HAVE_AVX2_INTRINSICS 1) set (HAVE_MM256_EXTRACT_EPI64 1) diff --git a/contrib/fastops-cmake/CMakeLists.txt b/contrib/fastops-cmake/CMakeLists.txt index e9aa4803583..1b09b736b2a 100644 --- a/contrib/fastops-cmake/CMakeLists.txt +++ b/contrib/fastops-cmake/CMakeLists.txt @@ -13,12 +13,10 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/fastops") set(SRCS "") -if(HAVE_AVX) +if(ARCH_AMD64) set (SRCS ${SRCS} "${LIBRARY_DIR}/fastops/avx/ops_avx.cpp") set_source_files_properties("${LIBRARY_DIR}/fastops/avx/ops_avx.cpp" PROPERTIES COMPILE_FLAGS "-mavx -DNO_AVX2") -endif() -if(HAVE_AVX2) set (SRCS ${SRCS} "${LIBRARY_DIR}/fastops/avx2/ops_avx2.cpp") set_source_files_properties("${LIBRARY_DIR}/fastops/avx2/ops_avx2.cpp" PROPERTIES COMPILE_FLAGS "-mavx2 -mfma") endif() diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 2b6c48f0b38..7d7666dff87 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -93,11 +93,9 @@ if(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64|arm64|ARM64") endif(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64|arm64|ARM64") -if(HAVE_SSE42) +if(ENABLE_AVX2 AND ENABLE_PCLMULQDQ) add_definitions(-DHAVE_SSE42) add_definitions(-DHAVE_PCLMUL) -elseif(FORCE_SSE42) - message(FATAL_ERROR "FORCE_SSE42=ON but unable to compile with SSE4.2 enabled") endif() set (HAVE_THREAD_LOCAL 1) @@ -429,7 +427,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc rocksdb_build_version.cc) -if(HAVE_SSE42) +if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ) set_source_files_properties( "${ROCKSDB_SOURCE_DIR}/util/crc32c.cc" PROPERTIES COMPILE_FLAGS "-msse4.2 -mpclmul") From c51429b1ef006b2bced059624312af5c1dc1f07f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 16 Nov 2023 15:24:47 +0000 Subject: [PATCH 526/813] Simplify more --- cmake/cpu_features.cmake | 30 +++++++++++------------------- 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 484c139f1a7..cfa9c314bc0 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -150,6 +150,10 @@ elseif (ARCH_AMD64) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mpclmul") endif () + if (ENABLE_BMI) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mbmi") + endif () + if (ENABLE_POPCNT) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mpopcnt") endif () @@ -160,32 +164,20 @@ elseif (ARCH_AMD64) if (ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx2") + if (ENABLE_BMI2) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mbmi2") + endif () endif () if (ENABLE_AVX512) set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx512f -mavx512bw -mavx512vl") - endif () - - if (ENABLE_AVX512 AND ENABLE_AVX512_VBMI) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx512vbmi") - endif () - - if (ENABLE_BMI) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -mbmi") - endif () - - if (ENABLE_AVX2 AND ENABLE_BMI2) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -mbmi2") + if (ENABLE_AVX512_VBMI) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -mavx512vbmi") + endif () endif () if (ENABLE_AVX512_FOR_SPEC_OP) - set (X86_INTRINSICS_FLAGS "") - if (1) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") - endif () - if (1) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx512f -mavx512bw -mavx512vl -mprefer-vector-width=256") - endif () + set (X86_INTRINSICS_FLAGS "-mbmi -mavx512f -mavx512bw -mavx512vl -mprefer-vector-width=256") endif () else () From a18b7155919d7a7ffc10814043dde3e2c913f620 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 18 Nov 2023 15:44:45 +0000 Subject: [PATCH 527/813] Fix a bug --- src/Storages/AlterCommands.cpp | 17 +++++++++++++++-- .../02916_addcolumn_nested.reference | 1 + .../0_stateless/02916_addcolumn_nested.sql | 5 +++++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7eeaa2d4594..f5293c52bb0 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -33,6 +33,8 @@ #include #include +#include + namespace DB { @@ -403,10 +405,21 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const auto transformed_columns = temporary_metadata.columns.getAll(); - for (auto it = transformed_columns.rbegin(); it != transformed_columns.rend(); it++) + auto add_column = [&](const String & name) { - const auto & transformed_column = temporary_metadata.columns.get(it->name); + const auto & transformed_column = temporary_metadata.columns.get(name); metadata.columns.add(transformed_column, after_column, first); + }; + + if (!after_column.empty() || first) + { + for (const auto & col: transformed_columns | std::views::reverse) + add_column(col.name); + } + else + { + for (const auto & col: transformed_columns) + add_column(col.name); } } else diff --git a/tests/queries/0_stateless/02916_addcolumn_nested.reference b/tests/queries/0_stateless/02916_addcolumn_nested.reference index 869d4336c62..7d79cd8731f 100644 --- a/tests/queries/0_stateless/02916_addcolumn_nested.reference +++ b/tests/queries/0_stateless/02916_addcolumn_nested.reference @@ -1,3 +1,4 @@ CREATE TABLE default.nested_table\n(\n `id` UInt64,\n `first` Nested(a Int8, b String)\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 CREATE TABLE default.nested_table\n(\n `id` UInt64,\n `second.c` Array(Int8),\n `second.d` Array(String),\n `first` Nested(a Int8, b String)\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 CREATE TABLE default.nested_table\n(\n `third` Nested(e Int8, f String),\n `id` UInt64,\n `second.c` Array(Int8),\n `second.d` Array(String),\n `first` Nested(a Int8, b String)\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.nested_table\n(\n `third` Nested(e Int8, f String),\n `id` UInt64,\n `second.c` Array(Int8),\n `second.d` Array(String),\n `first` Nested(a Int8, b String),\n `fourth.g` Array(Int8),\n `fourth.h` Array(String)\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02916_addcolumn_nested.sql b/tests/queries/0_stateless/02916_addcolumn_nested.sql index b23854824b5..1e64fca6a15 100644 --- a/tests/queries/0_stateless/02916_addcolumn_nested.sql +++ b/tests/queries/0_stateless/02916_addcolumn_nested.sql @@ -14,4 +14,9 @@ SET flatten_nested = 0; ALTER TABLE nested_table ADD COLUMN third Nested(e Int8, f String) FIRST; SHOW CREATE nested_table; +SET flatten_nested = 1; + +ALTER TABLE nested_table ADD COLUMN fourth Nested(g Int8, h String); +SHOW CREATE nested_table; + DROP TABLE nested_table; From 773715a562cdcba9ac9dc0730a773201634c7326 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 18 Nov 2023 17:30:49 +0100 Subject: [PATCH 528/813] finalize tmp_out --- src/IO/BufferWithOwnMemory.h | 6 ++++++ src/IO/Lz4DeflatingWriteBuffer.cpp | 2 ++ 2 files changed, 8 insertions(+) diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index 39c83e9167c..5c9a69893df 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -191,6 +191,12 @@ private: memory.resize(2 * prev_size + 1); Base::set(memory.data() + prev_size, memory.size() - prev_size, 0); } + + void finalizeImpl() final + { + /// there is no need to allocate twice more memory at finalize() + /// So make that call no op, do not call here nextImpl() + } }; } diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 076b8c44f91..e952e6400ec 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -41,6 +41,8 @@ namespace ~SinkToOut() noexcept(false) { + tmp_out.finalize(); + if (cur_out == sink) return; From d1c56e3dab845baefe0514f86c1b49be02538a6c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 18 Nov 2023 17:40:15 +0000 Subject: [PATCH 529/813] Send fatal logs by default in clickhouse-local --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 36020d22cc0..f3b551b08d2 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -612,7 +612,7 @@ void LocalServer::processConfig() else if (logging || is_interactive) { config().setString("logger", "logger"); - auto log_level_default = is_interactive && !logging ? "none" : level; + auto log_level_default = is_interactive && !logging ? "fatal" : level; config().setString("logger.level", config().getString("log-level", config().getString("send_logs_level", log_level_default))); buildLoggers(config(), logger(), "clickhouse-local"); logging_initialized = true; From 6eedd1649db62fcfbeab6eda8039e6217e155c49 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 18 Nov 2023 17:55:38 +0000 Subject: [PATCH 530/813] Resubmit: Better except for SSL authentication --- src/Server/TCPHandler.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1da9806b4f5..1c2c16496f0 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1431,8 +1431,11 @@ void TCPHandler::receiveHello() getClientAddress(client_info)); return; } - catch (...) + catch (const Exception & e) { + if (e.code() != DB::ErrorCodes::AUTHENTICATION_FAILED) + throw; + tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication"); } } From d56cbda1850974da1f4cf7d66ef52d002d3b7244 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Nov 2023 19:07:59 +0100 Subject: [PATCH 531/813] Add metrics for the number of queued jobs, which is useful for the IO thread pool --- programs/benchmark/Benchmark.cpp | 3 +- programs/copier/ClusterCopier.cpp | 3 +- src/Backups/BackupsWorker.cpp | 7 ++- src/Common/AsyncLoader.cpp | 1 + src/Common/AsyncLoader.h | 5 ++- src/Common/CurrentMetrics.cpp | 43 +++++++++++++++++++ src/Common/ThreadPool.cpp | 40 ++++++++--------- src/Common/ThreadPool.h | 14 ++++-- src/Common/examples/parallel_aggregation.cpp | 3 +- src/Common/examples/parallel_aggregation2.cpp | 3 +- .../examples/thread_creation_latency.cpp | 7 +-- src/Common/tests/gtest_async_loader.cpp | 2 + .../gtest_thread_pool_concurrent_wait.cpp | 5 ++- .../tests/gtest_thread_pool_global_full.cpp | 7 +-- src/Common/tests/gtest_thread_pool_limit.cpp | 3 +- src/Common/tests/gtest_thread_pool_loop.cpp | 3 +- .../gtest_thread_pool_schedule_exception.cpp | 5 ++- src/Coordination/Standalone/Context.cpp | 1 + src/Databases/DatabaseOnDisk.cpp | 3 +- src/Databases/DatabaseOrdinary.cpp | 3 +- src/Databases/TablesLoader.cpp | 3 +- .../CacheDictionaryUpdateQueue.cpp | 3 +- src/Dictionaries/HashedDictionary.cpp | 5 ++- src/Disks/IDisk.h | 5 ++- src/Disks/IO/ThreadPoolReader.cpp | 3 +- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 2 + .../AzureBlobStorage/AzureObjectStorage.cpp | 4 +- ...jectStorageRemoteMetadataRestoreHelper.cpp | 5 ++- .../ObjectStorageIteratorAsync.cpp | 1 + .../ObjectStorageIteratorAsync.h | 4 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 2 + src/IO/SharedThreadPools.cpp | 20 ++++++--- src/IO/SharedThreadPools.h | 5 ++- src/Interpreters/Aggregator.cpp | 7 +-- src/Interpreters/AsynchronousInsertQueue.cpp | 3 +- src/Interpreters/Context.cpp | 9 ++-- src/Interpreters/DDLWorker.cpp | 7 +-- src/Interpreters/DatabaseCatalog.cpp | 3 +- src/Interpreters/InterpreterSystemQuery.cpp | 3 +- src/Interpreters/loadMetadata.cpp | 5 ++- src/Interpreters/threadPoolCallbackRunner.h | 6 +-- src/Processors/Executors/PipelineExecutor.cpp | 3 +- .../Formats/Impl/DWARFBlockInputFormat.cpp | 3 +- .../Impl/ParallelFormattingOutputFormat.h | 3 +- .../Formats/Impl/ParallelParsingInputFormat.h | 3 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 3 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 5 ++- .../Transforms/AggregatingTransform.h | 2 + src/Storages/Distributed/DistributedSink.cpp | 4 +- src/Storages/Hive/StorageHive.cpp | 3 +- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 3 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 + src/Storages/StorageAzureBlob.cpp | 3 +- src/Storages/StorageDistributed.cpp | 3 +- src/Storages/StorageS3.cpp | 7 +-- src/Storages/System/StorageSystemReplicas.cpp | 3 +- utils/keeper-bench/Runner.cpp | 5 ++- 57 files changed, 222 insertions(+), 99 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index ed3d4a1ea69..d6b8b38d84d 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -46,6 +46,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } namespace DB @@ -107,7 +108,7 @@ public: settings(settings_), shared_context(Context::createShared()), global_context(Context::createGlobal(shared_context.get())), - pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, concurrency) + pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, concurrency) { const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable; size_t connections_cnt = std::max(ports_.size(), hosts_.size()); diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index b2b4970d04f..7d58f35f62f 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -25,6 +25,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } namespace DB @@ -200,7 +201,7 @@ void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, { /// Fetch partitions list from a shard { - ThreadPool thread_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); + ThreadPool thread_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); for (const TaskShardPtr & task_shard : task_table.all_shards) thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index f6020deabec..4e24269fb25 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -31,8 +31,10 @@ namespace CurrentMetrics { extern const Metric BackupsThreads; extern const Metric BackupsThreadsActive; + extern const Metric BackupsThreadsScheduled; extern const Metric RestoreThreads; extern const Metric RestoreThreadsActive; + extern const Metric RestoreThreadsScheduled; } namespace DB @@ -264,6 +266,7 @@ public: CurrentMetrics::Metric metric_threads; CurrentMetrics::Metric metric_active_threads; + CurrentMetrics::Metric metric_scheduled_threads; size_t max_threads = 0; /// What to do with a new job if a corresponding thread pool is already running `max_threads` jobs: @@ -279,6 +282,7 @@ public: { metric_threads = CurrentMetrics::BackupsThreads; metric_active_threads = CurrentMetrics::BackupsThreadsActive; + metric_active_threads = CurrentMetrics::BackupsThreadsScheduled; max_threads = num_backup_threads; /// We don't use thread pool queues for thread pools with a lot of tasks otherwise that queue could be memory-wasting. use_queue = (thread_pool_id != ThreadPoolId::BACKUP_COPY_FILES); @@ -291,6 +295,7 @@ public: { metric_threads = CurrentMetrics::RestoreThreads; metric_active_threads = CurrentMetrics::RestoreThreadsActive; + metric_active_threads = CurrentMetrics::RestoreThreadsScheduled; max_threads = num_restore_threads; use_queue = (thread_pool_id != ThreadPoolId::RESTORE_TABLES_DATA); break; @@ -301,7 +306,7 @@ public: chassert(max_threads != 0); size_t max_free_threads = 0; size_t queue_size = use_queue ? 0 : max_threads; - auto thread_pool = std::make_unique(metric_threads, metric_active_threads, max_threads, max_free_threads, queue_size); + auto thread_pool = std::make_unique(metric_threads, metric_active_threads, metric_scheduled_threads, max_threads, max_free_threads, queue_size); auto * thread_pool_ptr = thread_pool.get(); thread_pools.emplace(thread_pool_id, std::move(thread_pool)); return *thread_pool_ptr; diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 2e96d3eab7d..cff34ac036d 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -179,6 +179,7 @@ AsyncLoader::AsyncLoader(std::vector pool_initializers, bool lo .thread_pool = std::make_unique( init.metric_threads, init.metric_active_threads, + init.metric_scheduled_threads, init.max_threads, /* max_free_threads = */ 0, init.max_threads), diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 77905319f00..0496549001e 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -271,8 +271,8 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs & // Basic usage example: // // Start async_loader with two thread pools (0=fg, 1=bg): // AsyncLoader async_loader({ -// {"FgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 2, .priority{0}} -// {"BgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 1, .priority{1}} +// {"FgPool", CurrentMetrics::AsyncLoaderThreads, ..., .max_threads = 2, .priority{0}} +// {"BgPool", CurrentMetrics::AsyncLoaderThreads, ..., .max_threads = 1, .priority{1}} // }); // // // Create and schedule a task consisting of three jobs. Job1 has no dependencies and is run first. @@ -368,6 +368,7 @@ public: String name; Metric metric_threads; Metric metric_active_threads; + Metric metric_scheduled_threads; size_t max_threads; Priority priority; }; diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index c929f4d86e2..5a4b6e80f75 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -71,92 +71,135 @@ M(RWLockActiveWriters, "Number of threads holding write lock in a table RWLock.") \ M(GlobalThread, "Number of threads in global thread pool.") \ M(GlobalThreadActive, "Number of threads in global thread pool running a task.") \ + M(GlobalThreadScheduled, "Number of queued or active jobs in global thread pool.") \ M(LocalThread, "Number of threads in local thread pools. The threads in local thread pools are taken from the global thread pool.") \ M(LocalThreadActive, "Number of threads in local thread pools running a task.") \ + M(LocalThreadScheduled, "Number of queued or active jobs in local thread pools.") \ M(MergeTreeDataSelectExecutorThreads, "Number of threads in the MergeTreeDataSelectExecutor thread pool.") \ M(MergeTreeDataSelectExecutorThreadsActive, "Number of threads in the MergeTreeDataSelectExecutor thread pool running a task.") \ + M(MergeTreeDataSelectExecutorThreadsScheduled, "Number of queued or active jobs in the MergeTreeDataSelectExecutor thread pool.") \ M(BackupsThreads, "Number of threads in the thread pool for BACKUP.") \ M(BackupsThreadsActive, "Number of threads in thread pool for BACKUP running a task.") \ + M(BackupsThreadsScheduled, "Number of queued or active jobs for BACKUP.") \ M(RestoreThreads, "Number of threads in the thread pool for RESTORE.") \ M(RestoreThreadsActive, "Number of threads in the thread pool for RESTORE running a task.") \ + M(RestoreThreadsScheduled, "Number of queued or active jobs for RESTORE.") \ M(MarksLoaderThreads, "Number of threads in thread pool for loading marks.") \ M(MarksLoaderThreadsActive, "Number of threads in the thread pool for loading marks running a task.") \ + M(MarksLoaderThreadsScheduled, "Number of queued or active jobs in the thread pool for loading marks.") \ M(IOPrefetchThreads, "Number of threads in the IO prefertch thread pool.") \ M(IOPrefetchThreadsActive, "Number of threads in the IO prefetch thread pool running a task.") \ + M(IOPrefetchThreadsScheduled, "Number of queued or active jobs in the IO prefetch thread pool.") \ M(IOWriterThreads, "Number of threads in the IO writer thread pool.") \ M(IOWriterThreadsActive, "Number of threads in the IO writer thread pool running a task.") \ + M(IOWriterThreadsScheduled, "Number of queued or active jobs in the IO writer thread pool.") \ M(IOThreads, "Number of threads in the IO thread pool.") \ M(IOThreadsActive, "Number of threads in the IO thread pool running a task.") \ + M(IOThreadsScheduled, "Number of queued or active jobs in the IO thread pool.") \ M(ThreadPoolRemoteFSReaderThreads, "Number of threads in the thread pool for remote_filesystem_read_method=threadpool.") \ M(ThreadPoolRemoteFSReaderThreadsActive, "Number of threads in the thread pool for remote_filesystem_read_method=threadpool running a task.") \ + M(ThreadPoolRemoteFSReaderThreadsScheduled, "Number of queued or active jobs in the thread pool for remote_filesystem_read_method=threadpool.") \ M(ThreadPoolFSReaderThreads, "Number of threads in the thread pool for local_filesystem_read_method=threadpool.") \ M(ThreadPoolFSReaderThreadsActive, "Number of threads in the thread pool for local_filesystem_read_method=threadpool running a task.") \ + M(ThreadPoolFSReaderThreadsScheduled, "Number of queued or active jobs in the thread pool for local_filesystem_read_method=threadpool.") \ M(BackupsIOThreads, "Number of threads in the BackupsIO thread pool.") \ M(BackupsIOThreadsActive, "Number of threads in the BackupsIO thread pool running a task.") \ + M(BackupsIOThreadsScheduled, "Number of queued or active jobs in the BackupsIO thread pool.") \ M(DiskObjectStorageAsyncThreads, "Obsolete metric, shows nothing.") \ M(DiskObjectStorageAsyncThreadsActive, "Obsolete metric, shows nothing.") \ M(StorageHiveThreads, "Number of threads in the StorageHive thread pool.") \ M(StorageHiveThreadsActive, "Number of threads in the StorageHive thread pool running a task.") \ + M(StorageHiveThreadsScheduled, "Number of queued or active jobs in the StorageHive thread pool.") \ M(TablesLoaderThreads, "Number of threads in the tables loader thread pool.") \ M(TablesLoaderThreadsActive, "Number of threads in the tables loader thread pool running a task.") \ + M(TablesLoaderThreadsScheduled, "Number of queued or active jobs in the tables loader thread pool.") \ M(DatabaseOrdinaryThreads, "Number of threads in the Ordinary database thread pool.") \ M(DatabaseOrdinaryThreadsActive, "Number of threads in the Ordinary database thread pool running a task.") \ + M(DatabaseOrdinaryThreadsScheduled, "Number of queued or active jobs in the Ordinary database thread pool.") \ M(DatabaseOnDiskThreads, "Number of threads in the DatabaseOnDisk thread pool.") \ M(DatabaseOnDiskThreadsActive, "Number of threads in the DatabaseOnDisk thread pool running a task.") \ + M(DatabaseOnDiskThreadsScheduled, "Number of queued or active jobs in the DatabaseOnDisk thread pool.") \ M(DatabaseCatalogThreads, "Number of threads in the DatabaseCatalog thread pool.") \ M(DatabaseCatalogThreadsActive, "Number of threads in the DatabaseCatalog thread pool running a task.") \ + M(DatabaseCatalogThreadsScheduled, "Number of queued or active jobs in the DatabaseCatalog thread pool.") \ M(DestroyAggregatesThreads, "Number of threads in the thread pool for destroy aggregate states.") \ M(DestroyAggregatesThreadsActive, "Number of threads in the thread pool for destroy aggregate states running a task.") \ + M(DestroyAggregatesThreadsScheduled, "Number of queued or active jobs in the thread pool for destroy aggregate states.") \ M(HashedDictionaryThreads, "Number of threads in the HashedDictionary thread pool.") \ M(HashedDictionaryThreadsActive, "Number of threads in the HashedDictionary thread pool running a task.") \ + M(HashedDictionaryThreadsScheduled, "Number of queued or active jobs in the HashedDictionary thread pool.") \ M(CacheDictionaryThreads, "Number of threads in the CacheDictionary thread pool.") \ M(CacheDictionaryThreadsActive, "Number of threads in the CacheDictionary thread pool running a task.") \ + M(CacheDictionaryThreadsScheduled, "Number of queued or active jobs in the CacheDictionary thread pool.") \ M(ParallelFormattingOutputFormatThreads, "Number of threads in the ParallelFormattingOutputFormatThreads thread pool.") \ M(ParallelFormattingOutputFormatThreadsActive, "Number of threads in the ParallelFormattingOutputFormatThreads thread pool running a task.") \ + M(ParallelFormattingOutputFormatThreadsScheduled, "Number of queued or active jobs in the ParallelFormattingOutputFormatThreads thread pool.") \ M(ParallelParsingInputFormatThreads, "Number of threads in the ParallelParsingInputFormat thread pool.") \ M(ParallelParsingInputFormatThreadsActive, "Number of threads in the ParallelParsingInputFormat thread pool running a task.") \ + M(ParallelParsingInputFormatThreadsScheduled, "Number of queued or active jobs in the ParallelParsingInputFormat thread pool.") \ M(MergeTreeBackgroundExecutorThreads, "Number of threads in the MergeTreeBackgroundExecutor thread pool.") \ M(MergeTreeBackgroundExecutorThreadsActive, "Number of threads in the MergeTreeBackgroundExecutor thread pool running a task.") \ + M(MergeTreeBackgroundExecutorThreadsScheduled, "Number of queued or active jobs in the MergeTreeBackgroundExecutor thread pool.") \ M(AsynchronousInsertThreads, "Number of threads in the AsynchronousInsert thread pool.") \ M(AsynchronousInsertThreadsActive, "Number of threads in the AsynchronousInsert thread pool running a task.") \ + M(AsynchronousInsertThreadsScheduled, "Number of queued or active jobs in the AsynchronousInsert thread pool.") \ M(StartupSystemTablesThreads, "Number of threads in the StartupSystemTables thread pool.") \ M(StartupSystemTablesThreadsActive, "Number of threads in the StartupSystemTables thread pool running a task.") \ + M(StartupSystemTablesThreadsScheduled, "Number of queued or active jobs in the StartupSystemTables thread pool.") \ M(AggregatorThreads, "Number of threads in the Aggregator thread pool.") \ M(AggregatorThreadsActive, "Number of threads in the Aggregator thread pool running a task.") \ + M(AggregatorThreadsScheduled, "Number of queued or active jobs in the Aggregator thread pool.") \ M(DDLWorkerThreads, "Number of threads in the DDLWorker thread pool for ON CLUSTER queries.") \ M(DDLWorkerThreadsActive, "Number of threads in the DDLWORKER thread pool for ON CLUSTER queries running a task.") \ + M(DDLWorkerThreadsScheduled, "Number of queued or active jobs in the DDLWORKER thread pool for ON CLUSTER queries.") \ M(StorageDistributedThreads, "Number of threads in the StorageDistributed thread pool.") \ M(StorageDistributedThreadsActive, "Number of threads in the StorageDistributed thread pool running a task.") \ + M(StorageDistributedThreadsScheduled, "Number of queued or active jobs in the StorageDistributed thread pool.") \ M(DistributedInsertThreads, "Number of threads used for INSERT into Distributed.") \ M(DistributedInsertThreadsActive, "Number of threads used for INSERT into Distributed running a task.") \ + M(DistributedInsertThreadsScheduled, "Number of queued or active jobs used for INSERT into Distributed.") \ M(StorageS3Threads, "Number of threads in the StorageS3 thread pool.") \ M(StorageS3ThreadsActive, "Number of threads in the StorageS3 thread pool running a task.") \ + M(StorageS3ThreadsScheduled, "Number of queued or active jobs in the StorageS3 thread pool.") \ M(ObjectStorageS3Threads, "Number of threads in the S3ObjectStorage thread pool.") \ M(ObjectStorageS3ThreadsActive, "Number of threads in the S3ObjectStorage thread pool running a task.") \ + M(ObjectStorageS3ThreadsScheduled, "Number of queued or active jobs in the S3ObjectStorage thread pool.") \ M(ObjectStorageAzureThreads, "Number of threads in the AzureObjectStorage thread pool.") \ M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \ + M(ObjectStorageAzureThreadsScheduled, "Number of queued or active jobs in the AzureObjectStorage thread pool.") \ M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \ M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \ + M(MergeTreePartsLoaderThreadsScheduled, "Number of queued or active jobs in the MergeTree parts loader thread pool.") \ M(MergeTreeOutdatedPartsLoaderThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(MergeTreeOutdatedPartsLoaderThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ + M(MergeTreeOutdatedPartsLoaderThreadsScheduled, "Number of queued or active jobs in the threadpool for loading Outdated data parts.") \ M(MergeTreePartsCleanerThreads, "Number of threads in the MergeTree parts cleaner thread pool.") \ M(MergeTreePartsCleanerThreadsActive, "Number of threads in the MergeTree parts cleaner thread pool running a task.") \ + M(MergeTreePartsCleanerThreadsScheduled, "Number of queued or active jobs in the MergeTree parts cleaner thread pool.") \ M(IDiskCopierThreads, "Number of threads for copying data between disks of different types.") \ M(IDiskCopierThreadsActive, "Number of threads for copying data between disks of different types running a task.") \ + M(IDiskCopierThreadsScheduled, "Number of queued or active jobs for copying data between disks of different types.") \ M(SystemReplicasThreads, "Number of threads in the system.replicas thread pool.") \ M(SystemReplicasThreadsActive, "Number of threads in the system.replicas thread pool running a task.") \ + M(SystemReplicasThreadsScheduled, "Number of queued or active jobs in the system.replicas thread pool.") \ M(RestartReplicaThreads, "Number of threads in the RESTART REPLICA thread pool.") \ M(RestartReplicaThreadsActive, "Number of threads in the RESTART REPLICA thread pool running a task.") \ + M(RestartReplicaThreadsScheduled, "Number of queued or active jobs in the RESTART REPLICA thread pool.") \ M(QueryPipelineExecutorThreads, "Number of threads in the PipelineExecutor thread pool.") \ M(QueryPipelineExecutorThreadsActive, "Number of threads in the PipelineExecutor thread pool running a task.") \ + M(QueryPipelineExecutorThreadsScheduled, "Number of queued or active jobs in the PipelineExecutor thread pool.") \ M(ParquetDecoderThreads, "Number of threads in the ParquetBlockInputFormat thread pool.") \ M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \ + M(ParquetDecoderThreadsScheduled, "Number of queued or active jobs in the ParquetBlockInputFormat thread pool.") \ M(ParquetEncoderThreads, "Number of threads in ParquetBlockOutputFormat thread pool.") \ M(ParquetEncoderThreadsActive, "Number of threads in ParquetBlockOutputFormat thread pool running a task.") \ + M(ParquetEncoderThreadsScheduled, "Number of queued or active jobs in ParquetBlockOutputFormat thread pool.") \ M(DWARFReaderThreads, "Number of threads in the DWARFBlockInputFormat thread pool.") \ M(DWARFReaderThreadsActive, "Number of threads in the DWARFBlockInputFormat thread pool running a task.") \ + M(DWARFReaderThreadsScheduled, "Number of queued or active jobs in the DWARFBlockInputFormat thread pool.") \ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ + M(OutdatedPartsLoadingThreadsScheduled, "Number of queued or active jobs in the threadpool for loading Outdated data parts.") \ M(DistributedBytesToInsert, "Number of pending bytes to process for asynchronous insertion into Distributed tables. Number of bytes for every shard is summed.") \ M(BrokenDistributedBytesToInsert, "Number of bytes for asynchronous insertion into Distributed tables that has been marked as broken. Number of bytes for every shard is summed.") \ M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \ diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 4a5bdeffcee..8cba13373b9 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -25,13 +25,14 @@ namespace CurrentMetrics { extern const Metric GlobalThread; extern const Metric GlobalThreadActive; + extern const Metric GlobalThreadScheduled; } static constexpr auto DEFAULT_THREAD_NAME = "ThreadPool"; template -ThreadPoolImpl::ThreadPoolImpl(Metric metric_threads_, Metric metric_active_threads_) - : ThreadPoolImpl(metric_threads_, metric_active_threads_, getNumberOfPhysicalCPUCores()) +ThreadPoolImpl::ThreadPoolImpl(Metric metric_threads_, Metric metric_active_threads_, Metric metric_scheduled_jobs_) + : ThreadPoolImpl(metric_threads_, metric_active_threads_, metric_scheduled_jobs_, getNumberOfPhysicalCPUCores()) { } @@ -40,8 +41,9 @@ template ThreadPoolImpl::ThreadPoolImpl( Metric metric_threads_, Metric metric_active_threads_, + Metric metric_scheduled_jobs_, size_t max_threads_) - : ThreadPoolImpl(metric_threads_, metric_active_threads_, max_threads_, max_threads_, max_threads_) + : ThreadPoolImpl(metric_threads_, metric_active_threads_, metric_scheduled_jobs_, max_threads_, max_threads_, max_threads_) { } @@ -49,12 +51,14 @@ template ThreadPoolImpl::ThreadPoolImpl( Metric metric_threads_, Metric metric_active_threads_, + Metric metric_scheduled_jobs_, size_t max_threads_, size_t max_free_threads_, size_t queue_size_, bool shutdown_on_exception_) : metric_threads(metric_threads_) , metric_active_threads(metric_active_threads_) + , metric_scheduled_jobs(metric_scheduled_jobs_) , max_threads(max_threads_) , max_free_threads(std::min(max_free_threads_, max_threads)) , queue_size(queue_size_ ? std::max(queue_size_, max_threads) : 0 /* zero means the queue is unlimited */) @@ -187,6 +191,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: jobs.emplace(std::move(job), priority, + metric_scheduled_jobs, /// Tracing context on this thread is used as parent context for the sub-thread that runs the job propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread(), /// capture_frame_pointers @@ -346,13 +351,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// This is inside the loop to also reset previous thread names set inside the jobs. setThreadName(DEFAULT_THREAD_NAME); - /// A copy of parent trace context - DB::OpenTelemetry::TracingContextOnThread parent_thread_trace_context; - - std::vector thread_frame_pointers; - /// Get a job from the queue. - Job job; + std::optional job_data; { std::unique_lock lock(mutex); @@ -393,12 +393,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ } /// boost::priority_queue does not provide interface for getting non-const reference to an element - /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. - job = std::move(const_cast(jobs.top().job)); - parent_thread_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); - DB::Exception::enable_job_stack_trace = jobs.top().enable_job_stack_trace; - if (DB::Exception::enable_job_stack_trace) - thread_frame_pointers = std::move(const_cast &>(jobs.top().frame_pointers)); + /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority. + job_data = std::move(const_cast(jobs.top())); jobs.pop(); /// We don't run jobs after `shutdown` is set, but we have to properly dequeue all jobs and finish them. @@ -412,18 +408,17 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ ALLOW_ALLOCATIONS_IN_SCOPE; /// Set up tracing context for this thread by its parent context. - DB::OpenTelemetry::TracingContextHolder thread_trace_context("ThreadPool::worker()", parent_thread_trace_context); + DB::OpenTelemetry::TracingContextHolder thread_trace_context("ThreadPool::worker()", job_data->thread_trace_context); /// Run the job. try { if (DB::Exception::enable_job_stack_trace) - DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); - + DB::Exception::thread_frame_pointers = std::move(job_data->frame_pointers); CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); - job(); + job_data->job(); if (thread_trace_context.root_span.isTraceEnabled()) { @@ -437,13 +432,13 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ else { /// If the thread name is not set, use the type name of the job instead - thread_trace_context.root_span.operation_name = demangle(job.target_type().name()); + thread_trace_context.root_span.operation_name = demangle(job_data->job.target_type().name()); } } /// job should be reset before decrementing scheduled_jobs to /// ensure that the Job destroyed before wait() returns. - job = {}; + job_data.reset(); } catch (...) { @@ -452,7 +447,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// job should be reset before decrementing scheduled_jobs to /// ensure that the Job destroyed before wait() returns. - job = {}; + job_data.reset(); } job_is_done = true; @@ -475,6 +470,7 @@ GlobalThreadPool::GlobalThreadPool( : FreeThreadPool( CurrentMetrics::GlobalThread, CurrentMetrics::GlobalThreadActive, + CurrentMetrics::GlobalThreadScheduled, max_threads_, max_free_threads_, queue_size_, diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index f5721146e09..c8eefedd838 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -41,18 +41,20 @@ public: using Metric = CurrentMetrics::Metric; /// Maximum number of threads is based on the number of physical cores. - ThreadPoolImpl(Metric metric_threads_, Metric metric_active_threads_); + ThreadPoolImpl(Metric metric_threads_, Metric metric_active_threads_, Metric metric_scheduled_jobs_); /// Size is constant. Up to num_threads are created on demand and then run until shutdown. explicit ThreadPoolImpl( Metric metric_threads_, Metric metric_active_threads_, + Metric metric_scheduled_jobs_, size_t max_threads_); /// queue_size - maximum number of running plus scheduled jobs. It can be greater than max_threads. Zero means unlimited. ThreadPoolImpl( Metric metric_threads_, Metric metric_active_threads_, + Metric metric_scheduled_jobs_, size_t max_threads_, size_t max_free_threads_, size_t queue_size_, @@ -113,6 +115,7 @@ private: Metric metric_threads; Metric metric_active_threads; + Metric metric_scheduled_jobs; size_t max_threads; size_t max_free_threads; @@ -127,14 +130,19 @@ private: { Job job; Priority priority; + CurrentMetrics::Increment metric_increment; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; /// Call stacks of all jobs' schedulings leading to this one std::vector frame_pointers; bool enable_job_stack_trace = false; - JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, bool capture_frame_pointers = false) - : job(job_), priority(priority_), thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers) + JobWithPriority( + Job job_, Priority priority_, CurrentMetrics::Metric metric, + const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, + bool capture_frame_pointers) + : job(job_), priority(priority_), metric_increment(metric), + thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers) { if (!capture_frame_pointers) return; diff --git a/src/Common/examples/parallel_aggregation.cpp b/src/Common/examples/parallel_aggregation.cpp index cf7a3197fef..20f5f1c5224 100644 --- a/src/Common/examples/parallel_aggregation.cpp +++ b/src/Common/examples/parallel_aggregation.cpp @@ -33,6 +33,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } struct SmallLock @@ -254,7 +255,7 @@ int main(int argc, char ** argv) std::cerr << std::fixed << std::setprecision(2); - ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, num_threads); + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, num_threads); Source data(n); diff --git a/src/Common/examples/parallel_aggregation2.cpp b/src/Common/examples/parallel_aggregation2.cpp index 1b0ad760490..e7136707dbd 100644 --- a/src/Common/examples/parallel_aggregation2.cpp +++ b/src/Common/examples/parallel_aggregation2.cpp @@ -29,6 +29,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } template @@ -281,7 +282,7 @@ int main(int argc, char ** argv) std::cerr << std::fixed << std::setprecision(2); - ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, num_threads); + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, num_threads); Source data(n); diff --git a/src/Common/examples/thread_creation_latency.cpp b/src/Common/examples/thread_creation_latency.cpp index 2434759c968..60fb27dc345 100644 --- a/src/Common/examples/thread_creation_latency.cpp +++ b/src/Common/examples/thread_creation_latency.cpp @@ -19,6 +19,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } namespace DB @@ -72,7 +73,7 @@ int main(int argc, char ** argv) test(n, "Create and destroy ThreadPool each iteration", [] { - ThreadPool tp(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 1); + ThreadPool tp(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 1); tp.scheduleOrThrowOnError(f); tp.wait(); }); @@ -93,7 +94,7 @@ int main(int argc, char ** argv) }); { - ThreadPool tp(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 1); + ThreadPool tp(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 1); test(n, "Schedule job for Threadpool each iteration", [&tp] { @@ -103,7 +104,7 @@ int main(int argc, char ** argv) } { - ThreadPool tp(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 128); + ThreadPool tp(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 128); test(n, "Schedule job for Threadpool with 128 threads each iteration", [&tp] { diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index dfcbf27b9fc..28d47593e78 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -21,6 +21,7 @@ namespace CurrentMetrics { extern const Metric TablesLoaderThreads; extern const Metric TablesLoaderThreadsActive; + extern const Metric TablesLoaderThreadsScheduled; } namespace DB::ErrorCodes @@ -62,6 +63,7 @@ struct AsyncLoaderTest .name = fmt::format("Pool{}", pool_id), .metric_threads = CurrentMetrics::TablesLoaderThreads, .metric_active_threads = CurrentMetrics::TablesLoaderThreadsActive, + .metric_scheduled_threads = CurrentMetrics::TablesLoaderThreadsScheduled, .max_threads = desc.max_threads, .priority = desc.priority }); diff --git a/src/Common/tests/gtest_thread_pool_concurrent_wait.cpp b/src/Common/tests/gtest_thread_pool_concurrent_wait.cpp index f93017129dd..ddaff3382db 100644 --- a/src/Common/tests/gtest_thread_pool_concurrent_wait.cpp +++ b/src/Common/tests/gtest_thread_pool_concurrent_wait.cpp @@ -12,6 +12,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } TEST(ThreadPool, ConcurrentWait) @@ -25,14 +26,14 @@ TEST(ThreadPool, ConcurrentWait) constexpr size_t num_threads = 4; constexpr size_t num_jobs = 4; - ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, num_threads); + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, num_threads); for (size_t i = 0; i < num_jobs; ++i) pool.scheduleOrThrowOnError(worker); constexpr size_t num_waiting_threads = 4; - ThreadPool waiting_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, num_waiting_threads); + ThreadPool waiting_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, num_waiting_threads); for (size_t i = 0; i < num_waiting_threads; ++i) waiting_pool.scheduleOrThrowOnError([&pool] { pool.wait(); }); diff --git a/src/Common/tests/gtest_thread_pool_global_full.cpp b/src/Common/tests/gtest_thread_pool_global_full.cpp index 1b2ded9c7e1..4507998be3c 100644 --- a/src/Common/tests/gtest_thread_pool_global_full.cpp +++ b/src/Common/tests/gtest_thread_pool_global_full.cpp @@ -11,6 +11,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } /// Test what happens if local ThreadPool cannot create a ThreadFromGlobalPool. @@ -34,7 +35,7 @@ TEST(ThreadPool, GlobalFull1) auto func = [&] { ++counter; while (counter != num_jobs) {} }; - ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, num_jobs); + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, num_jobs); for (size_t i = 0; i < capacity; ++i) pool.scheduleOrThrowOnError(func); @@ -72,11 +73,11 @@ TEST(ThreadPool, GlobalFull2) std::atomic counter = 0; auto func = [&] { ++counter; while (counter != capacity + 1) {} }; - ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, capacity, 0, capacity); + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, capacity, 0, capacity); for (size_t i = 0; i < capacity; ++i) pool.scheduleOrThrowOnError(func); - ThreadPool another_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 1); + ThreadPool another_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 1); EXPECT_THROW(another_pool.scheduleOrThrowOnError(func), DB::Exception); ++counter; diff --git a/src/Common/tests/gtest_thread_pool_limit.cpp b/src/Common/tests/gtest_thread_pool_limit.cpp index b47c8cdad18..66c6f8dc122 100644 --- a/src/Common/tests/gtest_thread_pool_limit.cpp +++ b/src/Common/tests/gtest_thread_pool_limit.cpp @@ -8,6 +8,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } /// Test for thread self-removal when number of free threads in pool is too large. @@ -16,7 +17,7 @@ namespace CurrentMetrics template int test() { - Pool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 10, 2, 10); + Pool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 10, 2, 10); std::atomic counter{0}; for (size_t i = 0; i < 10; ++i) diff --git a/src/Common/tests/gtest_thread_pool_loop.cpp b/src/Common/tests/gtest_thread_pool_loop.cpp index 170a888ff72..4257c0b73a5 100644 --- a/src/Common/tests/gtest_thread_pool_loop.cpp +++ b/src/Common/tests/gtest_thread_pool_loop.cpp @@ -9,6 +9,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } TEST(ThreadPool, Loop) @@ -18,7 +19,7 @@ TEST(ThreadPool, Loop) for (size_t i = 0; i < 1000; ++i) { size_t threads = 16; - ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, threads); + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, threads); for (size_t j = 0; j < threads; ++j) pool.scheduleOrThrowOnError([&] { ++res; }); pool.wait(); diff --git a/src/Common/tests/gtest_thread_pool_schedule_exception.cpp b/src/Common/tests/gtest_thread_pool_schedule_exception.cpp index d8e00b5314c..1b4f27e59e0 100644 --- a/src/Common/tests/gtest_thread_pool_schedule_exception.cpp +++ b/src/Common/tests/gtest_thread_pool_schedule_exception.cpp @@ -9,11 +9,12 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } static bool check() { - ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 10); + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 10); /// The throwing thread. pool.scheduleOrThrowOnError([] { throw std::runtime_error("Hello, world!"); }); @@ -53,7 +54,7 @@ TEST(ThreadPool, ExceptionFromSchedule) static bool check2() { - ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 2); + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 2); try { diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index ef23e85e54b..4001673e01a 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -25,6 +25,7 @@ namespace CurrentMetrics extern const Metric BackgroundSchedulePoolSize; extern const Metric IOWriterThreads; extern const Metric IOWriterThreadsActive; + extern const Metric IOWriterThreadsScheduled; } namespace DB diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 96c084a261c..549711150b8 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -30,6 +30,7 @@ namespace CurrentMetrics { extern const Metric DatabaseOnDiskThreads; extern const Metric DatabaseOnDiskThreadsActive; + extern const Metric DatabaseOnDiskThreadsScheduled; } namespace DB @@ -627,7 +628,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat } /// Read and parse metadata in parallel - ThreadPool pool(CurrentMetrics::DatabaseOnDiskThreads, CurrentMetrics::DatabaseOnDiskThreadsActive); + ThreadPool pool(CurrentMetrics::DatabaseOnDiskThreads, CurrentMetrics::DatabaseOnDiskThreadsActive, CurrentMetrics::DatabaseOnDiskThreadsScheduled); for (const auto & file : metadata_files) { pool.scheduleOrThrowOnError([&]() diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 51d37b84e14..1b2a7b9d5e2 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -34,6 +34,7 @@ namespace CurrentMetrics { extern const Metric DatabaseOrdinaryThreads; extern const Metric DatabaseOrdinaryThreadsActive; + extern const Metric DatabaseOrdinaryThreadsScheduled; } namespace DB @@ -106,7 +107,7 @@ void DatabaseOrdinary::loadStoredObjects(ContextMutablePtr local_context, Loadin std::atomic dictionaries_processed{0}; std::atomic tables_processed{0}; - ThreadPool pool(CurrentMetrics::DatabaseOrdinaryThreads, CurrentMetrics::DatabaseOrdinaryThreadsActive); + ThreadPool pool(CurrentMetrics::DatabaseOrdinaryThreads, CurrentMetrics::DatabaseOrdinaryThreadsActive, CurrentMetrics::DatabaseOrdinaryThreadsScheduled); /// We must attach dictionaries before attaching tables /// because while we're attaching tables we may need to have some dictionaries attached diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index f8b4e7fe33b..5cc7e77a6de 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -15,6 +15,7 @@ namespace CurrentMetrics { extern const Metric TablesLoaderThreads; extern const Metric TablesLoaderThreadsActive; + extern const Metric TablesLoaderThreadsScheduled; } namespace DB @@ -32,7 +33,7 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database , referential_dependencies("ReferentialDeps") , loading_dependencies("LoadingDeps") , all_loading_dependencies("LoadingDeps") - , pool(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive) + , pool(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, CurrentMetrics::TablesLoaderThreadsScheduled) { metadata.default_database = global_context->getCurrentDatabase(); log = &Poco::Logger::get("TablesLoader"); diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp index 3aa3cbb9a4b..1e9b1da390a 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp @@ -9,6 +9,7 @@ namespace CurrentMetrics { extern const Metric CacheDictionaryThreads; extern const Metric CacheDictionaryThreadsActive; + extern const Metric CacheDictionaryThreadsScheduled; } namespace DB @@ -33,7 +34,7 @@ CacheDictionaryUpdateQueue::CacheDictionaryUpdateQueue( , configuration(configuration_) , update_func(std::move(update_func_)) , update_queue(configuration.max_update_queue_size) - , update_pool(CurrentMetrics::CacheDictionaryThreads, CurrentMetrics::CacheDictionaryThreadsActive, configuration.max_threads_for_updates) + , update_pool(CurrentMetrics::CacheDictionaryThreads, CurrentMetrics::CacheDictionaryThreadsActive, CurrentMetrics::CacheDictionaryThreadsScheduled, configuration.max_threads_for_updates) { for (size_t i = 0; i < configuration.max_threads_for_updates; ++i) update_pool.scheduleOrThrowOnError([this] { updateThreadFunction(); }); diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 0556e2bb266..9c5dfeef6ca 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -32,6 +32,7 @@ namespace CurrentMetrics { extern const Metric HashedDictionaryThreads; extern const Metric HashedDictionaryThreadsActive; + extern const Metric HashedDictionaryThreadsScheduled; } namespace DB @@ -59,7 +60,7 @@ public: explicit ParallelDictionaryLoader(HashedDictionary & dictionary_) : dictionary(dictionary_) , shards(dictionary.configuration.shards) - , pool(CurrentMetrics::HashedDictionaryThreads, CurrentMetrics::HashedDictionaryThreadsActive, shards) + , pool(CurrentMetrics::HashedDictionaryThreads, CurrentMetrics::HashedDictionaryThreadsActive, CurrentMetrics::HashedDictionaryThreadsScheduled, shards) , shards_queues(shards) { UInt64 backlog = dictionary.configuration.shard_load_queue_backlog; @@ -229,7 +230,7 @@ HashedDictionary::~HashedDictionary() return; size_t shards = std::max(configuration.shards, 1); - ThreadPool pool(CurrentMetrics::HashedDictionaryThreads, CurrentMetrics::HashedDictionaryThreadsActive, shards); + ThreadPool pool(CurrentMetrics::HashedDictionaryThreads, CurrentMetrics::HashedDictionaryThreadsActive, CurrentMetrics::HashedDictionaryThreadsScheduled, shards); size_t hash_tables_count = 0; auto schedule_destroy = [&hash_tables_count, &pool](auto & container) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 6911fd86db2..5fcf9a9dce4 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -38,6 +38,7 @@ namespace CurrentMetrics { extern const Metric IDiskCopierThreads; extern const Metric IDiskCopierThreadsActive; + extern const Metric IDiskCopierThreadsScheduled; } namespace DB @@ -117,13 +118,13 @@ public: /// Default constructor. IDisk(const String & name_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) : name(name_) - , copying_thread_pool(CurrentMetrics::IDiskCopierThreads, CurrentMetrics::IDiskCopierThreadsActive, config.getUInt(config_prefix + ".thread_pool_size", 16)) + , copying_thread_pool(CurrentMetrics::IDiskCopierThreads, CurrentMetrics::IDiskCopierThreadsActive, CurrentMetrics::IDiskCopierThreadsScheduled, config.getUInt(config_prefix + ".thread_pool_size", 16)) { } explicit IDisk(const String & name_) : name(name_) - , copying_thread_pool(CurrentMetrics::IDiskCopierThreads, CurrentMetrics::IDiskCopierThreadsActive, 16) + , copying_thread_pool(CurrentMetrics::IDiskCopierThreads, CurrentMetrics::IDiskCopierThreadsActive, CurrentMetrics::IDiskCopierThreadsScheduled, 16) { } diff --git a/src/Disks/IO/ThreadPoolReader.cpp b/src/Disks/IO/ThreadPoolReader.cpp index cd3f2d8dea0..15cbcdf5b9b 100644 --- a/src/Disks/IO/ThreadPoolReader.cpp +++ b/src/Disks/IO/ThreadPoolReader.cpp @@ -64,6 +64,7 @@ namespace CurrentMetrics extern const Metric Read; extern const Metric ThreadPoolFSReaderThreads; extern const Metric ThreadPoolFSReaderThreadsActive; + extern const Metric ThreadPoolFSReaderThreadsScheduled; } @@ -88,7 +89,7 @@ static bool hasBugInPreadV2() #endif ThreadPoolReader::ThreadPoolReader(size_t pool_size, size_t queue_size_) - : pool(std::make_unique(CurrentMetrics::ThreadPoolFSReaderThreads, CurrentMetrics::ThreadPoolFSReaderThreadsActive, pool_size, pool_size, queue_size_)) + : pool(std::make_unique(CurrentMetrics::ThreadPoolFSReaderThreads, CurrentMetrics::ThreadPoolFSReaderThreadsActive, CurrentMetrics::ThreadPoolFSReaderThreadsScheduled, pool_size, pool_size, queue_size_)) { } diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 0ec5e0fd6c1..ac599bb3547 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -29,6 +29,7 @@ namespace CurrentMetrics extern const Metric RemoteRead; extern const Metric ThreadPoolRemoteFSReaderThreads; extern const Metric ThreadPoolRemoteFSReaderThreadsActive; + extern const Metric ThreadPoolRemoteFSReaderThreadsScheduled; } namespace DB @@ -59,6 +60,7 @@ namespace ThreadPoolRemoteFSReader::ThreadPoolRemoteFSReader(size_t pool_size, size_t queue_size_) : pool(std::make_unique(CurrentMetrics::ThreadPoolRemoteFSReaderThreads, CurrentMetrics::ThreadPoolRemoteFSReaderThreadsActive, + CurrentMetrics::ThreadPoolRemoteFSReaderThreadsScheduled, pool_size, pool_size, queue_size_)) { } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index fcb82daca95..068e2aebab1 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -14,11 +14,12 @@ #include #include + namespace CurrentMetrics { extern const Metric ObjectStorageAzureThreads; extern const Metric ObjectStorageAzureThreadsActive; - + extern const Metric ObjectStorageAzureThreadsScheduled; } namespace DB @@ -45,6 +46,7 @@ public: : IObjectStorageIteratorAsync( CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, + CurrentMetrics::ObjectStorageAzureThreadsScheduled, "ListObjectAzure") , client(client_) { diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 33b98cd328c..0314e0a7e92 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -13,6 +13,7 @@ namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } namespace DB @@ -156,7 +157,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::migrateToRestorableSchema() { LOG_INFO(disk->log, "Start migration to restorable schema for disk {}", disk->name); - ThreadPool pool{CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive}; + ThreadPool pool{CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled}; for (const auto & root : data_roots) if (disk->exists(root)) @@ -355,7 +356,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * { LOG_INFO(disk->log, "Starting restore files for disk {}", disk->name); - ThreadPool pool{CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive}; + ThreadPool pool{CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled}; auto restore_files = [this, &source_object_storage, &restore_information, &pool](const RelativePathsWithMetadata & objects) { std::vector keys_names; diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index 7425f629a5a..990e66fc4e5 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -2,6 +2,7 @@ #include + namespace DB { diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index b0dd3cef39c..a6abe03bac9 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -6,6 +6,7 @@ #include #include + namespace DB { @@ -15,8 +16,9 @@ public: IObjectStorageIteratorAsync( CurrentMetrics::Metric threads_metric, CurrentMetrics::Metric threads_active_metric, + CurrentMetrics::Metric threads_scheduled_metric, const std::string & thread_name) - : list_objects_pool(threads_metric, threads_active_metric, 1) + : list_objects_pool(threads_metric, threads_active_metric, threads_scheduled_metric, 1) , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, thread_name)) { } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index b36185249af..3af316bf0cf 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -37,6 +37,7 @@ namespace CurrentMetrics { extern const Metric ObjectStorageS3Threads; extern const Metric ObjectStorageS3ThreadsActive; + extern const Metric ObjectStorageS3ThreadsScheduled; } @@ -105,6 +106,7 @@ public: : IObjectStorageIteratorAsync( CurrentMetrics::ObjectStorageS3Threads, CurrentMetrics::ObjectStorageS3ThreadsActive, + CurrentMetrics::ObjectStorageS3ThreadsScheduled, "ListObjectS3") , client(client_) { diff --git a/src/IO/SharedThreadPools.cpp b/src/IO/SharedThreadPools.cpp index 6a0e953f0ef..4a0ef173669 100644 --- a/src/IO/SharedThreadPools.cpp +++ b/src/IO/SharedThreadPools.cpp @@ -7,14 +7,19 @@ namespace CurrentMetrics { extern const Metric IOThreads; extern const Metric IOThreadsActive; + extern const Metric IOThreadsScheduled; extern const Metric BackupsIOThreads; extern const Metric BackupsIOThreadsActive; + extern const Metric BackupsIOThreadsScheduled; extern const Metric MergeTreePartsLoaderThreads; extern const Metric MergeTreePartsLoaderThreadsActive; + extern const Metric MergeTreePartsLoaderThreadsScheduled; extern const Metric MergeTreePartsCleanerThreads; extern const Metric MergeTreePartsCleanerThreadsActive; + extern const Metric MergeTreePartsCleanerThreadsScheduled; extern const Metric MergeTreeOutdatedPartsLoaderThreads; extern const Metric MergeTreeOutdatedPartsLoaderThreadsActive; + extern const Metric MergeTreeOutdatedPartsLoaderThreadsScheduled; } namespace DB @@ -29,10 +34,12 @@ namespace ErrorCodes StaticThreadPool::StaticThreadPool( const String & name_, CurrentMetrics::Metric threads_metric_, - CurrentMetrics::Metric threads_active_metric_) + CurrentMetrics::Metric threads_active_metric_, + CurrentMetrics::Metric threads_scheduled_metric_) : name(name_) , threads_metric(threads_metric_) , threads_active_metric(threads_active_metric_) + , threads_scheduled_metric(threads_scheduled_metric_) { } @@ -47,6 +54,7 @@ void StaticThreadPool::initialize(size_t max_threads, size_t max_free_threads, s instance = std::make_unique( threads_metric, threads_active_metric, + threads_scheduled_metric, max_threads, max_free_threads, queue_size, @@ -110,31 +118,31 @@ void StaticThreadPool::setMaxTurboThreads(size_t max_threads_turbo_) StaticThreadPool & getIOThreadPool() { - static StaticThreadPool instance("IOThreadPool", CurrentMetrics::IOThreads, CurrentMetrics::IOThreadsActive); + static StaticThreadPool instance("IOThreadPool", CurrentMetrics::IOThreads, CurrentMetrics::IOThreadsActive, CurrentMetrics::IOThreadsScheduled); return instance; } StaticThreadPool & getBackupsIOThreadPool() { - static StaticThreadPool instance("BackupsIOThreadPool", CurrentMetrics::BackupsIOThreads, CurrentMetrics::BackupsIOThreadsActive); + static StaticThreadPool instance("BackupsIOThreadPool", CurrentMetrics::BackupsIOThreads, CurrentMetrics::BackupsIOThreadsActive, CurrentMetrics::BackupsIOThreadsScheduled); return instance; } StaticThreadPool & getActivePartsLoadingThreadPool() { - static StaticThreadPool instance("MergeTreePartsLoaderThreadPool", CurrentMetrics::MergeTreePartsLoaderThreads, CurrentMetrics::MergeTreePartsLoaderThreadsActive); + static StaticThreadPool instance("MergeTreePartsLoaderThreadPool", CurrentMetrics::MergeTreePartsLoaderThreads, CurrentMetrics::MergeTreePartsLoaderThreadsActive, CurrentMetrics::MergeTreeOutdatedPartsLoaderThreadsScheduled); return instance; } StaticThreadPool & getPartsCleaningThreadPool() { - static StaticThreadPool instance("MergeTreePartsCleanerThreadPool", CurrentMetrics::MergeTreePartsCleanerThreads, CurrentMetrics::MergeTreePartsCleanerThreadsActive); + static StaticThreadPool instance("MergeTreePartsCleanerThreadPool", CurrentMetrics::MergeTreePartsCleanerThreads, CurrentMetrics::MergeTreePartsCleanerThreadsActive, CurrentMetrics::MergeTreePartsCleanerThreadsScheduled); return instance; } StaticThreadPool & getOutdatedPartsLoadingThreadPool() { - static StaticThreadPool instance("MergeTreeOutdatedPartsLoaderThreadPool", CurrentMetrics::MergeTreeOutdatedPartsLoaderThreads, CurrentMetrics::MergeTreeOutdatedPartsLoaderThreadsActive); + static StaticThreadPool instance("MergeTreeOutdatedPartsLoaderThreadPool", CurrentMetrics::MergeTreeOutdatedPartsLoaderThreads, CurrentMetrics::MergeTreeOutdatedPartsLoaderThreadsActive, CurrentMetrics::MergeTreeOutdatedPartsLoaderThreadsScheduled); return instance; } diff --git a/src/IO/SharedThreadPools.h b/src/IO/SharedThreadPools.h index 188a2a4f003..f37f3acefe7 100644 --- a/src/IO/SharedThreadPools.h +++ b/src/IO/SharedThreadPools.h @@ -8,6 +8,7 @@ #include #include + namespace DB { @@ -17,7 +18,8 @@ public: StaticThreadPool( const String & name_, CurrentMetrics::Metric threads_metric_, - CurrentMetrics::Metric threads_active_metric_); + CurrentMetrics::Metric threads_active_metric_, + CurrentMetrics::Metric threads_scheduled_metric_); ThreadPool & get(); @@ -34,6 +36,7 @@ private: const String name; const CurrentMetrics::Metric threads_metric; const CurrentMetrics::Metric threads_active_metric; + const CurrentMetrics::Metric threads_scheduled_metric; std::unique_ptr instance; std::mutex mutex; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 129c02f032b..e2ddfbe3418 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -63,6 +63,7 @@ namespace CurrentMetrics extern const Metric TemporaryFilesForAggregation; extern const Metric AggregatorThreads; extern const Metric AggregatorThreadsActive; + extern const Metric AggregatorThreadsScheduled; } namespace DB @@ -2466,7 +2467,7 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b std::unique_ptr thread_pool; if (max_threads > 1 && data_variants.sizeWithoutOverflowRow() > 100000 /// TODO Make a custom threshold. && data_variants.isTwoLevel()) /// TODO Use the shared thread pool with the `merge` function. - thread_pool = std::make_unique(CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, max_threads); + thread_pool = std::make_unique(CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, CurrentMetrics::AggregatorThreadsScheduled, max_threads); if (data_variants.without_key) blocks.emplace_back(prepareBlockAndFillWithoutKey( @@ -2656,7 +2657,7 @@ void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl( void NO_INLINE Aggregator::mergeWithoutKeyDataImpl( ManyAggregatedDataVariants & non_empty_data) const { - ThreadPool thread_pool{CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, params.max_threads}; + ThreadPool thread_pool{CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, CurrentMetrics::AggregatorThreadsScheduled, params.max_threads}; AggregatedDataVariantsPtr & res = non_empty_data[0]; @@ -3144,7 +3145,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari std::unique_ptr thread_pool; if (max_threads > 1 && total_input_rows > 100000) /// TODO Make a custom threshold. - thread_pool = std::make_unique(CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, max_threads); + thread_pool = std::make_unique(CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, CurrentMetrics::AggregatorThreadsScheduled, max_threads); for (const auto & bucket_blocks : bucket_to_blocks) { diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 04285a06a65..a0750122a5c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -39,6 +39,7 @@ namespace CurrentMetrics extern const Metric PendingAsyncInsert; extern const Metric AsynchronousInsertThreads; extern const Metric AsynchronousInsertThreadsActive; + extern const Metric AsynchronousInsertThreadsScheduled; } namespace ProfileEvents @@ -175,7 +176,7 @@ AsynchronousInsertQueue::AsynchronousInsertQueue(ContextPtr context_, size_t poo , pool_size(pool_size_) , flush_on_shutdown(flush_on_shutdown_) , queue_shards(pool_size) - , pool(CurrentMetrics::AsynchronousInsertThreads, CurrentMetrics::AsynchronousInsertThreadsActive, pool_size) + , pool(CurrentMetrics::AsynchronousInsertThreads, CurrentMetrics::AsynchronousInsertThreadsActive, CurrentMetrics::AsynchronousInsertThreadsScheduled, pool_size) { if (!pool_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "pool_size cannot be zero"); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c6640d582a2..c13be020933 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -135,10 +135,13 @@ namespace CurrentMetrics extern const Metric BackgroundCommonPoolSize; extern const Metric MarksLoaderThreads; extern const Metric MarksLoaderThreadsActive; + extern const Metric MarksLoaderThreadsScheduled; extern const Metric IOPrefetchThreads; extern const Metric IOPrefetchThreadsActive; + extern const Metric IOPrefetchThreadsScheduled; extern const Metric IOWriterThreads; extern const Metric IOWriterThreadsActive; + extern const Metric IOWriterThreadsScheduled; } @@ -2542,7 +2545,7 @@ ThreadPool & Context::getLoadMarksThreadpool() const auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); shared->load_marks_threadpool = std::make_unique( - CurrentMetrics::MarksLoaderThreads, CurrentMetrics::MarksLoaderThreadsActive, pool_size, pool_size, queue_size); + CurrentMetrics::MarksLoaderThreads, CurrentMetrics::MarksLoaderThreadsActive, CurrentMetrics::MarksLoaderThreadsScheduled, pool_size, pool_size, queue_size); }); return *shared->load_marks_threadpool; @@ -2725,7 +2728,7 @@ ThreadPool & Context::getPrefetchThreadpool() const auto pool_size = config.getUInt(".prefetch_threadpool_pool_size", 100); auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); shared->prefetch_threadpool = std::make_unique( - CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, pool_size, pool_size, queue_size); + CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, CurrentMetrics::IOPrefetchThreadsScheduled, pool_size, pool_size, queue_size); }); return *shared->prefetch_threadpool; @@ -4765,7 +4768,7 @@ ThreadPool & Context::getThreadPoolWriter() const auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); shared->threadpool_writer = std::make_unique( - CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size); + CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, CurrentMetrics::IOWriterThreadsScheduled, pool_size, pool_size, queue_size); }); return *shared->threadpool_writer; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index a9930036e7e..a43c61f3d66 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -46,6 +46,7 @@ namespace CurrentMetrics { extern const Metric DDLWorkerThreads; extern const Metric DDLWorkerThreadsActive; + extern const Metric DDLWorkerThreadsScheduled; } namespace DB @@ -93,7 +94,7 @@ DDLWorker::DDLWorker( { LOG_WARNING(log, "DDLWorker is configured to use multiple threads. " "It's not recommended because queries can be reordered. Also it may cause some unknown issues to appear."); - worker_pool = std::make_unique(CurrentMetrics::DDLWorkerThreads, CurrentMetrics::DDLWorkerThreadsActive, pool_size); + worker_pool = std::make_unique(CurrentMetrics::DDLWorkerThreads, CurrentMetrics::DDLWorkerThreadsActive, CurrentMetrics::DDLWorkerThreadsScheduled, pool_size); } queue_dir = zk_root_dir; @@ -1113,7 +1114,7 @@ void DDLWorker::runMainThread() /// It will wait for all threads in pool to finish and will not rethrow exceptions (if any). /// We create new thread pool to forget previous exceptions. if (1 < pool_size) - worker_pool = std::make_unique(CurrentMetrics::DDLWorkerThreads, CurrentMetrics::DDLWorkerThreadsActive, pool_size); + worker_pool = std::make_unique(CurrentMetrics::DDLWorkerThreads, CurrentMetrics::DDLWorkerThreadsActive, CurrentMetrics::DDLWorkerThreadsScheduled, pool_size); /// Clear other in-memory state, like server just started. current_tasks.clear(); last_skipped_entry_name.reset(); @@ -1152,7 +1153,7 @@ void DDLWorker::runMainThread() initialized = false; /// Wait for pending async tasks if (1 < pool_size) - worker_pool = std::make_unique(CurrentMetrics::DDLWorkerThreads, CurrentMetrics::DDLWorkerThreadsActive, pool_size); + worker_pool = std::make_unique(CurrentMetrics::DDLWorkerThreads, CurrentMetrics::DDLWorkerThreadsActive, CurrentMetrics::DDLWorkerThreadsScheduled, pool_size); LOG_INFO(log, "Lost ZooKeeper connection, will try to connect again: {}", getCurrentExceptionMessage(true)); } else diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8dc90e1a2f1..e0ce2b99b14 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -44,6 +44,7 @@ namespace CurrentMetrics extern const Metric TablesToDropQueueSize; extern const Metric DatabaseCatalogThreads; extern const Metric DatabaseCatalogThreadsActive; + extern const Metric DatabaseCatalogThreadsScheduled; } namespace DB @@ -1024,7 +1025,7 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() LOG_INFO(log, "Found {} partially dropped tables. Will load them and retry removal.", dropped_metadata.size()); - ThreadPool pool(CurrentMetrics::DatabaseCatalogThreads, CurrentMetrics::DatabaseCatalogThreadsActive); + ThreadPool pool(CurrentMetrics::DatabaseCatalogThreads, CurrentMetrics::DatabaseCatalogThreadsActive, CurrentMetrics::DatabaseCatalogThreadsScheduled); for (const auto & elem : dropped_metadata) { pool.scheduleOrThrowOnError([&]() diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b42745b726c..ff95b3014dc 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -76,6 +76,7 @@ namespace CurrentMetrics { extern const Metric RestartReplicaThreads; extern const Metric RestartReplicaThreadsActive; + extern const Metric RestartReplicaThreadsScheduled; } namespace DB @@ -809,7 +810,7 @@ void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context) size_t threads = std::min(static_cast(getNumberOfPhysicalCPUCores()), replica_names.size()); LOG_DEBUG(log, "Will restart {} replicas using {} threads", replica_names.size(), threads); - ThreadPool pool(CurrentMetrics::RestartReplicaThreads, CurrentMetrics::RestartReplicaThreadsActive, threads); + ThreadPool pool(CurrentMetrics::RestartReplicaThreads, CurrentMetrics::RestartReplicaThreadsActive, CurrentMetrics::RestartReplicaThreadsScheduled, threads); for (auto & replica : replica_names) { diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 3612dbfdc4e..44b57396bd5 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -32,6 +32,7 @@ namespace CurrentMetrics { extern const Metric StartupSystemTablesThreads; extern const Metric StartupSystemTablesThreadsActive; + extern const Metric StartupSystemTablesThreadsScheduled; } namespace DB @@ -377,7 +378,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons if (!tables_started) { /// It's not quite correct to run DDL queries while database is not started up. - ThreadPool pool(CurrentMetrics::StartupSystemTablesThreads, CurrentMetrics::StartupSystemTablesThreadsActive); + ThreadPool pool(CurrentMetrics::StartupSystemTablesThreads, CurrentMetrics::StartupSystemTablesThreadsActive, CurrentMetrics::StartupSystemTablesThreadsScheduled); DatabaseCatalog::instance().getSystemDatabase()->startupTables(pool, LoadingStrictnessLevel::FORCE_RESTORE); } @@ -472,7 +473,7 @@ void convertDatabasesEnginesIfNeed(ContextMutablePtr context) void startupSystemTables() { - ThreadPool pool(CurrentMetrics::StartupSystemTablesThreads, CurrentMetrics::StartupSystemTablesThreadsActive); + ThreadPool pool(CurrentMetrics::StartupSystemTablesThreads, CurrentMetrics::StartupSystemTablesThreadsActive, CurrentMetrics::StartupSystemTablesThreadsScheduled); DatabaseCatalog::instance().getSystemDatabase()->startupTables(pool, LoadingStrictnessLevel::FORCE_RESTORE); } diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index eb90b61cf31..2b943110273 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -24,9 +24,10 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & if (thread_group) CurrentThread::attachToGroup(thread_group); - SCOPE_EXIT_SAFE({ + SCOPE_EXIT_SAFE( + { { - /// Release all captutred resources before detaching thread group + /// Release all captured resources before detaching thread group /// Releasing has to use proper memory tracker which has been set here before callback [[maybe_unused]] auto tmp = std::move(my_callback); @@ -34,7 +35,6 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & if (thread_group) CurrentThread::detachFromGroupIfNotDetached(); - }); setThreadName(thread_name.data()); diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 37af391fba3..812b64ccdb8 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -22,6 +22,7 @@ namespace CurrentMetrics { extern const Metric QueryPipelineExecutorThreads; extern const Metric QueryPipelineExecutorThreadsActive; + extern const Metric QueryPipelineExecutorThreadsScheduled; } namespace DB @@ -332,7 +333,7 @@ void PipelineExecutor::initializeExecution(size_t num_threads, bool concurrency_ tasks.fill(queue); if (num_threads > 1) - pool = std::make_unique(CurrentMetrics::QueryPipelineExecutorThreads, CurrentMetrics::QueryPipelineExecutorThreadsActive, num_threads); + pool = std::make_unique(CurrentMetrics::QueryPipelineExecutorThreads, CurrentMetrics::QueryPipelineExecutorThreadsActive, CurrentMetrics::QueryPipelineExecutorThreadsScheduled, num_threads); } void PipelineExecutor::spawnThreads() diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp index c5f8059f93c..19ed5c94dfd 100644 --- a/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp @@ -28,6 +28,7 @@ namespace CurrentMetrics { extern const Metric DWARFReaderThreads; extern const Metric DWARFReaderThreadsActive; + extern const Metric DWARFReaderThreadsScheduled; } namespace DB @@ -238,7 +239,7 @@ void DWARFBlockInputFormat::initializeIfNeeded() LOG_DEBUG(&Poco::Logger::get("DWARF"), "{} units, reading in {} threads", units_queue.size(), num_threads); - pool.emplace(CurrentMetrics::DWARFReaderThreads, CurrentMetrics::DWARFReaderThreadsActive, num_threads); + pool.emplace(CurrentMetrics::DWARFReaderThreads, CurrentMetrics::DWARFReaderThreadsActive, CurrentMetrics::DWARFReaderThreadsScheduled, num_threads); for (size_t i = 0; i < num_threads; ++i) pool->scheduleOrThrowOnError( [this, thread_group = CurrentThread::getGroup()]() diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index bf8968dd376..c2f08479730 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -21,6 +21,7 @@ namespace CurrentMetrics { extern const Metric ParallelFormattingOutputFormatThreads; extern const Metric ParallelFormattingOutputFormatThreadsActive; + extern const Metric ParallelFormattingOutputFormatThreadsScheduled; } namespace DB @@ -80,7 +81,7 @@ public: explicit ParallelFormattingOutputFormat(Params params) : IOutputFormat(params.header, params.out) , internal_formatter_creator(params.internal_formatter_creator) - , pool(CurrentMetrics::ParallelFormattingOutputFormatThreads, CurrentMetrics::ParallelFormattingOutputFormatThreadsActive, params.max_threads_for_parallel_formatting) + , pool(CurrentMetrics::ParallelFormattingOutputFormatThreads, CurrentMetrics::ParallelFormattingOutputFormatThreadsActive, CurrentMetrics::ParallelFormattingOutputFormatThreadsScheduled, params.max_threads_for_parallel_formatting) { LOG_TEST(&Poco::Logger::get("ParallelFormattingOutputFormat"), "Parallel formatting is being used"); diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index f61dc3fbc78..be9e50b854b 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -18,6 +18,7 @@ namespace CurrentMetrics { extern const Metric ParallelParsingInputFormatThreads; extern const Metric ParallelParsingInputFormatThreadsActive; + extern const Metric ParallelParsingInputFormatThreadsScheduled; } namespace DB @@ -101,7 +102,7 @@ public: , min_chunk_bytes(params.min_chunk_bytes) , max_block_size(params.max_block_size) , is_server(params.is_server) - , pool(CurrentMetrics::ParallelParsingInputFormatThreads, CurrentMetrics::ParallelParsingInputFormatThreadsActive, params.max_threads) + , pool(CurrentMetrics::ParallelParsingInputFormatThreads, CurrentMetrics::ParallelParsingInputFormatThreadsActive, CurrentMetrics::ParallelParsingInputFormatThreadsScheduled, params.max_threads) { // One unit for each thread, including segmentator and reader, plus a // couple more units so that the segmentation thread doesn't spuriously diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index c9ac2438fc0..d37c2dc1160 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -28,6 +28,7 @@ namespace CurrentMetrics { extern const Metric ParquetDecoderThreads; extern const Metric ParquetDecoderThreadsActive; + extern const Metric ParquetDecoderThreadsScheduled; } namespace DB @@ -377,7 +378,7 @@ ParquetBlockInputFormat::ParquetBlockInputFormat( , pending_chunks(PendingChunk::Compare { .row_group_first = format_settings_.parquet.preserve_order }) { if (max_decoding_threads > 1) - pool = std::make_unique(CurrentMetrics::ParquetDecoderThreads, CurrentMetrics::ParquetDecoderThreadsActive, max_decoding_threads); + pool = std::make_unique(CurrentMetrics::ParquetDecoderThreads, CurrentMetrics::ParquetDecoderThreadsActive, CurrentMetrics::ParquetDecoderThreadsScheduled, max_decoding_threads); } ParquetBlockInputFormat::~ParquetBlockInputFormat() diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index fbf8b3a7c87..fb9f853dc01 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -13,6 +13,7 @@ namespace CurrentMetrics { extern const Metric ParquetEncoderThreads; extern const Metric ParquetEncoderThreadsActive; + extern const Metric ParquetEncoderThreadsScheduled; } namespace DB @@ -79,7 +80,9 @@ ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Blo { if (format_settings.parquet.parallel_encoding && format_settings.max_threads > 1) pool = std::make_unique( - CurrentMetrics::ParquetEncoderThreads, CurrentMetrics::ParquetEncoderThreadsActive, + CurrentMetrics::ParquetEncoderThreads, + CurrentMetrics::ParquetEncoderThreadsActive, + CurrentMetrics::ParquetEncoderThreadsScheduled, format_settings.max_threads); using C = FormatSettings::ParquetCompression; diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 3420cdeaa50..61a6acd6bc8 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -13,6 +13,7 @@ namespace CurrentMetrics { extern const Metric DestroyAggregatesThreads; extern const Metric DestroyAggregatesThreadsActive; + extern const Metric DestroyAggregatesThreadsScheduled; } namespace DB @@ -95,6 +96,7 @@ struct ManyAggregatedData const auto pool = std::make_unique( CurrentMetrics::DestroyAggregatesThreads, CurrentMetrics::DestroyAggregatesThreadsActive, + CurrentMetrics::DestroyAggregatesThreadsScheduled, variants.size()); for (auto && variant : variants) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 65a4aa2741a..8c58e304203 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -31,8 +31,6 @@ #include #include #include -#include -#include #include #include @@ -43,6 +41,7 @@ namespace CurrentMetrics extern const Metric DistributedSend; extern const Metric DistributedInsertThreads; extern const Metric DistributedInsertThreadsActive; + extern const Metric DistributedInsertThreadsScheduled; } namespace ProfileEvents @@ -465,6 +464,7 @@ void DistributedSink::writeSync(const Block & block) pool.emplace( CurrentMetrics::DistributedInsertThreads, CurrentMetrics::DistributedInsertThreadsActive, + CurrentMetrics::DistributedInsertThreadsScheduled, max_threads, max_threads, jobs_count); if (!throttler && (settings.max_network_bandwidth || settings.max_network_bytes)) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 1587354452e..f03136e4edf 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -46,6 +46,7 @@ namespace CurrentMetrics { extern const Metric StorageHiveThreads; extern const Metric StorageHiveThreadsActive; + extern const Metric StorageHiveThreadsScheduled; } namespace DB @@ -861,7 +862,7 @@ HiveFiles StorageHive::collectHiveFiles( Int64 hive_max_query_partitions = context_->getSettings().max_partitions_to_read; /// Mutext to protect hive_files, which maybe appended in multiple threads std::mutex hive_files_mutex; - ThreadPool pool{CurrentMetrics::StorageHiveThreads, CurrentMetrics::StorageHiveThreadsActive, max_threads}; + ThreadPool pool{CurrentMetrics::StorageHiveThreads, CurrentMetrics::StorageHiveThreadsActive, CurrentMetrics::StorageHiveThreadsScheduled, max_threads}; if (!partitions.empty()) { for (const auto & partition : partitions) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 1a7a0b5b2c1..a5f503718b6 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -14,6 +14,7 @@ namespace CurrentMetrics { extern const Metric MergeTreeBackgroundExecutorThreads; extern const Metric MergeTreeBackgroundExecutorThreadsActive; + extern const Metric MergeTreeBackgroundExecutorThreadsScheduled; } namespace DB @@ -40,7 +41,7 @@ MergeTreeBackgroundExecutor::MergeTreeBackgroundExecutor( , metric(metric_) , max_tasks_metric(max_tasks_metric_, 2 * max_tasks_count) // active + pending , pool(std::make_unique( - CurrentMetrics::MergeTreeBackgroundExecutorThreads, CurrentMetrics::MergeTreeBackgroundExecutorThreadsActive)) + CurrentMetrics::MergeTreeBackgroundExecutorThreads, CurrentMetrics::MergeTreeBackgroundExecutorThreadsActive, CurrentMetrics::MergeTreeBackgroundExecutorThreadsScheduled)) { if (max_tasks_count == 0) throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Task count for MergeTreeBackgroundExecutor must not be zero"); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e521491c2d5..d1a285b8818 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -53,6 +53,7 @@ namespace CurrentMetrics { extern const Metric MergeTreeDataSelectExecutorThreads; extern const Metric MergeTreeDataSelectExecutorThreadsActive; + extern const Metric MergeTreeDataSelectExecutorThreadsScheduled; } namespace DB @@ -1075,6 +1076,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd ThreadPool pool( CurrentMetrics::MergeTreeDataSelectExecutorThreads, CurrentMetrics::MergeTreeDataSelectExecutorThreadsActive, + CurrentMetrics::MergeTreeDataSelectExecutorThreadsScheduled, num_threads); for (size_t part_index = 0; part_index < parts.size(); ++part_index) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 2e0703a8df3..b43f25b0fff 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -53,6 +53,7 @@ namespace CurrentMetrics { extern const Metric ObjectStorageAzureThreads; extern const Metric ObjectStorageAzureThreadsActive; + extern const Metric ObjectStorageAzureThreadsScheduled; } namespace ProfileEvents @@ -1087,7 +1088,7 @@ StorageAzureBlobSource::StorageAzureBlobSource( , file_iterator(file_iterator_) , need_only_count(need_only_count_) , query_info(query_info_) - , create_reader_pool(CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, 1) + , create_reader_pool(CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, CurrentMetrics::ObjectStorageAzureThreadsScheduled, 1) , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "AzureReader")) { reader = createReader(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 94ce525bc38..2dedc8abdda 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -134,6 +134,7 @@ namespace CurrentMetrics { extern const Metric StorageDistributedThreads; extern const Metric StorageDistributedThreadsActive; + extern const Metric StorageDistributedThreadsScheduled; } namespace DB @@ -1214,7 +1215,7 @@ void StorageDistributed::initializeFromDisk() const auto & disks = data_volume->getDisks(); /// Make initialization for large number of disks parallel. - ThreadPool pool(CurrentMetrics::StorageDistributedThreads, CurrentMetrics::StorageDistributedThreadsActive, disks.size()); + ThreadPool pool(CurrentMetrics::StorageDistributedThreads, CurrentMetrics::StorageDistributedThreadsActive, CurrentMetrics::StorageDistributedThreadsScheduled, disks.size()); for (const DiskPtr & disk : disks) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 63ed84680c9..80ee1e9339d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -77,6 +77,7 @@ namespace CurrentMetrics { extern const Metric StorageS3Threads; extern const Metric StorageS3ThreadsActive; + extern const Metric StorageS3ThreadsScheduled; } namespace ProfileEvents @@ -147,7 +148,7 @@ public: , virtual_columns(virtual_columns_) , read_keys(read_keys_) , request_settings(request_settings_) - , list_objects_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) + , list_objects_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, 1) , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, "ListObjects")) , file_progress_callback(file_progress_callback_) { @@ -499,7 +500,7 @@ StorageS3Source::ReadTaskIterator::ReadTaskIterator( size_t max_threads_count) : callback(callback_) { - ThreadPool pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, max_threads_count); + ThreadPool pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, max_threads_count); auto pool_scheduler = threadPoolCallbackRunner(pool, "S3ReadTaskItr"); std::vector> keys; @@ -564,7 +565,7 @@ StorageS3Source::StorageS3Source( , file_iterator(file_iterator_) , max_parsing_threads(max_parsing_threads_) , need_only_count(need_only_count_) - , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) + , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, 1) , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3Reader")) { } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index ffefd41327d..d9a12095443 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -23,6 +23,7 @@ namespace CurrentMetrics { extern const Metric SystemReplicasThreads; extern const Metric SystemReplicasThreadsActive; + extern const Metric SystemReplicasThreadsScheduled; } namespace DB @@ -59,7 +60,7 @@ private: public: explicit StatusRequestsPool(size_t max_threads) - : thread_pool(CurrentMetrics::SystemReplicasThreads, CurrentMetrics::SystemReplicasThreadsActive, max_threads) + : thread_pool(CurrentMetrics::SystemReplicasThreads, CurrentMetrics::SystemReplicasThreadsActive, CurrentMetrics::SystemReplicasThreadsScheduled, max_threads) , log(&Poco::Logger::get("StatusRequestsPool")) {} diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 13855c6d94e..611ca948c53 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -10,10 +10,12 @@ #include #include + namespace CurrentMetrics { extern const Metric LocalThread; extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; } namespace DB::ErrorCodes @@ -106,7 +108,7 @@ Runner::Runner( std::cerr << "---- Run options ----\n" << std::endl; - pool.emplace(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, concurrency); + pool.emplace(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, concurrency); queue.emplace(concurrency); } @@ -461,4 +463,3 @@ Runner::~Runner() pool->wait(); generator->cleanup(*connections[0]); } - From 16e7bd201ba8a7818b69e2c30ef7c5f4cc0ebfe2 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Sat, 18 Nov 2023 18:22:33 +0000 Subject: [PATCH 532/813] Add experimental flag for PostgresSQL table engine. Also add documentation on how to enable it --- .../table-engines/integrations/materialized-postgresql.md | 8 ++++++++ src/Core/Settings.h | 1 + src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 5 +++++ 3 files changed, 14 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index 02afec5cfd6..4d83ca79d5c 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -8,6 +8,14 @@ sidebar_label: MaterializedPostgreSQL Creates ClickHouse table with an initial data dump of PostgreSQL table and starts replication process, i.e. executes background job to apply new changes as they happen on PostgreSQL table in the remote PostgreSQL database. +:::note +This table engine is experimental. To use it, set `allow_experimental_materialized_postgresql_table` to 1 in your configuration files or by using the `SET` command: +```sql +SET allow_experimental_materialized_postgresql_table=1 +``` +::: + + If more than one table is required, it is highly recommended to use the [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) database engine instead of the table engine and use the `materialized_postgresql_tables_list` setting, which specifies the tables to be replicated (will also be possible to add database `schema`). It will be much better in terms of CPU, fewer connections and fewer replication slots inside the remote PostgreSQL database. ## Creating a Table {#creating-a-table} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3b90a3e068b..f1379d162ae 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -791,6 +791,7 @@ class IColumn; M(Bool, optimize_uniq_to_count, true, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) \ \ /** Experimental functions */ \ + M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the materialized PostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index a7650983db8..50d0aa67f0b 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -546,6 +546,11 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); + if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "MaterializedPostgreSQL is an experimental table engine." + " You can enable it with the `allow_experimental_materialized_postgresql_table` setting"); + } if (!args.storage_def->order_by && args.storage_def->primary_key) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); From 593f04a6b5825182940e1f241989d801b57ed93d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Nov 2023 20:19:24 +0100 Subject: [PATCH 533/813] Fix style --- src/IO/SharedThreadPools.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/SharedThreadPools.cpp b/src/IO/SharedThreadPools.cpp index 4a0ef173669..6af5aab7a38 100644 --- a/src/IO/SharedThreadPools.cpp +++ b/src/IO/SharedThreadPools.cpp @@ -130,7 +130,7 @@ StaticThreadPool & getBackupsIOThreadPool() StaticThreadPool & getActivePartsLoadingThreadPool() { - static StaticThreadPool instance("MergeTreePartsLoaderThreadPool", CurrentMetrics::MergeTreePartsLoaderThreads, CurrentMetrics::MergeTreePartsLoaderThreadsActive, CurrentMetrics::MergeTreeOutdatedPartsLoaderThreadsScheduled); + static StaticThreadPool instance("MergeTreePartsLoaderThreadPool", CurrentMetrics::MergeTreePartsLoaderThreads, CurrentMetrics::MergeTreePartsLoaderThreadsActive, CurrentMetrics::MergeTreePartsLoaderThreadsScheduled); return instance; } From 75cebb3c29c54979218f0f0f80f46609851059b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 18 Nov 2023 20:20:21 +0100 Subject: [PATCH 534/813] Fix build --- src/Coordination/Standalone/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 4001673e01a..6942e866f23 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -263,7 +263,7 @@ ThreadPool & Context::getThreadPoolWriter() const auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); shared->threadpool_writer = std::make_unique( - CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size); + CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, CurrentMetrics::IOWriterThreadsScheduled, pool_size, pool_size, queue_size); }); return *shared->threadpool_writer; From 7c81d9b2b73ee35f8cf317397c2bd23dcd2336ea Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 18 Nov 2023 19:38:33 +0000 Subject: [PATCH 535/813] Fix test_keeper_auth --- tests/integration/test_keeper_auth/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_keeper_auth/test.py b/tests/integration/test_keeper_auth/test.py index e247984cc6a..78fbf84bbe2 100644 --- a/tests/integration/test_keeper_auth/test.py +++ b/tests/integration/test_keeper_auth/test.py @@ -1,6 +1,7 @@ import pytest import time from helpers.cluster import ClickHouseCluster +from helpers import keeper_utils from kazoo.client import KazooClient, KazooState from kazoo.security import ACL, make_digest_acl, make_acl from kazoo.exceptions import ( @@ -26,6 +27,7 @@ SUPERAUTH = "super:admin" def started_cluster(): try: cluster.start() + keeper_utils.wait_until_connected(cluster, node) yield cluster From a8abec6cacc8341927b6b06994f47bf5ff53884c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 18 Nov 2023 21:45:17 +0000 Subject: [PATCH 536/813] merge_row_policy: cleanup after merge --- src/Storages/StorageMerge.cpp | 67 +++++++++-------------------------- src/Storages/StorageMerge.h | 22 +++--------- 2 files changed, 21 insertions(+), 68 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0e088797100..10c98fe6ce8 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -380,7 +380,7 @@ void StorageMerge::read( query_plan.addStep(std::move(step)); } -/// A transient object of this helper class is created +/// An object of this helper class is created /// when processing a Merge table data source (subordinary table) /// that has row policies /// to guarantee that these row policies are applied @@ -392,16 +392,16 @@ public: /// Add to data stream columns that are needed only for row policies /// SELECT x from T if T has row policy y=42 /// required y in data pipeline - void extendNames(Names &); + void extendNames(Names &) const; /// Use storage facilities to filter data /// optimization /// does not guarantee accuracy, but reduces number of rows - void addStorageFilter(SourceStepWithFilter *); + void addStorageFilter(SourceStepWithFilter *) const; /// Create explicit filter transform to exclude /// rows that are not conform to row level policy - void addFilterTransform(QueryPipelineBuilder &); + void addFilterTransform(QueryPipelineBuilder &) const; private: std::string filter_column_name; // complex filter, may contain logic operations @@ -573,7 +573,8 @@ void ReadFromMerge::createChildPlans() Names column_names_as_aliases; Names real_column_names = column_names; - const auto & [database_name, _storage, _, table_name] = table; + const auto & database_name = std::get<0>(table); + const auto & table_name = std::get<3>(table); auto row_policy_filter_ptr = context->getRowPolicyFilter( database_name, table_name, @@ -584,7 +585,6 @@ void ReadFromMerge::createChildPlans() row_policy_data_opt->extendNames(real_column_names); } - if (!context->getSettingsRef().allow_experimental_analyzer) { auto storage_columns = storage_metadata_snapshot->getColumns(); @@ -645,7 +645,6 @@ void ReadFromMerge::createChildPlans() required_max_block_size, table, column_names_as_aliases.empty() ? std::move(real_column_names) : std::move(column_names_as_aliases), - // merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(), row_policy_data_opt, context, current_streams)); @@ -715,7 +714,6 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer return modified_query_info; } - bool recursivelyApplyToReadingSteps(QueryPlan::Node * node, const std::function & func) { bool ok = true; @@ -738,10 +736,10 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryPlan & plan, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & modified_query_info, - const QueryProcessingStage::Enum & processed_stage, + QueryProcessingStage::Enum processed_stage, const Block & header, const Aliases & aliases, - RowPolicyDataOpt & row_policy_data_opt, + const RowPolicyDataOpt & row_policy_data_opt, const StorageWithLockAndName & storage_with_lock, ContextMutablePtr modified_context, bool concat_streams) const @@ -822,37 +820,18 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( QueryPlan ReadFromMerge::createPlanForTable( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & modified_query_info, - const QueryProcessingStage::Enum & processed_stage, + QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, const StorageWithLockAndName & storage_with_lock, Names && real_column_names, - RowPolicyDataOpt & row_policy_data_opt, + const RowPolicyDataOpt & row_policy_data_opt, ContextMutablePtr modified_context, size_t streams_num) { const auto & [database_name, storage, _, table_name] = storage_with_lock; - // auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - // auto storage_snapshot = storage->getStorageSnapshot(storage_metadata_snapshot, context); - // auto modified_query_info = getModifiedQueryInfo(query_info, context, storage_with_lock, storage_snapshot); auto & modified_select = modified_query_info.query->as(); - // std::unique_ptr row_policy_data_ptr; - - // auto row_policy_filter_ptr = context->getRowPolicyFilter( - // database_name, - // table_name, - // RowPolicyFilterType::SELECT_FILTER); - // if (row_policy_filter_ptr) - // { - // row_policy_data_ptr = std::make_unique(row_policy_filter_ptr, storage, context); - // row_policy_data_ptr->extendNames(real_column_names); - // } - - // Aliases aliases; - // processAliases(real_column_names, storage_with_lock, aliases, sample_block, modified_context); - - // QueryPipelineBuilderPtr builder; if (!InterpreterSelectQuery::isQueryWithFinal(modified_query_info) && storage->needRewriteQueryWithFinal(real_column_names)) { /// NOTE: It may not work correctly in some cases, because query was analyzed without final. @@ -908,7 +887,6 @@ QueryPlan ReadFromMerge::createPlanForTable( if (!plan.isInitialized()) return {}; - /// move to applyFilters if (row_policy_data_opt) { if (auto * source_step_with_filter = dynamic_cast((plan.getRootNode()->step.get()))) @@ -917,16 +895,6 @@ QueryPlan ReadFromMerge::createPlanForTable( } } - // if (auto * read_from_merge_tree = typeid_cast(plan.getRootNode()->step.get())) - // { - // size_t filters_dags_size = filter_dags.size(); - // for (size_t i = 0; i < filters_dags_size; ++i) - // read_from_merge_tree->addFilter(filter_dags[i], filter_nodes.nodes[i]); - // } - - // builder = plan.buildQueryPipeline( - // QueryPlanOptimizationSettings::fromContext(modified_context), - // BuildQueryPipelineSettings::fromContext(modified_context)); applyFilters(plan); } else if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) @@ -964,14 +932,11 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter { storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_columns = storage_metadata_snapshot->getColumns(); - auto needed_columns = storage_columns.getAll/*Physical*/(); + auto needed_columns = storage_columns.getAll(); ASTPtr expr = row_policy_filter_ptr->expression; - auto syntax_result = TreeRewriter(local_context).analyze(expr, - needed_columns /*, - storage, - storage->getStorageSnapshot(storage_metadata_snapshot, local_context)*/); + auto syntax_result = TreeRewriter(local_context).analyze(expr, needed_columns); auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); @@ -992,7 +957,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter filter_column_name = added.getNames().front(); } -void ReadFromMerge::RowPolicyData::extendNames(Names & names) +void ReadFromMerge::RowPolicyData::extendNames(Names & names) const { boost::container::flat_set names_set(names.begin(), names.end()); NameSet added_names; @@ -1011,12 +976,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) } } -void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) +void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { step->addFilter(actions_dag, filter_column_name); } -void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & builder) +void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & builder) const { builder.addSimpleTransform([&](const Block & stream_header) { @@ -1198,7 +1163,7 @@ void ReadFromMerge::convertAndFilterSourceStream( const Block & header, const StorageMetadataPtr & metadata_snapshot, const Aliases & aliases, - RowPolicyDataOpt & row_policy_data_opt, + const RowPolicyDataOpt & row_policy_data_opt, ContextPtr local_context, QueryPipelineBuilder & builder, QueryProcessingStage::Enum processed_stage) diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index afdb9f8e13a..c6593c81fb7 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -181,20 +181,8 @@ private: using Aliases = std::vector; class RowPolicyData; - // using RowPolicyDataPtr = std::unique_ptr; using RowPolicyDataOpt = std::optional; - - /// Populates AliasData structures for further processing - /// using types from result query if possible - /// and removes alias columns from real_column_names - void processAliases( - Names & real_column_names, - const StorageWithLockAndName & storage_with_lock, - Aliases & aliases, - const Block & sample_block, - ContextMutablePtr modified_context); - std::vector table_aliases; std::vector table_row_policy_data_opts; @@ -206,11 +194,11 @@ private: QueryPlan createPlanForTable( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - const QueryProcessingStage::Enum & processed_stage, + QueryProcessingStage::Enum processed_stage, UInt64 max_block_size, const StorageWithLockAndName & storage_with_lock, Names && real_column_names, - RowPolicyDataOpt & row_policy_data_ptr, + const RowPolicyDataOpt & row_policy_data_ptr, ContextMutablePtr modified_context, size_t streams_num); @@ -218,10 +206,10 @@ private: QueryPlan & plan, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & modified_query_info, - const QueryProcessingStage::Enum & processed_stage, + QueryProcessingStage::Enum processed_stage, const Block & header, const Aliases & aliases, - RowPolicyDataOpt & row_policy_data_ptr, + const RowPolicyDataOpt & row_policy_data_ptr, const StorageWithLockAndName & storage_with_lock, ContextMutablePtr modified_context, bool concat_streams = false) const; @@ -235,7 +223,7 @@ private: const Block & header, const StorageMetadataPtr & metadata_snapshot, const Aliases & aliases, - RowPolicyDataOpt & row_policy_data_ptr, + const RowPolicyDataOpt & row_policy_data_ptr, ContextPtr context, QueryPipelineBuilder & builder, QueryProcessingStage::Enum processed_stage); From 053b20a255d33dca78c8260e6f49cd94ac2545c7 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 19 Nov 2023 00:44:39 +0100 Subject: [PATCH 537/813] fix in_data pointer --- src/IO/Lz4DeflatingWriteBuffer.cpp | 16 +++++----------- src/IO/Lz4DeflatingWriteBuffer.h | 3 --- 2 files changed, 5 insertions(+), 14 deletions(-) diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index e952e6400ec..2dbb7f684ed 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -43,9 +43,6 @@ namespace { tmp_out.finalize(); - if (cur_out == sink) - return; - sink->write(tmp_out.buffer().begin(), tmp_out.count()); } @@ -67,8 +64,6 @@ namespace ErrorCodes Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) : WriteBufferWithOwnMemoryDecorator(std::move(out_), buf_size, existing_memory, alignment) - , in_data(nullptr) - , in_capacity(0) , tmp_memory(buf_size) { @@ -106,9 +101,6 @@ void Lz4DeflatingWriteBuffer::nextImpl() if (!offset()) return; - in_data = reinterpret_cast(working_buffer.begin()); - in_capacity = offset(); - if (first_time) { auto sink = SinkToOut(out.get(), tmp_memory, LZ4F_HEADER_SIZE_MAX); @@ -128,7 +120,10 @@ void Lz4DeflatingWriteBuffer::nextImpl() first_time = false; } - do + auto in_data = working_buffer.begin(); + auto in_capacity = offset(); + + while (in_capacity > 0) { /// Ensure that there is enough space for compressed block of minimal size size_t min_compressed_block_size = LZ4F_compressBound(1, &kPrefs); @@ -154,11 +149,10 @@ void Lz4DeflatingWriteBuffer::nextImpl() LZ4F_VERSION, LZ4F_getErrorName(compressed_size), sink.getCapacity()); in_capacity -= cur_buffer_size; - in_data = reinterpret_cast(working_buffer.end() - in_capacity); + in_data += cur_buffer_size; sink.advancePosition(compressed_size); } - while (in_capacity > 0); } void Lz4DeflatingWriteBuffer::finalizeBefore() diff --git a/src/IO/Lz4DeflatingWriteBuffer.h b/src/IO/Lz4DeflatingWriteBuffer.h index 65f4f0c7349..7bb8a5e6c0e 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.h +++ b/src/IO/Lz4DeflatingWriteBuffer.h @@ -32,9 +32,6 @@ private: LZ4F_preferences_t kPrefs; /// NOLINT LZ4F_compressionContext_t ctx; - void * in_data; - size_t in_capacity; - Memory<> tmp_memory; bool first_time = true; From 89b93adf825c8098c0c5681340797c6940125f43 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Sat, 18 Nov 2023 23:55:46 +0000 Subject: [PATCH 538/813] Fix trailing whitespace --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 50d0aa67f0b..784694c98e3 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -547,10 +547,8 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) metadata.setConstraints(args.constraints); if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) - { throw Exception(ErrorCodes::BAD_ARGUMENTS, "MaterializedPostgreSQL is an experimental table engine." " You can enable it with the `allow_experimental_materialized_postgresql_table` setting"); - } if (!args.storage_def->order_by && args.storage_def->primary_key) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); From ddd9a23116ce5185c92879d17124215da846ec7b Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Sun, 19 Nov 2023 00:02:33 +0000 Subject: [PATCH 539/813] Fix newline --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 784694c98e3..ca91a7a9dec 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -545,10 +545,11 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); - + if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) throw Exception(ErrorCodes::BAD_ARGUMENTS, "MaterializedPostgreSQL is an experimental table engine." " You can enable it with the `allow_experimental_materialized_postgresql_table` setting"); + if (!args.storage_def->order_by && args.storage_def->primary_key) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); From 5f0f9b0ad42ace358586d9b761b83c7198d01c76 Mon Sep 17 00:00:00 2001 From: melvynator Date: Sun, 19 Nov 2023 01:05:11 +0100 Subject: [PATCH 540/813] Test check style --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index ca91a7a9dec..36de3936321 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -546,7 +546,7 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); - if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) + if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) throw Exception(ErrorCodes::BAD_ARGUMENTS, "MaterializedPostgreSQL is an experimental table engine." " You can enable it with the `allow_experimental_materialized_postgresql_table` setting"); From 88b05253060fead93d1fd8ad7e2c1a139c03c9e7 Mon Sep 17 00:00:00 2001 From: melvynator Date: Sun, 19 Nov 2023 01:10:25 +0100 Subject: [PATCH 541/813] Style check --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 36de3936321..bf84617760f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -545,8 +545,8 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); - - if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) + + if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) throw Exception(ErrorCodes::BAD_ARGUMENTS, "MaterializedPostgreSQL is an experimental table engine." " You can enable it with the `allow_experimental_materialized_postgresql_table` setting"); From 3ade60a26183b1c2cc7ad64d080f41811a7028e8 Mon Sep 17 00:00:00 2001 From: melvynator Date: Sun, 19 Nov 2023 01:31:21 +0100 Subject: [PATCH 542/813] Keep trying to fix style --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index bf84617760f..96247e9e1a7 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -545,7 +545,7 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); - + if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) throw Exception(ErrorCodes::BAD_ARGUMENTS, "MaterializedPostgreSQL is an experimental table engine." " You can enable it with the `allow_experimental_materialized_postgresql_table` setting"); From a894671e8af10d4b5b70d79beccdba6be2e12174 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Sat, 18 Nov 2023 17:53:32 -0800 Subject: [PATCH 543/813] [Docs] Add perf tip for COUNT(DISTINCT expr) --- docs/en/sql-reference/aggregate-functions/reference/count.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/count.md b/docs/en/sql-reference/aggregate-functions/reference/count.md index a98c8e50174..a40108a331a 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/count.md +++ b/docs/en/sql-reference/aggregate-functions/reference/count.md @@ -34,6 +34,10 @@ The `SELECT count() FROM table` query is optimized by default using metadata fro However `SELECT count(nullable_column) FROM table` query can be optimized by enabling the [optimize_functions_to_subcolumns](../../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [null](../../../sql-reference/data-types/nullable.md#finding-null) subcolumn instead of reading and processing the whole column data. The query `SELECT count(n) FROM table` transforms to `SELECT sum(NOT n.null) FROM table`. +**Improving COUNT(DISTINCT expr) performance** + +If your `COUNT(DISTINCT expr)` query is slow, consider adding a [`GROUP BY`](../../../sql-reference/statements/select/group-by.md) clause as this improves parallelization. You can also use a [projection](../../../sql-reference/statements/alter/projection.md) to create an index on the target column used with `COUNT(DISTINCT target_col)`. + **Examples** Example 1: From 96e87322b525351b7d241c9e2c6ca125d2178b20 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Sat, 18 Nov 2023 18:10:43 -0800 Subject: [PATCH 544/813] [Docs] Add insert_distributed_sync to Core Settings docs --- 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 a22bd6e33e5..e61934d2168 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2716,6 +2716,10 @@ Default value: `0`. - [Distributed Table Engine](../../engines/table-engines/special/distributed.md/#distributed) - [Managing Distributed Tables](../../sql-reference/statements/system.md/#query-language-system-distributed) +## insert_distributed_sync {#insert_distributed_sync} + +Alias for [`distributed_foreground_insert`](#distributed_foreground_insert). + ## insert_shard_id {#insert_shard_id} If not `0`, specifies the shard of [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table into which the data will be inserted synchronously. From 5e1da38720d16672bba07da1b9b54fc081b1464e Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Sat, 18 Nov 2023 18:28:29 -0800 Subject: [PATCH 545/813] [Docs] Add details on why partitions improve query perf --- .../table-engines/mergetree-family/custom-partitioning-key.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index 7e564b23676..97d37e476ae 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -14,7 +14,7 @@ You should never use too granular of partitioning. Don't partition your data by Partitioning is available for the [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) family tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) tables). [Materialized views](../../../engines/table-engines/special/materializedview.md#materializedview) based on MergeTree tables support partitioning, as well. -A partition is a logical combination of records in a table by a specified criterion. You can set a partition by an arbitrary criterion, such as by month, by day, or by event type. Each partition is stored separately to simplify manipulations of this data. When accessing the data, ClickHouse uses the smallest subset of partitions possible. +A partition is a logical combination of records in a table by a specified criterion. You can set a partition by an arbitrary criterion, such as by month, by day, or by event type. Each partition is stored separately to simplify manipulations of this data. When accessing the data, ClickHouse uses the smallest subset of partitions possible. Partitions improve performance for queries containing a partitioning key because ClickHouse will filter for that partition before selecting the parts and granules within the partition. The partition is specified in the `PARTITION BY expr` clause when [creating a table](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table). The partition key can be any expression from the table columns. For example, to specify partitioning by month, use the expression `toYYYYMM(date_column)`: From 8062fb578c7b6c65d4d24efa28bb47e8c8783ae8 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Sat, 18 Nov 2023 18:38:49 -0800 Subject: [PATCH 546/813] [Docs] Recommend ReplacingMergeTree for frequent updates --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 +++- 1 file changed, 3 insertions(+), 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 e615c9ad9d3..d250cfd1a08 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -6,7 +6,7 @@ sidebar_label: MergeTree # MergeTree -The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most robust ClickHouse table engines. +The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most common and most robust ClickHouse table engines. 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. @@ -32,6 +32,8 @@ Main features: The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family. ::: +If you need to update rows frequently, we recommend using the [`ReplacingMergeTree`](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md). Using `ALTER TALBE 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} ``` sql From a398e3f51e499c5523e62ac4c296b0203304e672 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Sat, 18 Nov 2023 18:40:14 -0800 Subject: [PATCH 547/813] [Docs] Fix typo --- 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 d250cfd1a08..810b7596150 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -6,7 +6,7 @@ sidebar_label: MergeTree # MergeTree -The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most common and most robust ClickHouse table engines. +The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most commonly used and most robust ClickHouse table engines. 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. From f29777db2d77aee2d17034e6844377f65e5ddfe7 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Sat, 18 Nov 2023 18:41:20 -0800 Subject: [PATCH 548/813] [Docs] More typos --- 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 810b7596150..f0bc45b9f53 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -32,7 +32,7 @@ Main features: The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family. ::: -If you need to update rows frequently, we recommend using the [`ReplacingMergeTree`](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md). Using `ALTER TALBE 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. +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} From 9d3c62ec418802371fcf7f466163d683c1ec262a Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Sat, 18 Nov 2023 18:51:35 -0800 Subject: [PATCH 549/813] [Docs] Recommend against OPTIMIZE FINAL in OPTIMIZE page --- docs/en/sql-reference/statements/optimize.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 49843eaff9a..07b5a196096 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -5,7 +5,7 @@ sidebar_label: OPTIMIZE title: "OPTIMIZE Statement" --- -This query tries to initialize an unscheduled merge of data parts for tables. +This query tries to initialize an unscheduled merge of data parts for tables. Note that we generally recommend against using `OPTIMIZE TABLE ... FINAL` (see these [docs](/docs/en/optimize/avoidoptimizefinal)) as its use case is meant for administration, not for daily operations. :::note `OPTIMIZE` can’t fix the `Too many parts` error. From 097f80657c35c83ca522b412a040dfeaa36011d5 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sun, 19 Nov 2023 11:14:24 +0100 Subject: [PATCH 550/813] Fewer concurrent requests in 02908_many_requests_to_system_replicas --- .../02908_many_requests_to_system_replicas.reference | 2 +- .../0_stateless/02908_many_requests_to_system_replicas.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference index d7850e59dec..af0e50ec332 100644 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.reference @@ -1,5 +1,5 @@ Creating 300 tables -Making making 500 requests to system.replicas +Making making 200 requests to system.replicas Query system.replicas while waiting for other concurrent requests to finish 0 900 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 c620fcf4bea..f93175529c0 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 @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e NUM_TABLES=300 -CONCURRENCY=500 +CONCURRENCY=200 echo "Creating $NUM_TABLES tables" From 09f263e281ef4fe7336ee0f0f59a3785446955ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Nov 2023 13:20:01 +0300 Subject: [PATCH 551/813] Update fetchPostgreSQLTableStructure.cpp --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index e9ffdebc583..dec3f1ffe5a 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -263,7 +263,8 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "attnotnull AS not_null, attndims AS dims, atttypid as type_id, atttypmod as type_modifier " "FROM pg_attribute " "WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) " - "AND NOT attisdropped AND attnum > 0 order by attnum asc", where); + "AND NOT attisdropped AND attnum > 0 " + "ORDER BY attnum ASC", where); auto postgres_table_with_schema = postgres_schema.empty() ? postgres_table : doubleQuoteString(postgres_schema) + '.' + doubleQuoteString(postgres_table); table.physical_columns = readNamesAndTypesList(tx, postgres_table_with_schema, query, use_nulls, false); From 24fbe620d32ef624ed0f6003c0d9650b03086544 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 19 Nov 2023 12:14:53 +0100 Subject: [PATCH 552/813] fix build --- src/IO/Lz4DeflatingWriteBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 2dbb7f684ed..7def2da104f 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -120,7 +120,7 @@ void Lz4DeflatingWriteBuffer::nextImpl() first_time = false; } - auto in_data = working_buffer.begin(); + auto * in_data = working_buffer.begin(); auto in_capacity = offset(); while (in_capacity > 0) From cacc23b8b745f15eb6d84db002987b6229010932 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 19 Nov 2023 12:25:42 +0100 Subject: [PATCH 553/813] safe SinkToOut d-tor --- src/IO/Lz4DeflatingWriteBuffer.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 7def2da104f..0af205a426d 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -39,10 +39,9 @@ namespace cur_out->position() += size; } - ~SinkToOut() noexcept(false) + void finalize() { tmp_out.finalize(); - sink->write(tmp_out.buffer().begin(), tmp_out.count()); } @@ -117,6 +116,7 @@ void Lz4DeflatingWriteBuffer::nextImpl() LZ4F_VERSION, LZ4F_getErrorName(header_size)); sink.advancePosition(header_size); + sink.finalize(); first_time = false; } @@ -152,6 +152,7 @@ void Lz4DeflatingWriteBuffer::nextImpl() in_data += cur_buffer_size; sink.advancePosition(compressed_size); + sink.finalize(); } } @@ -173,6 +174,7 @@ void Lz4DeflatingWriteBuffer::finalizeBefore() LZ4F_VERSION, LZ4F_getErrorName(end_size), sink.getCapacity()); sink.advancePosition(end_size); + sink.finalize(); } void Lz4DeflatingWriteBuffer::finalizeAfter() From f27018c14196e938aa3b16cae387e46d2578501a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Nov 2023 15:21:11 +0100 Subject: [PATCH 554/813] Own CMake for GRPC --- cmake/limit_jobs.cmake | 4 +- contrib/grpc-cmake/CMakeLists.txt | 40 +- contrib/grpc-cmake/grpc.cmake | 1864 +++++++++++++++++++++++++++++ 3 files changed, 1868 insertions(+), 40 deletions(-) create mode 100644 contrib/grpc-cmake/grpc.cmake diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 28ccb62e10c..a43e208ff0d 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -21,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(WARNING "The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + message(INFO "The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -32,7 +32,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(WARNING "The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + message(INFO "The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 09ed2fe3f80..b8b5f5580c4 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -9,50 +9,14 @@ endif() set(_gRPC_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/grpc") set(_gRPC_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/grpc") -# Use re2 from ClickHouse contrib, not from gRPC third_party. -set(gRPC_RE2_PROVIDER "clickhouse" CACHE STRING "" FORCE) -set(_gRPC_RE2_INCLUDE_DIR "") -set(_gRPC_RE2_LIBRARIES ch_contrib::re2) - -# Use zlib from ClickHouse contrib, not from gRPC third_party. -set(gRPC_ZLIB_PROVIDER "clickhouse" CACHE STRING "" FORCE) -set(_gRPC_ZLIB_INCLUDE_DIR "") -set(_gRPC_ZLIB_LIBRARIES ch_contrib::zlib) - -# Use protobuf from ClickHouse contrib, not from gRPC third_party. -set(gRPC_PROTOBUF_PROVIDER "clickhouse" CACHE STRING "" FORCE) -set(_gRPC_PROTOBUF_LIBRARIES ch_contrib::protobuf) -set(_gRPC_PROTOBUF_PROTOC "protoc") -set(_gRPC_PROTOBUF_PROTOC_EXECUTABLE $) -set(_gRPC_PROTOBUF_PROTOC_LIBRARIES ch_contrib::protoc) - if(TARGET OpenSSL::SSL) set(gRPC_USE_UNSECURE_LIBRARIES FALSE) else() set(gRPC_USE_UNSECURE_LIBRARIES TRUE) endif() -# Use OpenSSL from ClickHouse contrib, not from gRPC third_party. -set(gRPC_SSL_PROVIDER "clickhouse" CACHE STRING "" FORCE) -set(_gRPC_SSL_INCLUDE_DIR "") -set(_gRPC_SSL_LIBRARIES OpenSSL::Crypto OpenSSL::SSL) - -# Use abseil-cpp from ClickHouse contrib, not from gRPC third_party. -set(gRPC_ABSL_PROVIDER "clickhouse" CACHE STRING "" FORCE) - -# We don't want to build C# extensions. -set(gRPC_BUILD_CSHARP_EXT OFF) - -# TODO: Remove this. We generally like to compile with C++23 but grpc isn't ready yet. -set (CMAKE_CXX_STANDARD 20) - -set(_gRPC_CARES_LIBRARIES ch_contrib::c-ares) -set(gRPC_CARES_PROVIDER "clickhouse" CACHE STRING "" FORCE) -add_subdirectory("${_gRPC_SOURCE_DIR}" "${_gRPC_BINARY_DIR}") - -# The contrib/grpc/CMakeLists.txt redefined the PROTOBUF_GENERATE_GRPC_CPP() function for its own purposes, -# so we need to redefine it back. -include("${ClickHouse_SOURCE_DIR}/contrib/grpc-cmake/protobuf_generate_grpc.cmake") +include(grpc.cmake) +include(protobuf_generate_grpc.cmake) set(gRPC_CPP_PLUGIN $) set(gRPC_PYTHON_PLUGIN $) diff --git a/contrib/grpc-cmake/grpc.cmake b/contrib/grpc-cmake/grpc.cmake new file mode 100644 index 00000000000..43d4edd191e --- /dev/null +++ b/contrib/grpc-cmake/grpc.cmake @@ -0,0 +1,1864 @@ +# This file was edited for ClickHouse. + +# GRPC global cmake file +# This currently builds C and C++ code. +# This file has been automatically generated from a template file. +# Please look at the templates directory instead. +# This file can be regenerated from the template by running +# tools/buildgen/generate_projects.sh +# +# Copyright 2015 gRPC authors. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# We want to use C++23, but GRPC is not ready +set (CMAKE_CXX_STANDARD 20) + +set(_gRPC_ZLIB_INCLUDE_DIR "") +set(_gRPC_ZLIB_LIBRARIES ch_contrib::zlib) + +set(_gRPC_CARES_LIBRARIES ch_contrib::c-ares) + +set(_gRPC_RE2_INCLUDE_DIR "") +set(_gRPC_RE2_LIBRARIES ch_contrib::re2) + +set(_gRPC_SSL_INCLUDE_DIR "") +set(_gRPC_SSL_LIBRARIES OpenSSL::Crypto OpenSSL::SSL) + +set(_gRPC_PROTOBUF_LIBRARIES ch_contrib::protobuf) +set(_gRPC_PROTOBUF_PROTOC "protoc") +set(_gRPC_PROTOBUF_PROTOC_EXECUTABLE $) +set(_gRPC_PROTOBUF_PROTOC_LIBRARIES ch_contrib::protoc) + + +if(UNIX) + if(${CMAKE_SYSTEM_NAME} MATCHES "Linux") + set(_gRPC_PLATFORM_LINUX ON) + elseif(${CMAKE_SYSTEM_NAME} MATCHES "Darwin") + set(_gRPC_PLATFORM_MAC ON) + elseif(${CMAKE_SYSTEM_NAME} MATCHES "iOS") + set(_gRPC_PLATFORM_IOS ON) + elseif(${CMAKE_SYSTEM_NAME} MATCHES "Android") + set(_gRPC_PLATFORM_ANDROID ON) + else() + set(_gRPC_PLATFORM_POSIX ON) + endif() +endif() + +if(UNIX AND NOT HAIKU) + # -pthread does more than -lpthread + set(THREADS_PREFER_PTHREAD_FLAG ON) + find_package(Threads) + set(_gRPC_ALLTARGETS_LIBRARIES ${CMAKE_DL_LIBS} Threads::Threads) + if(_gRPC_PLATFORM_LINUX OR _gRPC_PLATFORM_POSIX) + set(_gRPC_ALLTARGETS_LIBRARIES ${_gRPC_ALLTARGETS_LIBRARIES} rt) + endif() +endif() + +set(_gRPC_ADDRESS_SORTING_INCLUDE_DIR "${_gRPC_SOURCE_DIR}/third_party/address_sorting/include") +set(_gRPC_ADDRESS_SORTING_LIBRARIES address_sorting) + +set(UPB_ROOT_DIR ${_gRPC_SOURCE_DIR}/third_party/upb) + +set(_gRPC_UPB_INCLUDE_DIR "${UPB_ROOT_DIR}" "${_gRPC_SOURCE_DIR}/third_party/utf8_range") +set(_gRPC_UPB_GRPC_GENERATED_DIR "${_gRPC_SOURCE_DIR}/src//core/ext/upb-generated" "${_gRPC_SOURCE_DIR}/src//core/ext/upbdefs-generated") + +set(_gRPC_UPB_LIBRARIES upb) + +set(_gRPC_XXHASH_INCLUDE_DIR "${_gRPC_SOURCE_DIR}/third_party/xxhash") + +add_library(address_sorting + ${_gRPC_SOURCE_DIR}/third_party/address_sorting/address_sorting.c + ${_gRPC_SOURCE_DIR}/third_party/address_sorting/address_sorting_posix.c + ${_gRPC_SOURCE_DIR}/third_party/address_sorting/address_sorting_windows.c +) + +target_compile_features(address_sorting PUBLIC cxx_std_14) + +target_include_directories(address_sorting + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(address_sorting + ${_gRPC_ALLTARGETS_LIBRARIES} +) + + +add_library(gpr + ${_gRPC_SOURCE_DIR}/src/core/lib/config/config_vars.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/config/config_vars_non_generated.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/config/load_config.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thread_local.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/alloc.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/android/log.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/atm.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/iphone/cpu.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/linux/cpu.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/linux/log.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/log.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/msys/tmpfile.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/posix/cpu.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/posix/log.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/posix/string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/posix/sync.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/posix/time.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/posix/tmpfile.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/sync.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/sync_abseil.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/time.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/time_precise.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/windows/cpu.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/windows/log.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/windows/string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/windows/string_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/windows/sync.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/windows/time.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/windows/tmpfile.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gpr/wrap_memcpy.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/crash.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/examine_stack.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/fork.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/host_port.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/linux/env.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/mpscq.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/posix/env.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/posix/stat.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/posix/thd.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/strerror.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/tchar.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/time_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/windows/env.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/windows/stat.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/windows/thd.cc +) + +target_compile_features(gpr PUBLIC cxx_std_14) + +target_include_directories(gpr + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(gpr + ${_gRPC_ALLTARGETS_LIBRARIES} + absl::base + absl::core_headers + absl::flags + absl::flags_marshalling + absl::any_invocable + absl::memory + absl::random_random + absl::status + absl::cord + absl::str_format + absl::strings + absl::synchronization + absl::time + absl::optional + absl::variant +) +if(_gRPC_PLATFORM_ANDROID) + target_link_libraries(gpr + android + log + ) +endif() + + +add_library(grpc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/backend_metrics/backend_metric_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/census/grpc_context.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/channel_idle/channel_idle_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/channel_idle/idle_filter_state.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/backend_metric.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/backup_poller.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/channel_connectivity.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_channelz.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_service_config.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/config_selector.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/dynamic_filters.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/global_subchannel_pool.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/http_proxy_mapper.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/address_filtering.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/child_policy_handler.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/endpoint_list.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/client_load_reporting_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_balancer_addresses.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_client_stats.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/health_check_client.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/oob_backend_metric.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/outlier_detection/outlier_detection.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/priority/priority.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/ring_hash/ring_hash.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/rls/rls.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/weighted_round_robin/static_stride_scheduler.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/weighted_round_robin/weighted_round_robin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/weighted_target/weighted_target.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/xds/cds.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/xds/xds_cluster_impl.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/xds/xds_cluster_manager.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/xds/xds_cluster_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/xds/xds_override_host.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/xds/xds_wrr_locality.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/local_subchannel_pool.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/binder/binder_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/dns_resolver_ares.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/dns_resolver_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/event_engine/event_engine_client_channel_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/event_engine/service_config_helper.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/native/dns_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/fake/fake_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/google_c2p/google_c2p_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/polling_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/sockaddr/sockaddr_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/xds/xds_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_filter_legacy_call_data.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_service_config.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_throttle.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/service_config_channel_arg_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel_pool_interface.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel_stream_client.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/deadline/deadline_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/fault_injection/fault_injection_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/fault_injection/fault_injection_service_config_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/client/http_client_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/client_authority_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/http_filters_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/message_compress/compression_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/server/http_server_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/message_size/message_size_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/rbac/rbac_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/rbac/rbac_service_config_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/server_config_selector/server_config_selector_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/stateful_session/stateful_session_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/stateful_session/stateful_session_service_config_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/gcp/metadata_query.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/alpn/alpn.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/chttp2_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/chttp2_server.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/bin_decoder.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/bin_encoder.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/chttp2_transport.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/decode_huff.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/flow_control.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_data.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_goaway.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_ping.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_rst_stream.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_settings.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_window_update.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_encoder.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_encoder_table.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_parse_result.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_parser_table.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/http2_settings.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/http_trace.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/huffsyms.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/max_concurrent_streams_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/parsing.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/ping_abuse_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/ping_callbacks.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/ping_rate_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/stream_lists.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/varint.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/write_size_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/writing.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/inproc/inproc_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/inproc/inproc_transport.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/certs.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/clusters.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/config_dump.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/config_dump_shared.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/init_dump.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/listeners.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/memory.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/metrics.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/mutex_stats.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/server_info.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/admin/v3/tap.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/annotations/deprecation.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/annotations/resource.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/accesslog/v3/accesslog.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/bootstrap/v3/bootstrap.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/cluster/v3/circuit_breaker.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/cluster/v3/cluster.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/cluster/v3/filter.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/cluster/v3/outlier_detection.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/common/matcher/v3/matcher.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/address.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/backoff.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/base.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/config_source.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/event_service_config.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/extension.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/grpc_method_list.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/grpc_service.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/health_check.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/http_uri.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/protocol.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/proxy_protocol.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/resolver.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/socket_option.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/substitution_format_string.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/core/v3/udp_socket_config.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/endpoint/v3/endpoint.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/endpoint/v3/endpoint_components.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/endpoint/v3/load_report.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/listener/v3/api_listener.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/listener/v3/listener.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/listener/v3/listener_components.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/listener/v3/quic_config.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/listener/v3/udp_listener_config.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/metrics/v3/metrics_service.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/metrics/v3/stats.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/overload/v3/overload.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/rbac/v3/rbac.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/route/v3/route.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/route/v3/route_components.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/route/v3/scoped_route.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/tap/v3/common.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/datadog.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/dynamic_ot.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/http_tracer.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/lightstep.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/opencensus.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/opentelemetry.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/service.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/skywalking.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/trace.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/xray.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/config/trace/v3/zipkin.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/data/accesslog/v3/accesslog.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/clusters/aggregate/v3/cluster.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/filters/common/fault/v3/fault.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/filters/http/fault/v3/fault.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/filters/http/rbac/v3/rbac.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/filters/http/router/v3/router.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/filters/http/stateful_session/v3/stateful_session.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/filters/network/http_connection_manager/v3/http_connection_manager.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/http/stateful_session/cookie/v3/cookie.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/load_balancing_policies/client_side_weighted_round_robin/v3/client_side_weighted_round_robin.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/load_balancing_policies/common/v3/common.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/load_balancing_policies/pick_first/v3/pick_first.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/load_balancing_policies/ring_hash/v3/ring_hash.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/load_balancing_policies/wrr_locality/v3/wrr_locality.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/transport_sockets/tls/v3/cert.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/transport_sockets/tls/v3/common.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/transport_sockets/tls/v3/secret.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/transport_sockets/tls/v3/tls.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/extensions/transport_sockets/tls/v3/tls_spiffe_validator_config.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/service/discovery/v3/ads.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/service/discovery/v3/discovery.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/service/load_stats/v3/lrs.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/service/status/v3/csds.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/http/v3/cookie.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/http/v3/path_transformation.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/filter_state.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/http_inputs.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/metadata.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/node.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/number.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/path.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/regex.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/status_code_input.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/string.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/struct.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/matcher/v3/value.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/metadata/v3/metadata.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/tracing/v3/custom_tag.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/hash_policy.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/http.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/http_status.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/percent.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/range.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/ratelimit_strategy.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/ratelimit_unit.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/semantic_version.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/v3/token_bucket.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/annotations.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/expr/v1alpha1/checked.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/expr/v1alpha1/syntax.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/http.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/httpbody.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/any.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/descriptor.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/duration.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/empty.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/struct.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/timestamp.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/wrappers.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/rpc/status.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/opencensus/proto/trace/v1/trace_config.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/altscontext.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/handshaker.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/transport_security_common.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/health/v1/health.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/lb/v1/load_balancer.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/lookup/v1/rls.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/lookup/v1/rls_config.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/udpa/annotations/migrate.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/udpa/annotations/security.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/udpa/annotations/sensitive.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/udpa/annotations/status.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/udpa/annotations/versioning.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/validate/validate.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/annotations/v3/migrate.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/annotations/v3/security.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/annotations/v3/sensitive.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/annotations/v3/status.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/annotations/v3/versioning.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/core/v3/authority.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/core/v3/cidr.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/core/v3/collection_entry.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/core/v3/context_params.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/core/v3/extension.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/core/v3/resource.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/core/v3/resource_locator.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/core/v3/resource_name.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/data/orca/v3/orca_load_report.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/service/orca/v3/orca.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/matcher/v3/cel.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/matcher/v3/domain.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/matcher/v3/http_inputs.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/matcher/v3/ip.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/matcher/v3/matcher.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/matcher/v3/range.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/matcher/v3/regex.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/matcher/v3/string.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/v3/cel.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/v3/range.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/type/v3/typed_struct.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/certs.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/clusters.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/config_dump.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/config_dump_shared.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/init_dump.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/listeners.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/memory.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/metrics.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/mutex_stats.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/server_info.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/admin/v3/tap.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/annotations/deprecation.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/annotations/resource.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/accesslog/v3/accesslog.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/bootstrap/v3/bootstrap.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/cluster/v3/circuit_breaker.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/cluster/v3/cluster.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/cluster/v3/filter.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/cluster/v3/outlier_detection.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/common/matcher/v3/matcher.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/address.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/backoff.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/base.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/config_source.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/event_service_config.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/extension.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/grpc_method_list.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/grpc_service.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/health_check.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/http_uri.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/protocol.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/proxy_protocol.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/resolver.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/socket_option.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/substitution_format_string.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/core/v3/udp_socket_config.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/endpoint/v3/endpoint.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/endpoint/v3/endpoint_components.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/endpoint/v3/load_report.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/listener/v3/api_listener.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/listener/v3/listener.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/listener/v3/listener_components.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/listener/v3/quic_config.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/listener/v3/udp_listener_config.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/metrics/v3/metrics_service.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/metrics/v3/stats.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/overload/v3/overload.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/rbac/v3/rbac.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/route/v3/route.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/route/v3/route_components.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/route/v3/scoped_route.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/tap/v3/common.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/datadog.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/dynamic_ot.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/http_tracer.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/lightstep.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/opencensus.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/opentelemetry.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/service.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/skywalking.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/trace.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/xray.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/config/trace/v3/zipkin.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/data/accesslog/v3/accesslog.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/clusters/aggregate/v3/cluster.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/filters/common/fault/v3/fault.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/filters/http/fault/v3/fault.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/filters/http/rbac/v3/rbac.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/filters/http/router/v3/router.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/filters/http/stateful_session/v3/stateful_session.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/filters/network/http_connection_manager/v3/http_connection_manager.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/http/stateful_session/cookie/v3/cookie.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/transport_sockets/tls/v3/cert.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/transport_sockets/tls/v3/common.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/transport_sockets/tls/v3/secret.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/transport_sockets/tls/v3/tls.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/extensions/transport_sockets/tls/v3/tls_spiffe_validator_config.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/service/discovery/v3/ads.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/service/discovery/v3/discovery.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/service/load_stats/v3/lrs.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/service/status/v3/csds.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/http/v3/cookie.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/http/v3/path_transformation.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/filter_state.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/http_inputs.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/metadata.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/node.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/number.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/path.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/regex.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/status_code_input.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/string.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/struct.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/matcher/v3/value.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/metadata/v3/metadata.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/tracing/v3/custom_tag.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/hash_policy.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/http.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/http_status.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/percent.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/range.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/ratelimit_strategy.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/ratelimit_unit.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/semantic_version.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/envoy/type/v3/token_bucket.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/api/annotations.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/api/expr/v1alpha1/checked.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/api/expr/v1alpha1/syntax.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/api/http.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/api/httpbody.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/protobuf/any.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/protobuf/descriptor.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/protobuf/duration.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/protobuf/empty.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/protobuf/struct.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/protobuf/timestamp.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/protobuf/wrappers.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/google/rpc/status.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/opencensus/proto/trace/v1/trace_config.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/src/proto/grpc/lookup/v1/rls_config.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/udpa/annotations/migrate.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/udpa/annotations/security.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/udpa/annotations/sensitive.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/udpa/annotations/status.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/udpa/annotations/versioning.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/validate/validate.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/annotations/v3/migrate.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/annotations/v3/security.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/annotations/v3/sensitive.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/annotations/v3/status.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/annotations/v3/versioning.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/core/v3/authority.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/core/v3/cidr.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/core/v3/collection_entry.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/core/v3/context_params.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/core/v3/extension.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/core/v3/resource.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/core/v3/resource_locator.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/core/v3/resource_name.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/matcher/v3/cel.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/matcher/v3/domain.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/matcher/v3/http_inputs.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/matcher/v3/ip.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/matcher/v3/matcher.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/matcher/v3/range.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/matcher/v3/regex.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/matcher/v3/string.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/v3/cel.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/v3/range.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upbdefs-generated/xds/type/v3/typed_struct.upbdefs.c + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/certificate_provider_store.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/file_watcher_certificate_provider_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_api.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_audit_logger_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_bootstrap.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_bootstrap_grpc.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_certificate_provider.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_channel_stack_modifier.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_client.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_client_grpc.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_client_stats.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_cluster.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_cluster_specifier_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_common_types.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_health_status.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_http_fault_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_http_filters.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_http_rbac_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_http_stateful_session_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_lb_policy_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_listener.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_route_config.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_routing.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_server_config_fetcher.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/xds/xds_transport_grpc.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/address_utils/parse_address.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/address_utils/sockaddr_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/backoff/backoff.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/backoff/random_early_detection.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/call_tracer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_args.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_args_preconditioning.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack_builder.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack_builder_impl.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channelz.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channelz_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/connected_channel.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/promise_based_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/server_call_tracer_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/status_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/compression/compression.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/compression/compression_internal.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/compression/message_compress.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/config/core_configuration.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/event_log.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/histogram_view.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/stats.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/stats_data.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/ares_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/cf_engine/cf_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/cf_engine/cfstream_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/cf_engine/dns_service_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/channel_args_endpoint_config.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/default_event_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/default_event_engine_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/event_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/forkable.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/memory_allocator.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/ev_epoll1_linux.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/ev_poll_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/event_poller_posix_default.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/internal_errqueue.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/lockfree_event.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/posix_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/posix_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/posix_engine_listener.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/posix_engine_listener_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/tcp_socket_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/timer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/timer_heap.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/timer_manager.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/traced_buffer_list.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/wakeup_fd_eventfd.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/wakeup_fd_pipe.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/wakeup_fd_posix_default.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/resolved_address.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/shim.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/slice.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/slice_buffer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/tcp_socket_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thread_pool/thread_count.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thread_pool/thread_pool_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thread_pool/work_stealing_thread_pool.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thready_event_engine/thready_event_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/time_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/iocp.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/win_socket.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/windows_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/windows_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/windows_listener.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/work_queue/basic_work_queue.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/experiments/config.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/experiments/experiments.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/load_file.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/per_cpu.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/ref_counted_string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/status_helper.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/time.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/time_averaged_stats.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/validation_errors.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/work_serializer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/handshaker/proxy_mapper_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/http/format_request.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/http/httpcli.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/http/httpcli_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/http/parser.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/buffer_list.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/call_combiner.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/cfstream_handle.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/closure.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/combiner.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/dualstack_socket_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_cfstream.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/error.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/error_cfstream.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_apple.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_epoll1_linux.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_poll_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/event_engine_shims/closure.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/event_engine_shims/endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/event_engine_shims/tcp_client.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/exec_ctx.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/executor.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/fork_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/fork_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_fallback.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_host_name_max.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_sysconf.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/grpc_if_nametoindex_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/grpc_if_nametoindex_unsupported.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/internal_errqueue.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iocp_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_internal.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_posix_cfstream.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/load_file.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/lockfree_event.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/polling_entity.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/sockaddr_utils_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_factory_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_mutator.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_common_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_linux.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/systemd_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_cfstream.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_common.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_ifaddrs.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_noifaddrs.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/timer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_generic.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_heap.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_manager.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/unix_sockets_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/unix_sockets_posix_noop.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/vsock.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_eventfd.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_nospecial.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_pipe.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/json/json_object_loader.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/json/json_reader.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/json/json_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/json/json_writer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/load_balancing/lb_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/load_balancing/lb_policy_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/matchers/matchers.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/promise/activity.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/promise/party.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/promise/sleep.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/promise/trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resolver/resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resolver/resolver_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resolver/server_address.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/api.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/arena.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/memory_quota.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/periodic_update.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/resource_quota.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/thread_quota.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/audit_logging.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/authorization_policy_provider_vtable.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/evaluate_args.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/grpc_authorization_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/grpc_server_authz_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/matchers.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/rbac_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/stdout_logger.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/certificate_provider/certificate_provider_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/context/security_context.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/alts_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_linux.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_no_op.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_client_options.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_options.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_server_options.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/call_creds_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/channel_creds_registry_init.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/composite/composite_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/external/aws_external_account_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/external/aws_request_signer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/external/external_account_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/external/file_external_account_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/external/url_external_account_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/fake/fake_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/google_default/credentials_generic.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/google_default/google_default_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/iam/iam_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/insecure/insecure_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/jwt/json_token.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/jwt/jwt_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/jwt/jwt_verifier.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/local/local_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/oauth2/oauth2_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/plugin/plugin_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/ssl/ssl_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/grpc_tls_certificate_distributor.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/grpc_tls_certificate_match.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/grpc_tls_certificate_provider.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/grpc_tls_certificate_verifier.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/grpc_tls_credentials_options.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/tls_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/tls_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/xds/xds_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/alts/alts_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/fake/fake_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/insecure/insecure_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/load_system_roots_fallback.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/load_system_roots_supported.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/local/local_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/ssl/ssl_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/ssl_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/tls/tls_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/client_auth_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/secure_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/security_handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/server_auth_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/tsi_error.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/util/json_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/service_config/service_config_impl.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/service_config/service_config_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/b64.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/percent_encoding.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/slice.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/slice_buffer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/slice_refcount.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/slice_string_helpers.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/api_trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/builtins.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/byte_buffer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/byte_buffer_reader.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/call.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/call_details.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/call_log_batch.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/call_trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/channel.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/channel_init.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/channel_ping.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/channel_stack_type.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/completion_queue.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/completion_queue_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/event_string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/init.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/init_internally.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/lame_client.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/metadata_array.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/server.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/validate_metadata.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/version.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/batch_builder.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/bdp_estimator.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/connectivity_state.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/error_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/handshaker_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/http_connect_handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/metadata_batch.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/parsed_metadata.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/pid_controller.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/status_conversion.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/tcp_connect_handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/timeout_encoding.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/transport.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/transport_op_string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/uri/uri_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/plugin_registry/grpc_plugin_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/plugin_registry/grpc_plugin_registry_extra.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/crypt/aes_gcm.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/crypt/gsec.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_counter.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_crypter.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_frame_protector.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_record_protocol_crypter_common.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_seal_privacy_integrity_crypter.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_unseal_privacy_integrity_crypter.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/frame_handler.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/alts_handshaker_client.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/alts_shared_resource.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/alts_tsi_handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/alts_tsi_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/transport_security_common_api.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_grpc_integrity_only_record_protocol.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_grpc_privacy_integrity_record_protocol.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_grpc_record_protocol_common.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_iovec_record_protocol.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_zero_copy_grpc_protector.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/fake_transport_security.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/local_transport_security.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/ssl/key_logging/ssl_key_logging.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/ssl/session_cache/ssl_session_boringssl.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/ssl/session_cache/ssl_session_cache.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/ssl/session_cache/ssl_session_openssl.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/ssl_transport_security.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/ssl_transport_security_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/transport_security.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/transport_security_grpc.cc +) + +target_compile_features(grpc PUBLIC cxx_std_14) + +target_include_directories(grpc + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(grpc + ${_gRPC_ALLTARGETS_LIBRARIES} + ${_gRPC_RE2_LIBRARIES} + upb_json_lib + upb_textformat_lib + ${_gRPC_ZLIB_LIBRARIES} + absl::algorithm_container + absl::cleanup + absl::flat_hash_map + absl::flat_hash_set + absl::inlined_vector + absl::bind_front + absl::function_ref + absl::hash + absl::type_traits + absl::random_bit_gen_ref + absl::random_distributions + absl::statusor + absl::span + absl::utility + ${_gRPC_CARES_LIBRARIES} + gpr + ${_gRPC_SSL_LIBRARIES} + ${_gRPC_ADDRESS_SORTING_LIBRARIES} +) +if(_gRPC_PLATFORM_IOS OR _gRPC_PLATFORM_MAC) + target_link_libraries(grpc "-framework CoreFoundation") +endif() + +add_library(grpc_unsecure + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/backend_metrics/backend_metric_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/census/grpc_context.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/channel_idle/channel_idle_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/channel_idle/idle_filter_state.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/backend_metric.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/backup_poller.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/channel_connectivity.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_channelz.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_service_config.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/config_selector.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/dynamic_filters.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/global_subchannel_pool.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/http_proxy_mapper.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/address_filtering.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/child_policy_handler.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/endpoint_list.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/client_load_reporting_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_balancer_addresses.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_client_stats.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/health_check_client.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/oob_backend_metric.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/outlier_detection/outlier_detection.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/priority/priority.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/rls/rls.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/weighted_round_robin/static_stride_scheduler.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/weighted_round_robin/weighted_round_robin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/weighted_target/weighted_target.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/local_subchannel_pool.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/binder/binder_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/dns_resolver_ares.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/dns_resolver_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/event_engine/event_engine_client_channel_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/event_engine/service_config_helper.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/native/dns_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/fake/fake_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/polling_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/sockaddr/sockaddr_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_filter_legacy_call_data.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_service_config.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_throttle.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/service_config_channel_arg_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel_pool_interface.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel_stream_client.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/deadline/deadline_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/fault_injection/fault_injection_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/fault_injection/fault_injection_service_config_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/client/http_client_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/client_authority_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/http_filters_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/message_compress/compression_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/http/server/http_server_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/filters/message_size/message_size_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/chttp2_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/chttp2_server.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/bin_decoder.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/bin_encoder.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/chttp2_transport.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/decode_huff.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/flow_control.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_data.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_goaway.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_ping.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_rst_stream.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_settings.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_window_update.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_encoder.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_encoder_table.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_parse_result.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_parser_table.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/http2_settings.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/http_trace.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/huffsyms.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/max_concurrent_streams_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/parsing.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/ping_abuse_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/ping_callbacks.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/ping_rate_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/stream_lists.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/varint.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/write_size_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/writing.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/inproc/inproc_plugin.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/inproc/inproc_transport.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/annotations.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/http.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/any.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/descriptor.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/duration.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/empty.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/struct.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/timestamp.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/wrappers.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/rpc/status.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/altscontext.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/handshaker.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/transport_security_common.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/health/v1/health.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/lb/v1/load_balancer.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/lookup/v1/rls.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/validate/validate.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/data/orca/v3/orca_load_report.upb.c + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/xds/service/orca/v3/orca.upb.c + ${_gRPC_SOURCE_DIR}/src/core/lib/address_utils/parse_address.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/address_utils/sockaddr_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/backoff/backoff.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/backoff/random_early_detection.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/call_tracer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_args.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_args_preconditioning.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack_builder.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack_builder_impl.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channel_trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channelz.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/channelz_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/connected_channel.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/promise_based_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/server_call_tracer_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/channel/status_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/compression/compression.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/compression/compression_internal.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/compression/message_compress.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/config/core_configuration.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/event_log.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/histogram_view.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/stats.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/stats_data.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/debug/trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/ares_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/cf_engine/cf_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/cf_engine/cfstream_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/cf_engine/dns_service_resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/channel_args_endpoint_config.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/default_event_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/default_event_engine_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/event_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/forkable.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/memory_allocator.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/ev_epoll1_linux.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/ev_poll_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/event_poller_posix_default.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/internal_errqueue.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/lockfree_event.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/posix_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/posix_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/posix_engine_listener.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/posix_engine_listener_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/tcp_socket_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/timer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/timer_heap.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/timer_manager.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/traced_buffer_list.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/wakeup_fd_eventfd.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/wakeup_fd_pipe.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/posix_engine/wakeup_fd_posix_default.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/resolved_address.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/shim.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/slice.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/slice_buffer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/tcp_socket_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thread_pool/thread_count.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thread_pool/thread_pool_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thread_pool/work_stealing_thread_pool.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/thready_event_engine/thready_event_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/time_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/iocp.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/win_socket.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/windows_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/windows_engine.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/windows/windows_listener.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/event_engine/work_queue/basic_work_queue.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/experiments/config.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/experiments/experiments.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/load_file.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/per_cpu.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/ref_counted_string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/status_helper.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/time.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/time_averaged_stats.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/validation_errors.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/gprpp/work_serializer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/handshaker/proxy_mapper_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/http/format_request.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/http/httpcli.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/http/parser.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/buffer_list.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/call_combiner.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/cfstream_handle.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/closure.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/combiner.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/dualstack_socket_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_cfstream.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/error.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/error_cfstream.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_apple.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_epoll1_linux.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_poll_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/event_engine_shims/closure.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/event_engine_shims/endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/event_engine_shims/tcp_client.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/exec_ctx.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/executor.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/fork_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/fork_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_fallback.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_host_name_max.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_sysconf.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/grpc_if_nametoindex_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/grpc_if_nametoindex_unsupported.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/internal_errqueue.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iocp_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_internal.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_posix_cfstream.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/load_file.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/lockfree_event.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/polling_entity.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/sockaddr_utils_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_factory_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_mutator.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_common_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_linux.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/systemd_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_cfstream.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_common.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_ifaddrs.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_noifaddrs.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/timer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_generic.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_heap.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_manager.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/unix_sockets_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/unix_sockets_posix_noop.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/vsock.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_eventfd.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_nospecial.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_pipe.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_posix.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/json/json_object_loader.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/json/json_reader.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/json/json_writer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/load_balancing/lb_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/load_balancing/lb_policy_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/promise/activity.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/promise/party.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/promise/sleep.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/promise/trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resolver/resolver.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resolver/resolver_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resolver/server_address.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/api.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/arena.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/memory_quota.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/periodic_update.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/resource_quota.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/thread_quota.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/resource_quota/trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/authorization_policy_provider_vtable.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/evaluate_args.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/authorization/grpc_server_authz_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/certificate_provider/certificate_provider_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/context/security_context.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_linux.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_no_op.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_windows.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_client_options.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_options.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_server_options.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/call_creds_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/composite/composite_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/fake/fake_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/insecure/insecure_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/plugin/plugin_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/tls_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/fake/fake_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/insecure/insecure_security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/load_system_roots_fallback.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/load_system_roots_supported.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/security_connector/security_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/client_auth_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/secure_endpoint.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/security_handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/server_auth_filter.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/transport/tsi_error.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/security/util/json_util.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/service_config/service_config_impl.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/service_config/service_config_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/b64.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/percent_encoding.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/slice.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/slice_buffer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/slice_refcount.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/slice/slice_string_helpers.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/api_trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/builtins.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/byte_buffer.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/byte_buffer_reader.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/call.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/call_details.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/call_log_batch.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/call_trace.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/channel.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/channel_init.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/channel_ping.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/channel_stack_type.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/completion_queue.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/completion_queue_factory.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/event_string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/init.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/init_internally.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/lame_client.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/metadata_array.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/server.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/validate_metadata.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/surface/version.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/batch_builder.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/bdp_estimator.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/connectivity_state.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/error_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/handshaker_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/http_connect_handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/metadata_batch.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/parsed_metadata.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/pid_controller.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/status_conversion.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/tcp_connect_handshaker.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/timeout_encoding.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/transport.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/transport/transport_op_string.cc + ${_gRPC_SOURCE_DIR}/src/core/lib/uri/uri_parser.cc + ${_gRPC_SOURCE_DIR}/src/core/plugin_registry/grpc_plugin_registry.cc + ${_gRPC_SOURCE_DIR}/src/core/plugin_registry/grpc_plugin_registry_noextra.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/transport_security_common_api.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/fake_transport_security.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/local_transport_security.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/transport_security.cc + ${_gRPC_SOURCE_DIR}/src/core/tsi/transport_security_grpc.cc + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/message/accessors.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/build_enum.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/decode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/internal/base92.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/link.c + ${gRPC_ADDITIONAL_DLL_SRC} +) + +target_compile_features(grpc_unsecure PUBLIC cxx_std_14) + +target_include_directories(grpc_unsecure + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(grpc_unsecure + ${_gRPC_ALLTARGETS_LIBRARIES} + upb_collections_lib + upb + ${_gRPC_ZLIB_LIBRARIES} + absl::algorithm_container + absl::cleanup + absl::flat_hash_map + absl::flat_hash_set + absl::inlined_vector + absl::bind_front + absl::function_ref + absl::hash + absl::type_traits + absl::random_bit_gen_ref + absl::random_distributions + absl::statusor + absl::span + absl::utility + ${_gRPC_CARES_LIBRARIES} + gpr + ${_gRPC_ADDRESS_SORTING_LIBRARIES} +) +if(_gRPC_PLATFORM_IOS OR _gRPC_PLATFORM_MAC) + target_link_libraries(grpc_unsecure "-framework CoreFoundation") +endif() + +add_library(upb + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/base/status.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/collections/array.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/collections/map.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/collections/map_sorter.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/hash/common.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/lex/atoi.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/lex/round_trip.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/lex/strtod.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/lex/unicode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mem/alloc.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mem/arena.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/message/message.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_table/extension_registry.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_table/internal/message.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_table/message.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/wire/decode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/wire/decode_fast.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/wire/encode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/wire/eps_copy_input_stream.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/wire/reader.c +) + +target_compile_features(upb PUBLIC cxx_std_14) + +target_include_directories(upb + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(upb + ${_gRPC_ALLTARGETS_LIBRARIES} + utf8_range_lib +) + + +add_library(upb_collections_lib + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/base/status.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/collections/array.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/collections/map.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/collections/map_sorter.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/hash/common.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mem/alloc.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mem/arena.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/message/message.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_table/extension_registry.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_table/internal/message.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_table/message.c +) + +target_compile_features(upb_collections_lib PUBLIC cxx_std_14) + +target_include_directories(upb_collections_lib + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(upb_collections_lib + ${_gRPC_ALLTARGETS_LIBRARIES} +) + + + +add_library(upb_json_lib + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/descriptor.upb.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/json/decode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/json/encode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/message/accessors.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/build_enum.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/decode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/internal/base92.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/internal/encode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/link.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/def_builder.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/def_pool.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/def_type.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/desc_state.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/enum_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/enum_reserved_range.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/enum_value_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/extension_range.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/field_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/file_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/message.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/message_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/message_reserved_range.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/method_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/oneof_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/service_def.c +) + +target_compile_features(upb_json_lib PUBLIC cxx_std_14) + +target_include_directories(upb_json_lib + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(upb_json_lib + ${_gRPC_ALLTARGETS_LIBRARIES} + upb_collections_lib + upb +) + + +add_library(upb_textformat_lib + ${_gRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/descriptor.upb.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/message/accessors.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/build_enum.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/decode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/internal/base92.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/internal/encode.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/mini_descriptor/link.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/def_builder.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/def_pool.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/def_type.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/desc_state.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/enum_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/enum_reserved_range.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/enum_value_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/extension_range.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/field_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/file_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/message.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/message_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/message_reserved_range.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/method_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/oneof_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/reflection/service_def.c + ${_gRPC_SOURCE_DIR}/third_party/upb/upb/text/encode.c +) + +target_compile_features(upb_textformat_lib PUBLIC cxx_std_14) + +target_include_directories(upb_textformat_lib + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(upb_textformat_lib + ${_gRPC_ALLTARGETS_LIBRARIES} + upb_collections_lib + upb +) + + +add_library(utf8_range_lib + ${_gRPC_SOURCE_DIR}/third_party/utf8_range/naive.c + ${_gRPC_SOURCE_DIR}/third_party/utf8_range/range2-neon.c + ${_gRPC_SOURCE_DIR}/third_party/utf8_range/range2-sse.c +) + +target_compile_features(utf8_range_lib PUBLIC cxx_std_14) + +target_include_directories(utf8_range_lib + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(utf8_range_lib + ${_gRPC_ALLTARGETS_LIBRARIES} +) + + +add_library(grpc++ + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/client/binder_connector.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/client/channel_create.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/client/channel_create_impl.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/client/connection_id_generator.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/client/endpoint_binder_pool.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/client/jni_utils.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/client/security_policy_setting.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/security_policy/binder_security_policy.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/server/binder_server.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/server/binder_server_credentials.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/transport/binder_transport.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/utils/ndk_binder.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/utils/transport_stream_receiver_impl.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/wire_format/binder_android.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/wire_format/binder_constants.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/wire_format/transaction.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/wire_format/wire_reader_impl.cc + ${_gRPC_SOURCE_DIR}/src/core/ext/transport/binder/wire_format/wire_writer.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/channel_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/client_callback.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/client_context.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/client_interceptor.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/client_stats_interceptor.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/create_channel.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/create_channel_internal.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/create_channel_posix.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/insecure_credentials.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/secure_credentials.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/xds_credentials.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/alarm.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/auth_property_iterator.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/channel_arguments.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/channel_filter.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/completion_queue_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/resource_quota_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/rpc_method.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/secure_auth_context.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/secure_channel_arguments.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/secure_create_auth_context.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/tls_certificate_provider.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/tls_certificate_verifier.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/tls_credentials_options.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/validate_service_config.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/version_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/async_generic_service.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/backend_metric_recorder.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/channel_argument_option.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/create_default_thread_pool.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/external_connection_acceptor_impl.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/health/default_health_check_service.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/health/health_check_service.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/health/health_check_service_server_builder_option.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/insecure_server_credentials.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/secure_server_credentials.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_builder.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_callback.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_context.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_posix.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/xds_server_credentials.cc + ${_gRPC_SOURCE_DIR}/src/cpp/thread_manager/thread_manager.cc + ${_gRPC_SOURCE_DIR}/src/cpp/util/byte_buffer_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/util/status.cc + ${_gRPC_SOURCE_DIR}/src/cpp/util/string_ref.cc + ${_gRPC_SOURCE_DIR}/src/cpp/util/time_cc.cc + ${gRPC_UPB_GEN_DUPL_SRC} +) + +target_compile_features(grpc++ PUBLIC cxx_std_14) + +target_include_directories(grpc++ + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(grpc++ + ${_gRPC_ALLTARGETS_LIBRARIES} + grpc + ${_gRPC_PROTOBUF_LIBRARIES} +) + +add_library(grpc++_unsecure + ${_gRPC_SOURCE_DIR}/src/cpp/client/channel_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/client_callback.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/client_context.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/client_interceptor.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/client_stats_interceptor.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/create_channel.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/create_channel_internal.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/create_channel_posix.cc + ${_gRPC_SOURCE_DIR}/src/cpp/client/insecure_credentials.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/alarm.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/channel_arguments.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/channel_filter.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/completion_queue_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/insecure_create_auth_context.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/resource_quota_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/rpc_method.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/validate_service_config.cc + ${_gRPC_SOURCE_DIR}/src/cpp/common/version_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/async_generic_service.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/backend_metric_recorder.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/channel_argument_option.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/create_default_thread_pool.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/external_connection_acceptor_impl.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/health/default_health_check_service.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/health/health_check_service.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/health/health_check_service_server_builder_option.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/insecure_server_credentials.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_builder.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_callback.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_context.cc + ${_gRPC_SOURCE_DIR}/src/cpp/server/server_posix.cc + ${_gRPC_SOURCE_DIR}/src/cpp/thread_manager/thread_manager.cc + ${_gRPC_SOURCE_DIR}/src/cpp/util/byte_buffer_cc.cc + ${_gRPC_SOURCE_DIR}/src/cpp/util/status.cc + ${_gRPC_SOURCE_DIR}/src/cpp/util/string_ref.cc + ${_gRPC_SOURCE_DIR}/src/cpp/util/time_cc.cc + ${gRPC_UPB_GEN_DUPL_SRC} +) + +target_compile_features(grpc++_unsecure PUBLIC cxx_std_14) + +target_include_directories(grpc++_unsecure + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(grpc++_unsecure + ${_gRPC_ALLTARGETS_LIBRARIES} + grpc_unsecure + ${_gRPC_PROTOBUF_LIBRARIES} +) + +add_library(grpc_plugin_support + ${_gRPC_SOURCE_DIR}/src/compiler/cpp_generator.cc + ${_gRPC_SOURCE_DIR}/src/compiler/csharp_generator.cc + ${_gRPC_SOURCE_DIR}/src/compiler/node_generator.cc + ${_gRPC_SOURCE_DIR}/src/compiler/objective_c_generator.cc + ${_gRPC_SOURCE_DIR}/src/compiler/php_generator.cc + ${_gRPC_SOURCE_DIR}/src/compiler/proto_parser_helper.cc + ${_gRPC_SOURCE_DIR}/src/compiler/python_generator.cc + ${_gRPC_SOURCE_DIR}/src/compiler/ruby_generator.cc +) + +target_compile_features(grpc_plugin_support PUBLIC cxx_std_14) + +target_include_directories(grpc_plugin_support + PUBLIC ${_gRPC_SOURCE_DIR}/include + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) +target_link_libraries(grpc_plugin_support + ${_gRPC_ALLTARGETS_LIBRARIES} + ${_gRPC_PROTOBUF_LIBRARIES} + ${_gRPC_PROTOBUF_PROTOC_LIBRARIES} +) + + +add_executable(grpc_cpp_plugin + ${_gRPC_SOURCE_DIR}/src/compiler/cpp_plugin.cc +) +target_compile_features(grpc_cpp_plugin PUBLIC cxx_std_14) +target_include_directories(grpc_cpp_plugin + PRIVATE + ${_gRPC_SOURCE_DIR} + ${_gRPC_SOURCE_DIR}/include + ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} + ${_gRPC_RE2_INCLUDE_DIR} + ${_gRPC_SSL_INCLUDE_DIR} + ${_gRPC_UPB_GENERATED_DIR} + ${_gRPC_UPB_GRPC_GENERATED_DIR} + ${_gRPC_UPB_INCLUDE_DIR} + ${_gRPC_XXHASH_INCLUDE_DIR} + ${_gRPC_ZLIB_INCLUDE_DIR} +) + +target_link_libraries(grpc_cpp_plugin + ${_gRPC_ALLTARGETS_LIBRARIES} + grpc_plugin_support +) From b117d8ed12f2f3839a03fc4882f5228ff7d010b5 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Sun, 19 Nov 2023 14:28:33 +0000 Subject: [PATCH 555/813] create interpreter with changed select_query_options --- src/Interpreters/InterpreterSelectQueryAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index abcc2031047..eed9d03ab5a 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -209,7 +209,7 @@ Block InterpreterSelectQueryAnalyzer::getSampleBlock(const QueryTreeNodePtr & qu { auto select_query_options_copy = select_query_options; select_query_options_copy.only_analyze = true; - InterpreterSelectQueryAnalyzer interpreter(query_tree, context, select_query_options); + InterpreterSelectQueryAnalyzer interpreter(query_tree, context, select_query_options_copy); return interpreter.getSampleBlock(); } From 3213443ee2a53699f341b72916ca6011fe6f87cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Nov 2023 15:42:35 +0100 Subject: [PATCH 556/813] Own CMake for GRPC --- cmake/limit_jobs.cmake | 4 ++-- contrib/grpc-cmake/grpc.cmake | 10 ---------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index a43e208ff0d..9d693f65528 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -21,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(INFO "The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + message(INFORMATION "The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -32,7 +32,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(INFO "The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + message(INFORMATION "The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () diff --git a/contrib/grpc-cmake/grpc.cmake b/contrib/grpc-cmake/grpc.cmake index 43d4edd191e..c2488539211 100644 --- a/contrib/grpc-cmake/grpc.cmake +++ b/contrib/grpc-cmake/grpc.cmake @@ -55,16 +55,6 @@ if(UNIX) endif() endif() -if(UNIX AND NOT HAIKU) - # -pthread does more than -lpthread - set(THREADS_PREFER_PTHREAD_FLAG ON) - find_package(Threads) - set(_gRPC_ALLTARGETS_LIBRARIES ${CMAKE_DL_LIBS} Threads::Threads) - if(_gRPC_PLATFORM_LINUX OR _gRPC_PLATFORM_POSIX) - set(_gRPC_ALLTARGETS_LIBRARIES ${_gRPC_ALLTARGETS_LIBRARIES} rt) - endif() -endif() - set(_gRPC_ADDRESS_SORTING_INCLUDE_DIR "${_gRPC_SOURCE_DIR}/third_party/address_sorting/include") set(_gRPC_ADDRESS_SORTING_LIBRARIES address_sorting) From 907f168e0dea4df5ac305fecf63c9c9a3820e433 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Nov 2023 15:43:09 +0100 Subject: [PATCH 557/813] Own CMake for GRPC --- cmake/limit_jobs.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 9d693f65528..8e48fc9b9d8 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -21,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(INFORMATION "The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + message("The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -32,7 +32,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(INFORMATION "The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + message("The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () From cf4621444042c9dde066ddfe5b1dc50af04e2a8b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 19 Nov 2023 15:09:52 +0000 Subject: [PATCH 558/813] Always send fatal level logs --- programs/client/Client.cpp | 1 - programs/local/LocalServer.cpp | 19 ++++--------------- src/Client/ClientBase.h | 2 -- 3 files changed, 4 insertions(+), 18 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index d29824581fa..8029f97a992 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1241,7 +1241,6 @@ void Client::processConfig() global_context->setCurrentQueryId(query_id); } print_stack_trace = config().getBool("stacktrace", false); - logging_initialized = true; if (config().has("multiquery")) is_multiquery = true; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index f3b551b08d2..9a3f948a01c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -563,9 +563,6 @@ catch (...) void LocalServer::updateLoggerLevel(const String & logs_level) { - if (!logging_initialized) - return; - config().setString("logger.level", logs_level); updateLevels(config(), logger()); } @@ -607,21 +604,13 @@ void LocalServer::processConfig() Poco::AutoPtr pf = new OwnPatternFormatter; Poco::AutoPtr log = new OwnFormattingChannel(pf, new Poco::SimpleFileChannel(server_logs_file)); Poco::Logger::root().setChannel(log); - logging_initialized = true; - } - else if (logging || is_interactive) - { - config().setString("logger", "logger"); - auto log_level_default = is_interactive && !logging ? "fatal" : level; - config().setString("logger.level", config().getString("log-level", config().getString("send_logs_level", log_level_default))); - buildLoggers(config(), logger(), "clickhouse-local"); - logging_initialized = true; } else { - Poco::Logger::root().setLevel("none"); - Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::NullChannel())); - logging_initialized = false; + config().setString("logger", "logger"); + auto log_level_default = logging ? level : "fatal"; + config().setString("logger.level", config().getString("log-level", config().getString("send_logs_level", log_level_default))); + buildLoggers(config(), logger(), "clickhouse-local"); } shared_context = Context::createShared(); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 2156aae7181..9fde23cf775 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -321,8 +321,6 @@ protected: bool allow_merge_tree_settings = false; bool cancelled = false; - - bool logging_initialized = false; }; } From 60a17ee397bf98e07ece79bef06d6b025bc0dfe0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 Nov 2023 15:27:59 +0000 Subject: [PATCH 559/813] Fix build --- src/Backups/BackupIO_S3.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 76b9fba7b83..ea3f57c27ff 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -253,7 +253,6 @@ void BackupWriterS3::copyFile(const String & destination, const String & source, { LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); copyS3File( - client, client, /* src_bucket */ s3_uri.bucket, /* src_key= */ fs::path(s3_uri.key) / source, From 4fc658cd1fe751e082ed4cf5bbda581c1b5a145a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Nov 2023 16:31:03 +0100 Subject: [PATCH 560/813] Fix build --- src/Backups/BackupIO_S3.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 76b9fba7b83..ea3f57c27ff 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -253,7 +253,6 @@ void BackupWriterS3::copyFile(const String & destination, const String & source, { LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); copyS3File( - client, client, /* src_bucket */ s3_uri.bucket, /* src_key= */ fs::path(s3_uri.key) / source, From 62a87665c551f4efa9ae5b98000604e64590e79e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Nov 2023 16:31:03 +0100 Subject: [PATCH 561/813] Fix build --- src/Backups/BackupIO_S3.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 76b9fba7b83..ea3f57c27ff 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -253,7 +253,6 @@ void BackupWriterS3::copyFile(const String & destination, const String & source, { LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); copyS3File( - client, client, /* src_bucket */ s3_uri.bucket, /* src_key= */ fs::path(s3_uri.key) / source, From a3c9f13ac916ddecf74fba96d7c85cfe9c3f7468 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 Nov 2023 15:33:58 +0000 Subject: [PATCH 562/813] Add exclude for tryBase64Decode to backward compat test (follow-up to #56913) Fixes #56969 --- .../integration/test_backward_compatibility/test_functions.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index 94771a624e2..b6b6ef28de5 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -153,6 +153,9 @@ def test_string_functions(start_cluster): # mandatory or optional). The former lib produces a value based on implicit padding, the latter lib throws an error. "FROM_BASE64", "base64Decode", + # PR #56913 (in v23.11) corrected the way tryBase64Decode() behaved with invalid inputs. Old versions return garbage, new versions + # return an empty string (as it was always documented). + "tryBase64Decode", # Removed in 23.9 "meiliMatch", ] From 0513c93829d8c6d5f3226d98593b325d2d68f9df Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 19 Nov 2023 23:11:05 +0100 Subject: [PATCH 563/813] Prefer sccache to ccache by default --- cmake/ccache.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/ccache.cmake b/cmake/ccache.cmake index e8bf856332a..0df70d82d2c 100644 --- a/cmake/ccache.cmake +++ b/cmake/ccache.cmake @@ -9,10 +9,10 @@ if (CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" OR CMAKE_C_COMPILER_LAUNCHER MA return() endif() -set(COMPILER_CACHE "auto" CACHE STRING "Speedup re-compilations using the caching tools; valid options are 'auto' (ccache, then sccache), 'ccache', 'sccache', or 'disabled'") +set(COMPILER_CACHE "auto" CACHE STRING "Speedup re-compilations using the caching tools; valid options are 'auto' (sccache, then ccache), 'ccache', 'sccache', or 'disabled'") if(COMPILER_CACHE STREQUAL "auto") - find_program (CCACHE_EXECUTABLE NAMES ccache sccache) + find_program (CCACHE_EXECUTABLE NAMES sccache ccache) elseif (COMPILER_CACHE STREQUAL "ccache") find_program (CCACHE_EXECUTABLE ccache) elseif(COMPILER_CACHE STREQUAL "sccache") @@ -21,7 +21,7 @@ elseif(COMPILER_CACHE STREQUAL "disabled") message(STATUS "Using *ccache: no (disabled via configuration)") return() else() - message(${RECONFIGURE_MESSAGE_LEVEL} "The COMPILER_CACHE must be one of (auto|ccache|sccache|disabled), value: '${COMPILER_CACHE}'") + message(${RECONFIGURE_MESSAGE_LEVEL} "The COMPILER_CACHE must be one of (auto|sccache|ccache|disabled), value: '${COMPILER_CACHE}'") endif() From 33df68cd0149a901812cfb0276a097c042ed800f Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Sun, 19 Nov 2023 14:48:21 -0800 Subject: [PATCH 564/813] update 02003_memory_limit_in_client.sh --- tests/queries/0_stateless/02003_memory_limit_in_client.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh index 2d2493828c8..4017c3771a6 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.sh +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -1,4 +1,4 @@ -#!/usr/bin/bash -f +#!/usr/bin/env bash CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From cfdb66389220d28caa1226f70df6e1c6b7ffe0de Mon Sep 17 00:00:00 2001 From: melvynator Date: Mon, 20 Nov 2023 00:46:14 +0100 Subject: [PATCH 565/813] Remove a tab --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 96247e9e1a7..a287b96fe51 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -549,7 +549,7 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) throw Exception(ErrorCodes::BAD_ARGUMENTS, "MaterializedPostgreSQL is an experimental table engine." " You can enable it with the `allow_experimental_materialized_postgresql_table` setting"); - + if (!args.storage_def->order_by && args.storage_def->primary_key) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); From d385217012723ba7f380a5caf1348f429e25b14e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Nov 2023 01:15:04 +0100 Subject: [PATCH 566/813] One step back --- base/glibc-compatibility/CMakeLists.txt | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index c967fa5b11b..59f29093d5f 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -35,6 +35,13 @@ if (GLIBC_COMPATIBILITY) target_link_libraries(global-libs INTERFACE glibc-compatibility ${MEMCPY_LIBRARY}) + # TODO: remove it + install( + TARGETS glibc-compatibility ${MEMCPY_LIBRARY} + EXPORT global + ARCHIVE DESTINATION lib + ) + message (STATUS "Some symbols from glibc will be replaced for compatibility") elseif (CLICKHOUSE_OFFICIAL_BUILD) From b205d4919d8ce88e7776bcc0e6ff3de16460ec9f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Nov 2023 01:24:28 +0100 Subject: [PATCH 567/813] Remove garbage --- CMakeLists.txt | 8 -------- base/glibc-compatibility/CMakeLists.txt | 7 ------- cmake/darwin/default_libs.cmake | 6 ------ cmake/freebsd/default_libs.cmake | 6 ------ cmake/linux/default_libs.cmake | 6 ------ contrib/llvm-project-cmake/CMakeLists.txt | 3 +++ 6 files changed, 3 insertions(+), 33 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4fe7a1e05e7..2486b970c59 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -460,14 +460,6 @@ endif () message (STATUS "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE}") -include (GNUInstallDirs) - -# When testing for memory leaks with Valgrind, don't link tcmalloc or jemalloc. - -if (TARGET global-group) - install (EXPORT global DESTINATION cmake) -endif () - add_subdirectory (contrib EXCLUDE_FROM_ALL) if (NOT ENABLE_JEMALLOC) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 59f29093d5f..c967fa5b11b 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -35,13 +35,6 @@ if (GLIBC_COMPATIBILITY) target_link_libraries(global-libs INTERFACE glibc-compatibility ${MEMCPY_LIBRARY}) - # TODO: remove it - install( - TARGETS glibc-compatibility ${MEMCPY_LIBRARY} - EXPORT global - ARCHIVE DESTINATION lib - ) - message (STATUS "Some symbols from glibc will be replaced for compatibility") elseif (CLICKHOUSE_OFFICIAL_BUILD) diff --git a/cmake/darwin/default_libs.cmake b/cmake/darwin/default_libs.cmake index 42b8473cb75..cf0210d9b45 100644 --- a/cmake/darwin/default_libs.cmake +++ b/cmake/darwin/default_libs.cmake @@ -22,9 +22,3 @@ link_libraries(global-group) target_link_libraries(global-group INTERFACE $ ) - -# FIXME: remove when all contribs will get custom cmake lists -install( - TARGETS global-group global-libs - EXPORT global -) diff --git a/cmake/freebsd/default_libs.cmake b/cmake/freebsd/default_libs.cmake index 65bf296ee09..1eeb1a872bd 100644 --- a/cmake/freebsd/default_libs.cmake +++ b/cmake/freebsd/default_libs.cmake @@ -25,9 +25,3 @@ link_libraries(global-group) target_link_libraries(global-group INTERFACE $ ) - -# FIXME: remove when all contribs will get custom cmake lists -install( - TARGETS global-group global-libs - EXPORT global -) diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index 56a663a708e..8552097fa57 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -50,9 +50,3 @@ target_link_libraries(global-group INTERFACE $ -Wl,--end-group ) - -# FIXME: remove when all contribs will get custom cmake lists -install( - TARGETS global-group global-libs - EXPORT global -) diff --git a/contrib/llvm-project-cmake/CMakeLists.txt b/contrib/llvm-project-cmake/CMakeLists.txt index d6133f145bc..406bac73e90 100644 --- a/contrib/llvm-project-cmake/CMakeLists.txt +++ b/contrib/llvm-project-cmake/CMakeLists.txt @@ -61,6 +61,9 @@ set (REQUIRED_LLVM_LIBRARIES LLVMDemangle ) +# Skip useless "install" instructions from CMake: +set (LLVM_INSTALL_TOOLCHAIN_ONLY 1 CACHE INTERNAL "") + if (ARCH_AMD64) set (LLVM_TARGETS_TO_BUILD "X86" CACHE INTERNAL "") list(APPEND REQUIRED_LLVM_LIBRARIES LLVMX86Info LLVMX86Desc LLVMX86CodeGen) From f6e4c29669f85b15ae6c84938d5e5cc8ca2628d1 Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Mon, 20 Nov 2023 01:49:17 +0100 Subject: [PATCH 568/813] MaterializedMysql doc Add experimental flag for materializedMysql --- docs/en/engines/database-engines/materialized-mysql.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index b7e567c7b6c..f32698f84f6 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -7,7 +7,10 @@ sidebar_position: 70 # [experimental] MaterializedMySQL :::note -This is an experimental feature that should not be used in production. +This database engine is experimental. To use it, set `allow_experimental_database_materialized_mysql` to 1 in your configuration files or by using the `SET` command: +```sql +SET allow_experimental_database_materialized_mysql=1 +``` ::: Creates a ClickHouse database with all the tables existing in MySQL, and all the data in those tables. The ClickHouse server works as MySQL replica. It reads `binlog` and performs DDL and DML queries. From 6dc3efb3f9aef0bf65360a1b9a1413e9e8ead71b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Nov 2023 01:55:34 +0100 Subject: [PATCH 569/813] Fix build --- CMakeLists.txt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 2486b970c59..063cfc77302 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -21,8 +21,11 @@ include (cmake/clang_tidy.cmake) include (cmake/git.cmake) include (cmake/utils.cmake) +# This is needed to set up the CMAKE_INSTALL_BINDIR variable. +include (GNUInstallDirs) + # Ignore export() since we don't use it, -# but it gets broken with a global targets via link_libraries() +# but it gets broken with global targets via link_libraries() macro (export) endmacro () From ab5f3d12b63388f403ce7d1758076a47839a7b23 Mon Sep 17 00:00:00 2001 From: santrancisco Date: Mon, 20 Nov 2023 12:51:30 +1100 Subject: [PATCH 570/813] Fix sqlite file path validation to make sure it does not skip validation on relative path --- src/Databases/SQLite/SQLiteUtils.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Databases/SQLite/SQLiteUtils.cpp b/src/Databases/SQLite/SQLiteUtils.cpp index 152370050f1..4fe7c6f1707 100644 --- a/src/Databases/SQLite/SQLiteUtils.cpp +++ b/src/Databases/SQLite/SQLiteUtils.cpp @@ -26,10 +26,12 @@ void processSQLiteError(const String & message, bool throw_on_error) String validateSQLiteDatabasePath(const String & path, const String & user_files_path, bool need_check, bool throw_on_error) { - if (fs::path(path).is_relative()) - return fs::absolute(fs::path(user_files_path) / path).lexically_normal(); - + String absolute_path = fs::absolute(path).lexically_normal(); + + if (fs::path(path).is_relative()) + absolute_path = fs::absolute(fs::path(user_files_path) / path).lexically_normal(); + String absolute_user_files_path = fs::absolute(user_files_path).lexically_normal(); if (need_check && !absolute_path.starts_with(absolute_user_files_path)) From f05f572b3c34bd437218f23fc1140e30cf7a4c62 Mon Sep 17 00:00:00 2001 From: santrancisco Date: Mon, 20 Nov 2023 13:37:27 +1100 Subject: [PATCH 571/813] Fixed trailing whitespace --- src/Databases/SQLite/SQLiteUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/SQLite/SQLiteUtils.cpp b/src/Databases/SQLite/SQLiteUtils.cpp index 4fe7c6f1707..ddc2fb911e9 100644 --- a/src/Databases/SQLite/SQLiteUtils.cpp +++ b/src/Databases/SQLite/SQLiteUtils.cpp @@ -28,7 +28,7 @@ String validateSQLiteDatabasePath(const String & path, const String & user_files { String absolute_path = fs::absolute(path).lexically_normal(); - + if (fs::path(path).is_relative()) absolute_path = fs::absolute(fs::path(user_files_path) / path).lexically_normal(); From 96c603ef97ce63a617637b0c1f45053ce0b5899c Mon Sep 17 00:00:00 2001 From: Chuan-Zheng Lee Date: Mon, 20 Nov 2023 17:17:27 +1300 Subject: [PATCH 572/813] [Docs] MaterializedPostgreSQL: Change DETACH to DETACH PERMANENTLY If I'm not mistaken, ClickHouse/ClickHouse#35158 changed the syntax for dynamically removing tables from MaterializedPostgreSQL databases from `DETACH` to `DETACH PERMANENTLY`. Currently when just running `DETACH TABLE postgres_database.table_to_remove`, it shows an error: ``` DETACH TABLE not allowed, use DETACH PERMANENTLY. (NOT_IMPLEMENTED) ``` This adds the keyword `PERMANENTLY` to both places where `DETACH` occurs on the MaterializedPostgreSQL database engine page. --- docs/en/engines/database-engines/materialized-postgresql.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 4e978947e36..3aa6dd01ea3 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -8,7 +8,7 @@ sidebar_position: 60 Creates a ClickHouse database with tables from PostgreSQL database. Firstly, database with engine `MaterializedPostgreSQL` creates a snapshot of PostgreSQL database and loads required tables. Required tables can include any subset of tables from any subset of schemas from specified database. Along with the snapshot database engine acquires LSN and once initial dump of tables is performed - it starts pulling updates from WAL. After database is created, newly added tables to PostgreSQL database are not automatically added to replication. They have to be added manually with `ATTACH TABLE db.table` query. -Replication is implemented with PostgreSQL Logical Replication Protocol, which does not allow to replicate DDL, but allows to know whether replication breaking changes happened (column type changes, adding/removing columns). Such changes are detected and according tables stop receiving updates. In this case you should use `ATTACH`/ `DETACH` queries to reload table completely. If DDL does not break replication (for example, renaming a column) table will still receive updates (insertion is done by position). +Replication is implemented with PostgreSQL Logical Replication Protocol, which does not allow to replicate DDL, but allows to know whether replication breaking changes happened (column type changes, adding/removing columns). Such changes are detected and according tables stop receiving updates. In this case you should use `ATTACH`/ `DETACH PERMANENTLY` queries to reload table completely. If DDL does not break replication (for example, renaming a column) table will still receive updates (insertion is done by position). :::note This database engine is experimental. To use it, set `allow_experimental_database_materialized_postgresql` to 1 in your configuration files or by using the `SET` command: @@ -63,7 +63,7 @@ Before version 22.1, adding a table to replication left a non-removed temporary It is possible to remove specific tables from replication: ``` sql -DETACH TABLE postgres_database.table_to_remove; +DETACH TABLE postgres_database.table_to_remove PERMANENTLY; ``` ## PostgreSQL schema {#schema} From af0c954c9e14c23bad56670b5b4c626e32a08c57 Mon Sep 17 00:00:00 2001 From: Kevin Mingtarja Date: Mon, 20 Nov 2023 15:05:12 +0800 Subject: [PATCH 573/813] Add implicit constraint for CollapsingMergeTree sign column --- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../MergeTree/registerStorageMergeTree.cpp | 27 +++++++++++++--- ...onstraints_for_collapsing_engine.reference | 2 ++ ...olumn_constraints_for_collapsing_engine.sh | 31 +++++++++++++++++++ 4 files changed, 56 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference create mode 100644 tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 69307e74d1d..df1e8a17e18 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,6 +84,7 @@ struct Settings; M(UInt64, min_delay_to_insert_ms, 10, "Min delay of inserting data into MergeTree table in milliseconds, if there are a lot of unmerged parts in single partition.", 0) \ M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ 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) \ \ /* Part removal settings. */ \ M(UInt64, simultaneous_parts_removal_limit, 0, "Maximum number of parts to remove during one CleanupThread iteration (0 means unlimited).", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index f22d86499c2..9285dfcdd91 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -598,11 +598,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.projections.add(std::move(projection)); } - auto constraints = metadata.constraints.getConstraints(); - if (args.query.columns_list && args.query.columns_list->constraints) - for (auto & constraint : args.query.columns_list->constraints->children) - constraints.push_back(constraint); - metadata.constraints = ConstraintsDescription(constraints); auto column_ttl_asts = columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) @@ -620,6 +615,28 @@ static StoragePtr create(const StorageFactory::Arguments & args) args.getLocalContext()->checkMergeTreeSettingsConstraints(initial_storage_settings, storage_settings->changes()); metadata.settings_changes = args.storage_def->settings->ptr(); } + + auto constraints = metadata.constraints.getConstraints(); + if (args.query.columns_list && args.query.columns_list->constraints) + for (auto & constraint : args.query.columns_list->constraints->children) + constraints.push_back(constraint); + if (merging_params.mode == MergeTreeData::MergingParams::Collapsing && storage_settings->add_implicit_sign_column_constraint_for_collapsing_engine) + { + auto sign_column_check_constraint = std::make_unique(); + sign_column_check_constraint->name = "check_sign_column"; + sign_column_check_constraint->type = ASTConstraintDeclaration::Type::CHECK; + + Array valid_values_array; + valid_values_array.emplace_back(-1); + valid_values_array.emplace_back(1); + + auto valid_values_ast = std::make_unique(std::move(valid_values_array)); + auto sign_column_ast = std::make_unique(merging_params.sign_column); + sign_column_check_constraint->set(sign_column_check_constraint->expr, makeASTFunction("in", std::move(sign_column_ast), std::move(valid_values_ast))); + + constraints.push_back(std::move(sign_column_check_constraint)); + } + metadata.constraints = ConstraintsDescription(constraints); } else { diff --git a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference new file mode 100644 index 00000000000..5c6c001014d --- /dev/null +++ b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference @@ -0,0 +1,2 @@ +1 2504 1 +ok diff --git a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh new file mode 100644 index 00000000000..bee12afc511 --- /dev/null +++ b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +EXCEPTION_SUCCESS_TEXT=ok +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS collapsing_merge_tree;" + +# CollapsingSortedAlgorithm::merge() also has a check for sign column value +# optimize_on_insert = 0 is required to avoid this automatic merge behavior +$CLICKHOUSE_CLIENT --query="SET optimize_on_insert = 0;" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE collapsing_merge_tree +( + Key UInt32, + Count UInt16, + Sign Int8 +) +ENGINE=CollapsingMergeTree(Sign) ORDER BY Key +SETTINGS add_implicit_sign_column_constraint_for_collapsing_engine=1;" + +# Should succeed +$CLICKHOUSE_CLIENT --query="INSERT INTO collapsing_merge_tree VALUES (1, 2504, 1);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM collapsing_merge_tree;" + +# Should throw an exception +$CLICKHOUSE_CLIENT --query="INSERT INTO collapsing_merge_tree VALUES (1, 2504, 5);" 2>&1 \ + | grep -q VIOLATED_CONSTRAINT && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" + +$CLICKHOUSE_CLIENT --query="DROP TABLE collapsing_merge_tree;" From 1f044b11e36d17097fab066044edf7ebde746ffc Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Mon, 20 Nov 2023 13:56:03 +0800 Subject: [PATCH 574/813] Enhance unittest MergeTree.CombineFilters --- .../MergeTree/tests/gtest_combine_filters.cpp | 84 ++++++++++++++++++- 1 file changed, 83 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/tests/gtest_combine_filters.cpp b/src/Storages/MergeTree/tests/gtest_combine_filters.cpp index 64cccd4cbad..91e9e53c230 100644 --- a/src/Storages/MergeTree/tests/gtest_combine_filters.cpp +++ b/src/Storages/MergeTree/tests/gtest_combine_filters.cpp @@ -7,7 +7,7 @@ using namespace DB; /* The combineFilters function from MergeTreeRangeReader.cpp could be optimized with Intel's AVX512VBMI2 intrinsic, - * _mm512_mask_expandloadu_epi8. And this test is added to ensure that the vectorized code outputs the exact results + * _mm512_mask_expandloadu_epi8. And these tests are added to ensure that the vectorized code outputs the exact results * as the original scalar code when the required hardware feature is supported on the device. * * To avoid the contingency of the all-one/all-zero sequences, this test fills in the filters with alternating 1s and @@ -54,8 +54,81 @@ bool testCombineFilters(size_t size) return true; } +/* This test is to further test DB::combineFilters by combining two UInt8 columns. Given the implementation of + * DB::combineFilters, the non-zero values in the first column are contiguously replaced with the elements in the + * second column. And to validate the first column with arbitrary intervals, this test constructs its values in + * the following manner: the count of 0s between two consecutive 1s increases in step of 1. An example column + * with the size of 16 looks like: + * [1 1 0 1 0 0 1 0 0 0 1 0 0 0 0 1] + * + * The second column contains the consecutively incremented UInt8 integers between 0x00 and 0xFF, and when the overflow + * occurs, the value would reset to 0x00 and increment again. + */ +bool testCombineColumns(size_t size) +{ + auto generateFirstColumn = [] (size_t len, size_t & non_zero_count)->ColumnPtr + { + auto column = ColumnUInt8::create(len, 0); + auto & column_data = column->getData(); + + non_zero_count = 0; + for (size_t i = 0; i < len; non_zero_count++, i+=non_zero_count) + { + column_data[i] = 1; + } + + return column; + }; + + auto generateSecondColumn = [] (size_t len)->ColumnPtr + { + auto column = ColumnUInt8::create(len, 0); + auto & column_data = column->getData(); + + for (size_t i = 0; i < len; i++) + { + column_data[i] = static_cast(i); + } + + return column; + }; + + size_t non_zero_count = 0; + auto first_column = generateFirstColumn(size, non_zero_count); + const auto & first_column_data = typeid_cast(first_column.get())->getData(); + + /// The count of non-zero values in the first column should be the size of the second column. + auto second_column = generateSecondColumn(non_zero_count); + + auto result = combineFilters(first_column, second_column); + const auto & result_data = typeid_cast(result.get())->getData(); + + if (result->size() != size) return false; + + UInt8 expected = 0; + for (size_t i = 0; i < size; ++i) + { + if (first_column_data[i]) + { + if (result_data[i] != expected) + { + return false; + } + /// Integer overflow is speculated during the integer increments. It is the expected behavior. + expected++; + } + else + { + if (result_data[i] != 0) return false; + } + } + + return true; +} + TEST(MergeTree, CombineFilters) { + /// Tests with only 0/1 and fixed intervals. EXPECT_TRUE(testCombineFilters(1)); EXPECT_TRUE(testCombineFilters(2)); EXPECT_TRUE(testCombineFilters(63)); @@ -63,4 +136,13 @@ TEST(MergeTree, CombineFilters) EXPECT_TRUE(testCombineFilters(65)); EXPECT_TRUE(testCombineFilters(200)); EXPECT_TRUE(testCombineFilters(201)); + /// Extended tests: combination of two UInt8 columns. + EXPECT_TRUE(testCombineColumns(1)); + EXPECT_TRUE(testCombineColumns(2)); + EXPECT_TRUE(testCombineColumns(63)); + EXPECT_TRUE(testCombineColumns(64)); + EXPECT_TRUE(testCombineColumns(200)); + EXPECT_TRUE(testCombineColumns(201)); + EXPECT_TRUE(testCombineColumns(2000)); + EXPECT_TRUE(testCombineColumns(200000)); } From f226fa685bc55f7b81f4838a45a6037a4202dc51 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 20 Nov 2023 10:41:07 +0000 Subject: [PATCH 575/813] merge_row_policy: alias test separated out and disabled if analyzer --- tests/analyzer_tech_debt.txt | 1 + .../02763_row_policy_storage_merge.reference | 50 ------------------- .../02763_row_policy_storage_merge.sql.j2 | 34 ------------- ...3_row_policy_storage_merge_alias.reference | 49 ++++++++++++++++++ ...2763_row_policy_storage_merge_alias.sql.j2 | 41 +++++++++++++++ 5 files changed, 91 insertions(+), 84 deletions(-) create mode 100644 tests/queries/0_stateless/02763_row_policy_storage_merge_alias.reference create mode 100644 tests/queries/0_stateless/02763_row_policy_storage_merge_alias.sql.j2 diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e155ee72ebb..0984d238782 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -42,6 +42,7 @@ 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET 02404_memory_bound_merging 02725_agg_projection_resprect_PK +02763_row_policy_storage_merge_alias 02765_parallel_replicas_final_modifier 02784_parallel_replicas_automatic_decision_join 02818_parameterized_view_with_cte_multiple_usage diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference index 0b7664deb7d..9fa5612e7cd 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.reference +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.reference @@ -166,31 +166,6 @@ SELECT x, y from merge(currentDatabase(), 02763_merge 4 14 4 14 4 14 -02763_merge_aliases -x, y, z FROM 02763_a_merge -3 13 16 -4 14 18 -* FROM 02763_a_merge -3 13 16 -4 14 18 -x, y FROM 02763_a_merge -3 13 -4 14 -SELECT x, y FROM merge(currentDatabase(), 02763_alias) -3 13 -4 14 -SELECT x, y FROM merge(currentDatabase(), 02763_alias) -2 12 -3 13 -4 14 -SELECT x FROM merge(currentDatabase(), 02763_alias) -12 -13 -14 -SELECT y FROM merge(currentDatabase(), 02763_alias) -2 -3 -4 SETTINGS optimize_move_to_prewhere= 1 SELECT * FROM 02763_merge_log_1 3 13 @@ -337,28 +312,3 @@ SELECT x, y from merge(currentDatabase(), 02763_merge 4 14 4 14 4 14 -02763_merge_aliases -x, y, z FROM 02763_a_merge -3 13 16 -4 14 18 -* FROM 02763_a_merge -3 13 16 -4 14 18 -x, y FROM 02763_a_merge -3 13 -4 14 -SELECT x, y FROM merge(currentDatabase(), 02763_alias) -3 13 -4 14 -SELECT x, y FROM merge(currentDatabase(), 02763_alias) -2 12 -3 13 -4 14 -SELECT x FROM merge(currentDatabase(), 02763_alias) -12 -13 -14 -SELECT y FROM merge(currentDatabase(), 02763_alias) -2 -3 -4 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 index a22a9c5b641..0263e1a974f 100644 --- a/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge.sql.j2 @@ -3,7 +3,6 @@ DROP TABLE IF EXISTS 02763_merge_log_2; DROP TABLE IF EXISTS 02763_merge_merge_1; DROP TABLE IF EXISTS 02763_merge_merge_2; DROP TABLE IF EXISTS 02763_merge_fancycols; -DROP TABLE IF EXISTS 02763_merge_aliases; DROP ROW POLICY IF EXISTS 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY IF EXISTS 02763_filter_2 ON 02763_merge_merge_1; DROP ROW POLICY IF EXISTS 02763_filter_3 ON 02763_merge_log_1; @@ -132,36 +131,6 @@ SELECT x, y from merge(currentDatabase(), '02763_merge') ORDER BY x SETTINGS opt DROP TABLE 02763_merge_fancycols; -SELECT '02763_merge_aliases'; -CREATE TABLE 02763_alias (x UInt8, y UInt64, z UInt64 ALIAS plus(x,y)) ENGINE = MergeTree ORDER BY x; -INSERT INTO 02763_alias VALUES (1, 11), (2, 12), (3, 13), (4, 14); - -CREATE ROW POLICY 02763_filter_7 ON 02763_alias USING z>15 AS permissive TO ALL; - -CREATE TABLE 02763_a_merge (x UInt8, y UInt64, z UInt64) ENGINE = Merge(currentDatabase(), '02763_alias'); - - -SELECT 'x, y, z FROM 02763_a_merge'; -SELECT x, y, z FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; -SELECT '* FROM 02763_a_merge'; -SELECT * FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; -SELECT 'x, y FROM 02763_a_merge'; -SELECT x, y FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; -SELECT 'SELECT x, y FROM merge(currentDatabase(), 02763_alias)'; -SELECT x, y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; - -CREATE ROW POLICY 02763_filter_8 ON 02763_alias USING y>11 AS permissive TO ALL; - -SELECT 'SELECT x, y FROM merge(currentDatabase(), 02763_alias)'; -SELECT x, y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; -SELECT 'SELECT x FROM merge(currentDatabase(), 02763_alias)'; -SELECT y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; -SELECT 'SELECT y FROM merge(currentDatabase(), 02763_alias)'; -SELECT x FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; - -DROP TABLE 02763_alias; -DROP TABLE 02763_a_merge; - DROP ROW POLICY 02763_filter_1 ON 02763_merge_log_1; DROP ROW POLICY 02763_filter_2 ON 02763_merge_merge_1; @@ -171,7 +140,4 @@ DROP ROW POLICY 02763_filter_4 ON 02763_merge_merge_1; DROP ROW POLICY 02763_filter_5 ON 02763_merge_fancycols; DROP ROW POLICY 02763_filter_6 ON 02763_merge_fancycols; -DROP ROW POLICY 02763_filter_7 ON 02763_alias; -DROP ROW POLICY 02763_filter_8 ON 02763_alias; - {% endfor %} diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge_alias.reference b/tests/queries/0_stateless/02763_row_policy_storage_merge_alias.reference new file mode 100644 index 00000000000..56bfdbe0b18 --- /dev/null +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge_alias.reference @@ -0,0 +1,49 @@ +02763_merge_aliases +x, y, z FROM 02763_a_merge +3 13 16 +4 14 18 +* FROM 02763_a_merge +3 13 16 +4 14 18 +x, y FROM 02763_a_merge +3 13 +4 14 +SELECT x, y FROM merge(currentDatabase(), 02763_alias) +3 13 +4 14 +SELECT x, y FROM merge(currentDatabase(), 02763_alias) +2 12 +3 13 +4 14 +SELECT x FROM merge(currentDatabase(), 02763_alias) +12 +13 +14 +SELECT y FROM merge(currentDatabase(), 02763_alias) +2 +3 +4 +x, y, z FROM 02763_a_merge +3 13 16 +4 14 18 +* FROM 02763_a_merge +3 13 16 +4 14 18 +x, y FROM 02763_a_merge +3 13 +4 14 +SELECT x, y FROM merge(currentDatabase(), 02763_alias) +3 13 +4 14 +SELECT x, y FROM merge(currentDatabase(), 02763_alias) +2 12 +3 13 +4 14 +SELECT x FROM merge(currentDatabase(), 02763_alias) +12 +13 +14 +SELECT y FROM merge(currentDatabase(), 02763_alias) +2 +3 +4 diff --git a/tests/queries/0_stateless/02763_row_policy_storage_merge_alias.sql.j2 b/tests/queries/0_stateless/02763_row_policy_storage_merge_alias.sql.j2 new file mode 100644 index 00000000000..bdd456951dd --- /dev/null +++ b/tests/queries/0_stateless/02763_row_policy_storage_merge_alias.sql.j2 @@ -0,0 +1,41 @@ +DROP TABLE IF EXISTS 02763_alias; +DROP TABLE IF EXISTS 02763_a_merge; + + +SELECT '02763_merge_aliases'; +CREATE TABLE 02763_alias (x UInt8, y UInt64, z UInt64 ALIAS plus(x,y)) ENGINE = MergeTree ORDER BY x; +INSERT INTO 02763_alias VALUES (1, 11), (2, 12), (3, 13), (4, 14); + +CREATE ROW POLICY 02763_filter_7 ON 02763_alias USING z>15 AS permissive TO ALL; + +CREATE TABLE 02763_a_merge (x UInt8, y UInt64, z UInt64) ENGINE = Merge(currentDatabase(), '02763_alias'); + +{% for prew in [0 , 1] -%} + + + +SELECT 'x, y, z FROM 02763_a_merge'; +SELECT x, y, z FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT '* FROM 02763_a_merge'; +SELECT * FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'x, y FROM 02763_a_merge'; +SELECT x, y FROM 02763_a_merge ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT x, y FROM merge(currentDatabase(), 02763_alias)'; +SELECT x, y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + +CREATE ROW POLICY 02763_filter_8 ON 02763_alias USING y>11 AS permissive TO ALL; + +SELECT 'SELECT x, y FROM merge(currentDatabase(), 02763_alias)'; +SELECT x, y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT x FROM merge(currentDatabase(), 02763_alias)'; +SELECT y FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; +SELECT 'SELECT y FROM merge(currentDatabase(), 02763_alias)'; +SELECT x FROM merge(currentDatabase(), '02763_alias') ORDER BY x SETTINGS optimize_move_to_prewhere= {{prew}}; + +DROP ROW POLICY 02763_filter_8 ON 02763_alias; +{% endfor %} + +DROP TABLE 02763_alias; +DROP TABLE 02763_a_merge; + +DROP ROW POLICY 02763_filter_7 ON 02763_alias; From f9a8df4296b9c4cc657cb14d6cd2cfaa84d1a6ba Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Thu, 16 Nov 2023 13:57:58 +0100 Subject: [PATCH 576/813] Added comment to prevent using --remote to update submodules --- contrib/update-submodules.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/contrib/update-submodules.sh b/contrib/update-submodules.sh index b612d25352b..6d187f124a4 100755 --- a/contrib/update-submodules.sh +++ b/contrib/update-submodules.sh @@ -9,4 +9,8 @@ cd $GIT_DIR contrib/sparse-checkout/setup-sparse-checkout.sh git submodule init git submodule sync -git config --file .gitmodules --get-regexp .*path | sed 's/[^ ]* //' | xargs -I _ --max-procs 64 git submodule update --depth=1 --single-branch _ +# NOTE: do not use --remote for `git submodule update`[1] command, since the submodule references to the specific commit SHA1 in the subproject. +# It may cause unexpected behavior. Instead you need to commit a new SHA1 for a submodule. +# +# [1] - https://git-scm.com/book/en/v2/Git-Tools-Submodules +git config --file .gitmodules --get-regexp '.*path' | sed 's/[^ ]* //' | xargs -I _ --max-procs 64 git submodule update --depth=1 --single-branch _ From ebb66c1a9e33ccea792ee4ff64519767ea82cd67 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 20 Nov 2023 12:13:24 +0100 Subject: [PATCH 577/813] add comments --- src/IO/Lz4DeflatingWriteBuffer.cpp | 3 +++ src/IO/WriteBuffer.h | 7 ++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 0af205a426d..1f937ac545d 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -6,6 +6,9 @@ namespace { using namespace DB; + /// SinkToOut provides the safe way to do direct write into buffer's memory + /// When out->capacity() is not less that guaranteed_capacity, SinkToOut is pointing directly to out_'s memory. + /// Otherwise the writes are directed to the temporary memory. That data is copied to out_ at finalize call. class SinkToOut { public: diff --git a/src/IO/WriteBuffer.h b/src/IO/WriteBuffer.h index d29ca6d5c6c..67dbb9b2e7a 100644 --- a/src/IO/WriteBuffer.h +++ b/src/IO/WriteBuffer.h @@ -34,7 +34,12 @@ public: void set(Position ptr, size_t size) { BufferBase::set(ptr, size, 0); } /** write the data in the buffer (from the beginning of the buffer to the current position); - * set the position to the beginning; throw an exception, if something is wrong + * set the position to the beginning; throw an exception, if something is wrong. + * + * Next call doesn't guarantee that buffer capacity is regained after. + * Some buffers (i.g WriteBufferFromS3) flush its data only after certain amount of consumed data. + * If direct write is performed into [position(), buffer().end()) and its length is not enough, + * you need to fill it first (i.g with write call), after it the capacity is regained. */ inline void next() { From 088022df5ddad0347d29d31062a861af0c4b9d88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 20 Nov 2023 12:20:26 +0100 Subject: [PATCH 578/813] Add test --- .../0_stateless/02918_sqlite_path_check.reference | 2 ++ .../queries/0_stateless/02918_sqlite_path_check.sh | 13 +++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02918_sqlite_path_check.reference create mode 100755 tests/queries/0_stateless/02918_sqlite_path_check.sh diff --git a/tests/queries/0_stateless/02918_sqlite_path_check.reference b/tests/queries/0_stateless/02918_sqlite_path_check.reference new file mode 100644 index 00000000000..56b832a6469 --- /dev/null +++ b/tests/queries/0_stateless/02918_sqlite_path_check.reference @@ -0,0 +1,2 @@ +SQLite database file path '/etc/passwd' must be inside 'user_files' directory. (PATH_ACCESS_DENIED) +SQLite database file path '../../../../etc/passwd' must be inside 'user_files' directory. (PATH_ACCESS_DENIED) diff --git a/tests/queries/0_stateless/02918_sqlite_path_check.sh b/tests/queries/0_stateless/02918_sqlite_path_check.sh new file mode 100755 index 00000000000..1f250387a71 --- /dev/null +++ b/tests/queries/0_stateless/02918_sqlite_path_check.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function get_exception_message() +{ + $CLICKHOUSE_CLIENT --query "$1" |& head -n1 | sed 's/.*DB::Exception: \(.*\) (version.*/\1/g' +} + +get_exception_message "Select * from sqlite('/etc/passwd', 'something');" +get_exception_message "Select * from sqlite('../../../../etc/passwd', 'something'); From 44874859bb0a27635a590617baa8fc3d957b8dd6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 20 Nov 2023 12:21:12 +0100 Subject: [PATCH 579/813] Fix style --- src/Databases/SQLite/SQLiteUtils.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Databases/SQLite/SQLiteUtils.cpp b/src/Databases/SQLite/SQLiteUtils.cpp index ddc2fb911e9..19b8662707b 100644 --- a/src/Databases/SQLite/SQLiteUtils.cpp +++ b/src/Databases/SQLite/SQLiteUtils.cpp @@ -26,7 +26,6 @@ void processSQLiteError(const String & message, bool throw_on_error) String validateSQLiteDatabasePath(const String & path, const String & user_files_path, bool need_check, bool throw_on_error) { - String absolute_path = fs::absolute(path).lexically_normal(); if (fs::path(path).is_relative()) From 10fb40ece964947fc7e8d32641d8caaa66cd2a81 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Nov 2023 12:34:16 +0100 Subject: [PATCH 580/813] Fix --- src/Storages/S3Queue/S3QueueMetadataFactory.cpp | 1 + src/Storages/S3Queue/StorageS3Queue.cpp | 8 +++++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp index bd01bd52425..92cdab6355d 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp @@ -43,6 +43,7 @@ void S3QueueMetadataFactory::remove(const std::string & zookeeper_path) if (it == metadata_by_path.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata with zookeeper path {} does not exist", zookeeper_path); + chassert(it->second.ref_count > 0); if (--it->second.ref_count == 0) { try diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 72e74d3c2a0..d2cf074e6a1 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -112,7 +112,6 @@ StorageS3Queue::StorageS3Queue( , s3queue_settings(std::move(s3queue_settings_)) , zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *s3queue_settings)) , after_processing(s3queue_settings->after_processing) - , files_metadata(S3QueueMetadataFactory::instance().getOrCreate(zk_path, *s3queue_settings)) , configuration{configuration_} , format_settings(format_settings_) , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) @@ -157,6 +156,13 @@ StorageS3Queue::StorageS3Queue( void StorageS3Queue::startup() { + if (!files_metadata) + { + /// Get metadata manager from S3QueueMetadataFactory, + /// it will increase the ref count for the metadata object. + /// The ref count is decreased when StorageS3Queue::drop() method is called. + files_metadata = S3QueueMetadataFactory::instance().getOrCreate(zk_path, *s3queue_settings); + } if (task) task->activateAndSchedule(); } From 1ade4b797b1e18d82b52a113c74dad9b4e203487 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Nov 2023 12:48:49 +0100 Subject: [PATCH 581/813] Add a comment --- src/Storages/S3Queue/S3QueueMetadataFactory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.h b/src/Storages/S3Queue/S3QueueMetadataFactory.h index 55c2dfad5dd..c5e94d59050 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.h +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.h @@ -25,6 +25,7 @@ private: explicit Metadata(std::shared_ptr metadata_) : metadata(metadata_), ref_count(1) {} std::shared_ptr metadata; + /// TODO: the ref count should be kept in keeper, because of the case with distributed processing. size_t ref_count = 0; }; using MetadataByPath = std::unordered_map; From c3a3cf8d24d5ca1205c2fe9cb2d8d266ea276b00 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 20 Nov 2023 12:57:10 +0100 Subject: [PATCH 582/813] Make check for the limited cmake dependencies the part of sparse checkout --- contrib/update-submodules.sh | 8 ++++++++ docker/packager/binary/build.sh | 9 --------- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/contrib/update-submodules.sh b/contrib/update-submodules.sh index 6d187f124a4..b12f3f924dc 100755 --- a/contrib/update-submodules.sh +++ b/contrib/update-submodules.sh @@ -14,3 +14,11 @@ git submodule sync # # [1] - https://git-scm.com/book/en/v2/Git-Tools-Submodules git config --file .gitmodules --get-regexp '.*path' | sed 's/[^ ]* //' | xargs -I _ --max-procs 64 git submodule update --depth=1 --single-branch _ + +# We don't want to depend on any third-party CMake files. +# To check it, find and delete them. +grep -o -P '"contrib/[^"]+"' .gitmodules | + grep -v -P 'contrib/(llvm-project|google-protobuf|grpc|abseil-cpp|corrosion)' | + xargs -I@ find @ \ + -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' \ + -delete diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index f943011df9d..fd9bfcaabb2 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -34,15 +34,6 @@ cd /build/build_docker rm -f CMakeCache.txt -# We don't want to depend on any third-party CMake files. -# To check it, find and delete them. - -grep -o -P '"contrib/[^"]+"' ../.gitmodules | - grep -v -P 'llvm-project|google-protobuf|grpc|abseil-cpp|corrosion' | - xargs -I@ find ../@ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' | - xargs rm - - if [ -n "$MAKE_DEB" ]; then rm -rf /build/packages/root # NOTE: this is for backward compatibility with previous releases, From 28bb76d56861750a12a223a1da3401bc159fde02 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Nov 2023 13:21:46 +0100 Subject: [PATCH 583/813] address review comments --- .../Passes/LogicalExpressionOptimizerPass.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index e667b603020..dfaccbc5cdb 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -59,9 +59,11 @@ private: static bool tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node, const ContextPtr & context) { auto & function_node = node->as(); - assert(function_node.getFunctionName() == "or"); + chassert(function_node.getFunctionName() == "or"); + QueryTreeNodes or_operands; + or_operands.reserve(function_node.getArguments()->getNodes().size()); /// Indices of `equals` or `isNotDistinctFrom` functions in the vector above std::vector equals_functions_indices; @@ -88,9 +90,10 @@ private: const auto & func_name = argument_function->getFunctionName(); if (func_name == "equals" || func_name == "isNotDistinctFrom") + { equals_functions_indices.push_back(or_operands.size() - 1); - - if (func_name == "and") + } + else if (func_name == "and") { for (const auto & and_argument : argument_function->getArguments().getNodes()) { @@ -169,7 +172,7 @@ private: if (function->getFunctionName() == "equals") { /// We should replace `a = b` with `a <=> b` because we removed checks for IS NULL - need_reresolve = need_reresolve || function->getResultType()->isNullable(); + need_reresolve |= function->getResultType()->isNullable(); function->resolveAsFunction(strict_equals_function_resolver); new_or_operands.emplace_back(std::move(or_operands[i])); } From bf141f5affb23a78c5a393465b592f5fda896cf6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 20 Nov 2023 13:25:51 +0100 Subject: [PATCH 584/813] Update src/Core/Settings.h Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- 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 f1379d162ae..fd8637ca6b2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -791,7 +791,7 @@ class IColumn; M(Bool, optimize_uniq_to_count, true, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) \ \ /** Experimental functions */ \ - M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the materialized PostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ + M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ From 779a8971e05cfbc0dc43aecb8e25e98a1f8c94c6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 20 Nov 2023 13:33:39 +0100 Subject: [PATCH 585/813] Disable settings randomisation for `02896_memory_accounting_for_user.sh` (#56709) --- tests/queries/0_stateless/02896_memory_accounting_for_user.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02896_memory_accounting_for_user.sh b/tests/queries/0_stateless/02896_memory_accounting_for_user.sh index 72f4be1475d..f3016671420 100755 --- a/tests/queries/0_stateless/02896_memory_accounting_for_user.sh +++ b/tests/queries/0_stateless/02896_memory_accounting_for_user.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, long +# Tags: no-parallel, long, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From aa07403f2773d4abc50beb9f2674867f16a8b5ea Mon Sep 17 00:00:00 2001 From: Kevin Mingtarja Date: Mon, 20 Nov 2023 12:37:23 +0000 Subject: [PATCH 586/813] Make test 02918 executable --- ...2918_implicit_sign_column_constraints_for_collapsing_engine.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh diff --git a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh old mode 100644 new mode 100755 From 06c6282eb74d9b9292548f698f5c73b0362bb339 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Nov 2023 13:46:37 +0100 Subject: [PATCH 587/813] fix build --- src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index dfaccbc5cdb..081a27eb8fa 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -63,7 +63,7 @@ private: QueryTreeNodes or_operands; - or_operands.reserve(function_node.getArguments()->getNodes().size()); + or_operands.reserve(function_node.getArguments().getNodes().size()); /// Indices of `equals` or `isNotDistinctFrom` functions in the vector above std::vector equals_functions_indices; From fafd169e7b313c5e0a5046afe0183778e828edeb Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Mon, 20 Nov 2023 14:12:52 +0100 Subject: [PATCH 588/813] Update src/IO/Lz4DeflatingWriteBuffer.cpp Co-authored-by: Antonio Andelic --- src/IO/Lz4DeflatingWriteBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 1f937ac545d..8241bfd4f3c 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -7,7 +7,7 @@ namespace using namespace DB; /// SinkToOut provides the safe way to do direct write into buffer's memory - /// When out->capacity() is not less that guaranteed_capacity, SinkToOut is pointing directly to out_'s memory. + /// When out->capacity() is not less than guaranteed_capacity, SinkToOut is pointing directly to out_'s memory. /// Otherwise the writes are directed to the temporary memory. That data is copied to out_ at finalize call. class SinkToOut { From 5031f239c3ecb92d6da853ae2e14b82eadc0ba43 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 20 Nov 2023 14:28:59 +0100 Subject: [PATCH 589/813] Revert "s3 adaptive timeouts" --- base/poco/Net/src/HTTPServerSession.cpp | 1 + base/poco/Net/src/HTTPSession.cpp | 31 +---- docs/en/operations/settings/settings.md | 7 - src/Backups/BackupIO_S3.cpp | 9 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 1 + src/Core/Settings.h | 3 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 58 +++++--- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 15 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 8 +- src/IO/ConnectionTimeouts.cpp | 82 ----------- src/IO/ConnectionTimeouts.h | 2 - src/IO/HTTPCommon.cpp | 12 +- src/IO/HTTPCommon.h | 2 - src/IO/ReadBufferFromS3.cpp | 24 ++-- src/IO/ReadBufferFromS3.h | 4 +- src/IO/S3/Client.cpp | 12 +- src/IO/S3/Client.h | 10 +- src/IO/S3/PocoHTTPClient.cpp | 111 ++++----------- src/IO/S3/PocoHTTPClient.h | 5 - src/IO/S3/copyS3File.cpp | 26 ++-- src/IO/S3/copyS3File.h | 7 + src/IO/S3/tests/gtest_aws_s3_client.cpp | 3 +- src/IO/WriteBufferFromS3.cpp | 4 +- src/IO/WriteBufferFromS3.h | 3 + src/IO/tests/gtest_writebuffer_s3.cpp | 1 + src/Storages/StorageS3.cpp | 3 + src/Storages/StorageS3.h | 1 + src/Storages/StorageS3Settings.h | 3 +- .../configs/inf_s3_retries.xml | 1 - .../configs/s3_retries.xml | 1 - .../configs/storage_conf.xml | 16 +-- .../test_checking_s3_blobs_paranoid/test.py | 129 ++++-------------- .../configs/config.d/storage_conf.xml | 2 - .../test_storage_s3/configs/defaultS3.xml | 5 + .../test_storage_s3/configs/s3_retry.xml | 4 +- .../s3_mocks/unstable_server.py | 17 +-- tests/integration/test_storage_s3/test.py | 9 -- 37 files changed, 202 insertions(+), 430 deletions(-) diff --git a/base/poco/Net/src/HTTPServerSession.cpp b/base/poco/Net/src/HTTPServerSession.cpp index d4f2b24879e..f6d3c4e5b92 100644 --- a/base/poco/Net/src/HTTPServerSession.cpp +++ b/base/poco/Net/src/HTTPServerSession.cpp @@ -26,6 +26,7 @@ HTTPServerSession::HTTPServerSession(const StreamSocket& socket, HTTPServerParam _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) { setTimeout(pParams->getTimeout()); + this->socket().setReceiveTimeout(pParams->getTimeout()); } diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 8f951b3102c..d2663baaf9f 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -93,34 +93,9 @@ void HTTPSession::setTimeout(const Poco::Timespan& timeout) void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco::Timespan& sendTimeout, const Poco::Timespan& receiveTimeout) { - try - { - _connectionTimeout = connectionTimeout; - - if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) { - _sendTimeout = sendTimeout; - - if (connected()) - _socket.setSendTimeout(_sendTimeout); - } - - if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) { - _receiveTimeout = receiveTimeout; - - if (connected()) - _socket.setReceiveTimeout(_receiveTimeout); - } - } - catch (NetException &) - { -#ifndef NDEBUG - throw; -#else - // mute exceptions in release - // just in case when changing settings on socket is not allowed - // however it should be OK for timeouts -#endif - } + _connectionTimeout = connectionTimeout; + _sendTimeout = sendTimeout; + _receiveTimeout = receiveTimeout; } diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index edc1c9bdfd7..e61934d2168 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4826,10 +4826,3 @@ When set to `true` the metadata files are written with `VERSION_FULL_OBJECT_KEY` When set to `false` the metadata files are written with the previous format version, `VERSION_INLINE_DATA`. With that format only suffixes of object storage key names are are written to the metadata files. The prefix for all of object storage key names is set in configurations files at `storage_configuration.disks` section. Default value: `false`. - -## s3_use_adaptive_timeouts {#s3_use_adaptive_timeouts} - -When set to `true` than for all s3 requests first two attempts are made with low send and receive timeouts. -When set to `false` than all attempts are made with identical timeouts. - -Default value: `true`. diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index ea3f57c27ff..eb9dcf6b45a 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -55,9 +55,7 @@ namespace static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false, - request_settings.get_request_throttler, - request_settings.put_request_throttler, + /* for_disk_s3 = */ false, request_settings.get_request_throttler, request_settings.put_request_throttler, s3_uri.uri.getScheme()); client_configuration.endpointOverride = s3_uri.endpoint; @@ -169,6 +167,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s blob_path.size(), mode); copyS3File( + client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -230,6 +229,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src { LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( + client, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], @@ -268,7 +268,7 @@ void BackupWriterS3::copyFile(const String & destination, const String & source, void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { - copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, s3_settings.request_settings, {}, + copyDataToS3File(create_read_buffer, start_pos, length, client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, s3_settings.request_settings, {}, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); } @@ -298,6 +298,7 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) { return std::make_unique( client, + client, // already has long timeout s3_uri.bucket, fs::path(s3_uri.key) / file_name, DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index bedde0d7b39..302e05c8418 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -148,6 +148,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapsh const auto create_writer = [&](const auto & key) { return WriteBufferFromS3( + s3_client->client, s3_client->client, s3_client->uri.bucket, key, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bb5e4322485..ac4c6b6c17f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,7 +94,6 @@ class IColumn; 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(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) \ @@ -105,7 +104,7 @@ class IColumn; 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, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 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_request_timeout_ms, 3000, "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_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. Only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 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) \ diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 308db389ee1..3af316bf0cf 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -155,7 +155,7 @@ private: bool S3ObjectStorage::exists(const StoredObject & object) const { auto settings_ptr = s3_settings.get(); - return S3::objectExists(*client.get(), bucket, object.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + return S3::objectExists(*clients.get()->client, bucket, object.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); } std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT @@ -174,7 +174,7 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT (const std::string & path, size_t read_until_position) -> std::unique_ptr { return std::make_unique( - client.get(), + clients.get()->client, bucket, path, version_id, @@ -224,7 +224,7 @@ std::unique_ptr S3ObjectStorage::readObject( /// NOLINT { auto settings_ptr = s3_settings.get(); return std::make_unique( - client.get(), + clients.get()->client, bucket, object.remote_path, version_id, @@ -249,8 +249,10 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); + auto clients_ = clients.get(); return std::make_unique( - client.get(), + clients_->client, + clients_->client_with_long_timeout, bucket, object.remote_path, buf_size, @@ -264,12 +266,15 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix) const { auto settings_ptr = s3_settings.get(); - return std::make_shared(bucket, path_prefix, client.get(), settings_ptr->list_object_keys_size); + auto client_ptr = clients.get()->client; + + return std::make_shared(bucket, path_prefix, client_ptr, settings_ptr->list_object_keys_size); } void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto settings_ptr = s3_settings.get(); + auto client_ptr = clients.get()->client; S3::ListObjectsV2Request request; request.SetBucket(bucket); @@ -284,7 +289,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet { ProfileEvents::increment(ProfileEvents::S3ListObjects); ProfileEvents::increment(ProfileEvents::DiskS3ListObjects); - outcome = client.get()->ListObjectsV2(request); + outcome = client_ptr->ListObjectsV2(request); throwIfError(outcome); auto result = outcome.GetResult(); @@ -315,12 +320,14 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exists) { + auto client_ptr = clients.get()->client; + ProfileEvents::increment(ProfileEvents::S3DeleteObjects); ProfileEvents::increment(ProfileEvents::DiskS3DeleteObjects); S3::DeleteObjectRequest request; request.SetBucket(bucket); request.SetKey(object.remote_path); - auto outcome = client.get()->DeleteObject(request); + auto outcome = client_ptr->DeleteObject(request); throwIfUnexpectedError(outcome, if_exists); @@ -339,6 +346,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e } else { + auto client_ptr = clients.get()->client; auto settings_ptr = s3_settings.get(); size_t chunk_size_limit = settings_ptr->objects_chunk_size_to_delete; @@ -367,7 +375,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e S3::DeleteObjectsRequest request; request.SetBucket(bucket); request.SetDelete(delkeys); - auto outcome = client.get()->DeleteObjects(request); + auto outcome = client_ptr->DeleteObjects(request); throwIfUnexpectedError(outcome, if_exists); @@ -399,7 +407,7 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); + auto object_info = S3::getObjectInfo(*clients.get()->client, bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) return {}; @@ -415,7 +423,7 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true); + auto object_info = S3::getObjectInfo(*clients.get()->client, bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true); ObjectMetadata result; result.size_bytes = object_info.size; @@ -436,12 +444,12 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT /// Shortcut for S3 if (auto * dest_s3 = dynamic_cast(&object_storage_to); dest_s3 != nullptr) { - auto client_ = client.get(); + auto clients_ = clients.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*client_, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File( - client.get(), + copyS3File(clients_->client, + clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, @@ -465,11 +473,12 @@ void S3ObjectStorage::copyObject( // NOLINT const WriteSettings &, std::optional object_to_attributes) { - auto client_ = client.get(); + auto clients_ = clients.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*client_, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_, + copyS3File(clients_->client, + clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, @@ -490,25 +499,31 @@ void S3ObjectStorage::setNewSettings(std::unique_ptr && void S3ObjectStorage::shutdown() { + auto clients_ptr = clients.get(); /// This call stops any next retry attempts for ongoing S3 requests. /// If S3 request is failed and the method below is executed S3 client immediately returns the last failed S3 request outcome. /// If S3 is healthy nothing wrong will be happened and S3 requests will be processed in a regular way without errors. /// This should significantly speed up shutdown process if S3 is unhealthy. - const_cast(*client.get()).DisableRequestProcessing(); + const_cast(*clients_ptr->client).DisableRequestProcessing(); + const_cast(*clients_ptr->client_with_long_timeout).DisableRequestProcessing(); } void S3ObjectStorage::startup() { + auto clients_ptr = clients.get(); + /// Need to be enabled if it was disabled during shutdown() call. - const_cast(*client.get()).EnableRequestProcessing(); + const_cast(*clients_ptr->client).EnableRequestProcessing(); + const_cast(*clients_ptr->client_with_long_timeout).EnableRequestProcessing(); } void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, 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); + auto new_clients = std::make_unique(std::move(new_client), *new_s3_settings); s3_settings.set(std::move(new_s3_settings)); - client.set(std::move(new_client)); + clients.set(std::move(new_clients)); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( @@ -523,6 +538,9 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( endpoint, object_key_prefix); } +S3ObjectStorage::Clients::Clients(std::shared_ptr client_, const S3ObjectStorageSettings & settings) + : client(std::move(client_)), client_with_long_timeout(client->clone(std::nullopt, settings.request_settings.long_request_timeout_ms)) {} + ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string &) const { /// Path to store the new S3 object. diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 7d14482311f..b1b3fb22366 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -39,6 +39,16 @@ struct S3ObjectStorageSettings class S3ObjectStorage : public IObjectStorage { +public: + struct Clients + { + std::shared_ptr client; + std::shared_ptr client_with_long_timeout; + + Clients() = default; + Clients(std::shared_ptr client, const S3ObjectStorageSettings & settings); + }; + private: friend class S3PlainObjectStorage; @@ -53,7 +63,7 @@ private: String object_key_prefix_) : bucket(std::move(bucket_)) , object_key_prefix(std::move(object_key_prefix_)) - , client(std::move(client_)) + , clients(std::make_unique(std::move(client_), *s3_settings_)) , s3_settings(std::move(s3_settings_)) , s3_capabilities(s3_capabilities_) , version_id(std::move(version_id_)) @@ -174,8 +184,7 @@ private: std::string bucket; String object_key_prefix; - - MultiVersion client; + MultiVersion clients; MultiVersion s3_settings; S3Capabilities s3_capabilities; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 0232a6eb070..de88c876922 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -60,15 +60,13 @@ std::unique_ptr getClient( uri.uri.getScheme()); client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 1000); - client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 30000); + client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 3000); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100); client_configuration.endpointOverride = uri.endpoint; - client_configuration.http_keep_alive_timeout_ms = config.getUInt( - config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); + client_configuration.http_keep_alive_timeout_ms + = config.getUInt(config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); client_configuration.wait_on_pool_size_limit = false; - client_configuration.s3_use_adaptive_timeouts = config.getBool( - config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); /* * Override proxy configuration for backwards compatibility with old configuration format. diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 970afc75ec3..01fbaa4f817 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -133,86 +133,4 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings settings.http_receive_timeout); } -class SendReceiveTimeoutsForFirstAttempt -{ -private: - static constexpr size_t known_methods_count = 6; - using KnownMethodsArray = std::array; - static const KnownMethodsArray known_methods; - - /// HTTP_POST is used for CompleteMultipartUpload requests. Its latency could be high. - /// These requests need longer timeout, especially when minio is used. - /// The same assumption are made for HTTP_DELETE, HTTP_PATCH - /// That requests are more heavy that HTTP_GET, HTTP_HEAD, HTTP_PUT - - static constexpr Poco::Timestamp::TimeDiff first_byte_ms[known_methods_count][2] = - { - /* GET */ {200, 200}, - /* POST */ {200, 200}, - /* DELETE */ {200, 200}, - /* PUT */ {200, 200}, - /* HEAD */ {200, 200}, - /* PATCH */ {200, 200}, - }; - - static constexpr Poco::Timestamp::TimeDiff rest_bytes_ms[known_methods_count][2] = - { - /* GET */ {500, 500}, - /* POST */ {1000, 30000}, - /* DELETE */ {1000, 10000}, - /* PUT */ {1000, 3000}, - /* HEAD */ {500, 500}, - /* PATCH */ {1000, 10000}, - }; - - static_assert(sizeof(first_byte_ms) == sizeof(rest_bytes_ms)); - static_assert(sizeof(first_byte_ms) == known_methods_count * sizeof(Poco::Timestamp::TimeDiff) * 2); - - static size_t getMethodIndex(const String & method) - { - KnownMethodsArray::const_iterator it = std::find(known_methods.begin(), known_methods.end(), method); - chassert(it != known_methods.end()); - if (it == known_methods.end()) - return 0; - return std::distance(known_methods.begin(), it); - } - -public: - static std::pair getSendReceiveTimeout(const String & method, bool first_byte) - { - auto idx = getMethodIndex(method); - - if (first_byte) - return std::make_pair( - Poco::Timespan(first_byte_ms[idx][0] * 1000), - Poco::Timespan(first_byte_ms[idx][1] * 1000) - ); - - return std::make_pair( - Poco::Timespan(rest_bytes_ms[idx][0] * 1000), - Poco::Timespan(rest_bytes_ms[idx][1] * 1000) - ); - } -}; - -const SendReceiveTimeoutsForFirstAttempt::KnownMethodsArray SendReceiveTimeoutsForFirstAttempt::known_methods = -{ - "GET", "POST", "DELETE", "PUT", "HEAD", "PATCH" -}; - - -ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(const String & method, bool first_attempt, bool first_byte) const -{ - if (!first_attempt) - return *this; - - auto [send, recv] = SendReceiveTimeoutsForFirstAttempt::getSendReceiveTimeout(method, first_byte); - - auto aggressive = *this; - aggressive.send_timeout = saturate(send, send_timeout); - aggressive.receive_timeout = saturate(recv, receive_timeout); - - return aggressive; -} - } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index aabebdb836d..684af42827f 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -67,8 +67,6 @@ struct ConnectionTimeouts /// Timeouts for the case when we will try many addresses in a loop. static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings); static ConnectionTimeouts getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout); - - ConnectionTimeouts getAdaptiveTimeouts(const String & method, bool first_attempt, bool first_byte) const; }; } diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index cce394c67c9..65ffa51a466 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -50,6 +50,12 @@ namespace ErrorCodes namespace { + void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) + { + session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); + session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); + } + Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const ProxyConfiguration & proxy_configuration) { Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; @@ -353,12 +359,6 @@ namespace }; } -void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) -{ - session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); - session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); -} - void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout) { if (!response.getKeepAlive()) diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index c9968fc6915..de62b5d5c16 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -113,6 +113,4 @@ std::istream * receiveResponse( void assertResponseIsOk( const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false); - -void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index c9c9319c44c..f19978ccb47 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -167,9 +167,9 @@ bool ReadBufferFromS3::nextImpl() } size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 1; !next_result; ++attempt) + for (size_t attempt = 0; !next_result; ++attempt) { - bool last_attempt = attempt >= request_settings.max_single_read_retries; + bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); @@ -177,7 +177,7 @@ bool ReadBufferFromS3::nextImpl() { if (!impl) { - impl = initialize(attempt); + impl = initialize(); if (use_external_buffer) { @@ -232,9 +232,9 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons { size_t initial_n = n; size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 1; n > 0; ++attempt) + for (size_t attempt = 0; n > 0; ++attempt) { - bool last_attempt = attempt >= request_settings.max_single_read_retries; + bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; size_t bytes_copied = 0; ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); @@ -266,7 +266,7 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons try { - result = sendRequest(attempt, range_begin, range_begin + n - 1); + result = sendRequest(range_begin, range_begin + n - 1); std::istream & istr = result->GetBody(); copyFromIStreamWithProgressCallback(istr, to, n, progress_callback, &bytes_copied); @@ -304,8 +304,8 @@ bool ReadBufferFromS3::processException(Poco::Exception & e, size_t read_offset, LOG_DEBUG( log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, " - "Attempt: {}/{}, Message: {}", - bucket, key, version_id.empty() ? "Latest" : version_id, read_offset, attempt, request_settings.max_single_read_retries, e.message()); + "Attempt: {}, Message: {}", + bucket, key, version_id.empty() ? "Latest" : version_id, read_offset, attempt, e.message()); if (auto * s3_exception = dynamic_cast(&e)) @@ -463,7 +463,7 @@ ReadBufferFromS3::~ReadBufferFromS3() } } -std::unique_ptr ReadBufferFromS3::initialize(size_t attempt) +std::unique_ptr ReadBufferFromS3::initialize() { resetSessionIfNeeded(readAllRangeSuccessfully(), read_result); read_all_range_successfully = false; @@ -475,13 +475,13 @@ std::unique_ptr ReadBufferFromS3::initialize(size_t attempt) if (read_until_position && offset >= read_until_position) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); - read_result = sendRequest(attempt, offset, read_until_position ? std::make_optional(read_until_position - 1) : std::nullopt); + read_result = sendRequest(offset, read_until_position ? std::make_optional(read_until_position - 1) : std::nullopt); size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; return std::make_unique(read_result->GetBody(), buffer_size); } -Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, size_t range_begin, std::optional range_end_incl) const +Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin, std::optional range_end_incl) const { S3::GetObjectRequest req; req.SetBucket(bucket); @@ -489,8 +489,6 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, si if (!version_id.empty()) req.SetVersionId(version_id); - req.SetAdditionalCustomHeaderValue("clickhouse-request", fmt::format("attempt={}", attempt)); - if (range_end_incl) { req.SetRange(fmt::format("bytes={}-{}", range_begin, *range_end_incl)); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 101e25f8b43..0835e52a5b2 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -79,7 +79,7 @@ public: bool supportsReadAt() override { return true; } private: - std::unique_ptr initialize(size_t attempt); + std::unique_ptr initialize(); /// If true, if we destroy impl now, no work was wasted. Just for metrics. bool atEndOfRequestedRangeGuess(); @@ -88,7 +88,7 @@ private: /// Returns true if the error looks retriable. bool processException(Poco::Exception & e, size_t read_offset, size_t attempt) const; - Aws::S3::Model::GetObjectResult sendRequest(size_t attempt, size_t range_begin, std::optional range_end_incl) const; + Aws::S3::Model::GetObjectResult sendRequest(size_t range_begin, std::optional range_end_incl) const; bool readAllRangeSuccessfully() const; diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 4630e68fbb6..ceb7d275299 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -118,9 +118,16 @@ std::unique_ptr Client::create( new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, use_virtual_addressing)); } -std::unique_ptr Client::clone() const +std::unique_ptr Client::clone( + std::optional> override_retry_strategy, + std::optional override_request_timeout_ms) const { - return std::unique_ptr(new Client(*this, client_configuration)); + PocoHTTPClientConfiguration new_configuration = client_configuration; + if (override_retry_strategy.has_value()) + new_configuration.retryStrategy = *override_retry_strategy; + if (override_request_timeout_ms.has_value()) + new_configuration.requestTimeoutMs = *override_request_timeout_ms; + return std::unique_ptr(new Client(*this, new_configuration)); } namespace @@ -898,7 +905,6 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT s3_retry_attempts, enable_s3_requests_logging, for_disk_s3, - context->getGlobalContext()->getSettingsRef().s3_use_adaptive_timeouts, get_request_throttler, put_request_throttler, error_report); diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 5ad57a9d827..48310bc21af 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -118,7 +118,15 @@ public: Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, bool use_virtual_addressing); - std::unique_ptr clone() const; + /// Create a client with adjusted settings: + /// * override_retry_strategy can be used to disable retries to avoid nested retries when we have + /// a retry loop outside of S3 client. Specifically, for read and write buffers. Currently not + /// actually used. + /// * override_request_timeout_ms is used to increase timeout for CompleteMultipartUploadRequest + /// because it often sits idle for 10 seconds: https://github.com/ClickHouse/ClickHouse/pull/42321 + std::unique_ptr clone( + std::optional> override_retry_strategy = std::nullopt, + std::optional override_request_timeout_ms = std::nullopt) const; Client & operator=(const Client &) = delete; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 4a1b6def133..d0f248f48a6 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -99,7 +99,6 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( unsigned int s3_retry_attempts_, bool enable_s3_requests_logging_, bool for_disk_s3_, - bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, std::function error_report_) @@ -112,7 +111,6 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , for_disk_s3(for_disk_s3_) , get_request_throttler(get_request_throttler_) , put_request_throttler(put_request_throttler_) - , s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_) , error_report(error_report_) { } @@ -159,7 +157,6 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config Poco::Timespan(client_configuration.http_keep_alive_timeout_ms * 1000))) /// flag indicating whether keep-alive is enabled is set to each session upon creation , remote_host_filter(client_configuration.remote_host_filter) , s3_max_redirects(client_configuration.s3_max_redirects) - , s3_use_adaptive_timeouts(client_configuration.s3_use_adaptive_timeouts) , enable_s3_requests_logging(client_configuration.enable_s3_requests_logging) , for_disk_s3(client_configuration.for_disk_s3) , get_request_throttler(client_configuration.get_request_throttler) @@ -271,38 +268,6 @@ void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricT ProfileEvents::increment(disk_s3_events_map[static_cast(type)][static_cast(kind)], amount); } -String extractAttemptFromInfo(const Aws::String & request_info) -{ - static auto key = Aws::String("attempt="); - - auto key_begin = request_info.find(key, 0); - if (key_begin == Aws::String::npos) - return "1"; - - auto val_begin = key_begin + key.size(); - auto val_end = request_info.find(';', val_begin); - if (val_end == Aws::String::npos) - val_end = request_info.size(); - - return request_info.substr(val_begin, val_end-val_begin); -} - -String getOrEmpty(const Aws::Http::HeaderValueCollection & map, const String & key) -{ - auto it = map.find(key); - if (it == map.end()) - return {}; - return it->second; -} - -ConnectionTimeouts PocoHTTPClient::getTimeouts(const String & method, bool first_attempt, bool first_byte) const -{ - if (!s3_use_adaptive_timeouts) - return timeouts; - - return timeouts.getAdaptiveTimeouts(method, first_attempt, first_byte); -} - void PocoHTTPClient::makeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, @@ -317,25 +282,6 @@ void PocoHTTPClient::makeRequestInternal( makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); } -String getMethod(const Aws::Http::HttpRequest & request) -{ - switch (request.GetMethod()) - { - case Aws::Http::HttpMethod::HTTP_GET: - return Poco::Net::HTTPRequest::HTTP_GET; - case Aws::Http::HttpMethod::HTTP_POST: - return Poco::Net::HTTPRequest::HTTP_POST; - case Aws::Http::HttpMethod::HTTP_DELETE: - return Poco::Net::HTTPRequest::HTTP_DELETE; - case Aws::Http::HttpMethod::HTTP_PUT: - return Poco::Net::HTTPRequest::HTTP_PUT; - case Aws::Http::HttpMethod::HTTP_HEAD: - return Poco::Net::HTTPRequest::HTTP_HEAD; - case Aws::Http::HttpMethod::HTTP_PATCH: - return Poco::Net::HTTPRequest::HTTP_PATCH; - } -} - template void PocoHTTPClient::makeRequestInternalImpl( Aws::Http::HttpRequest & request, @@ -349,14 +295,9 @@ void PocoHTTPClient::makeRequestInternalImpl( Poco::Logger * log = &Poco::Logger::get("AWSClient"); auto uri = request.GetUri().GetURIString(); - auto method = getMethod(request); - - auto sdk_attempt = extractAttemptFromInfo(getOrEmpty(request.GetHeaders(), Aws::Http::SDK_REQUEST_HEADER)); - auto ch_attempt = extractAttemptFromInfo(getOrEmpty(request.GetHeaders(), "clickhouse-request")); - bool first_attempt = ch_attempt == "1" && sdk_attempt == "1"; if (enable_s3_requests_logging) - LOG_TEST(log, "Make request to: {}, aws sdk attempt: {}, clickhouse attempt: {}", uri, sdk_attempt, ch_attempt); + LOG_TEST(log, "Make request to: {}", uri); switch (request.GetMethod()) { @@ -407,29 +348,17 @@ void PocoHTTPClient::makeRequestInternalImpl( /// This can lead to request signature difference on S3 side. if constexpr (pooled) session = makePooledHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - http_connection_pool_size, - wait_on_pool_size_limit, - proxy_configuration); + target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit, proxy_configuration); else - session = makeHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - proxy_configuration); + session = makeHTTPSession(target_uri, timeouts, proxy_configuration); } else { if constexpr (pooled) session = makePooledHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true), - http_connection_pool_size, - wait_on_pool_size_limit); + target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit); else - session = makeHTTPSession( - target_uri, - getTimeouts(method, first_attempt, /*first_byte*/ true)); + session = makeHTTPSession(target_uri, timeouts); } /// In case of error this address will be written to logs @@ -463,7 +392,28 @@ void PocoHTTPClient::makeRequestInternalImpl( path_and_query = "/"; poco_request.setURI(path_and_query); - poco_request.setMethod(method); + + switch (request.GetMethod()) + { + case Aws::Http::HttpMethod::HTTP_GET: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_GET); + break; + case Aws::Http::HttpMethod::HTTP_POST: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_POST); + break; + case Aws::Http::HttpMethod::HTTP_DELETE: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); + break; + case Aws::Http::HttpMethod::HTTP_PUT: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); + break; + case Aws::Http::HttpMethod::HTTP_HEAD: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); + break; + case Aws::Http::HttpMethod::HTTP_PATCH: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); + break; + } /// Headers coming from SDK are lower-cased. for (const auto & [header_name, header_value] : request.GetHeaders()) @@ -488,7 +438,6 @@ void PocoHTTPClient::makeRequestInternalImpl( request.GetContentBody()->clear(); request.GetContentBody()->seekg(0); - setTimeouts(*session, getTimeouts(method, first_attempt, /*first_byte*/ false)); auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); if (enable_s3_requests_logging) LOG_TEST(log, "Written {} bytes to request body", size); @@ -498,8 +447,6 @@ void PocoHTTPClient::makeRequestInternalImpl( LOG_TEST(log, "Receiving response..."); auto & response_body_stream = session->receiveResponse(poco_response); - setTimeouts(*session, getTimeouts(method, first_attempt, /*first_byte*/ false)); - watch.stop(); addMetric(request, S3MetricType::Microseconds, watch.elapsedMicroseconds()); @@ -551,7 +498,6 @@ void PocoHTTPClient::makeRequestInternalImpl( /// Request is successful but for some special requests we can have actual error message in body if (status_code >= SUCCESS_RESPONSE_MIN && status_code <= SUCCESS_RESPONSE_MAX && checkRequestCanReturn2xxAndErrorInBody(request)) { - /// reading the full response std::string response_string((std::istreambuf_iterator(response_body_stream)), std::istreambuf_iterator()); @@ -566,6 +512,7 @@ void PocoHTTPClient::makeRequestInternalImpl( addMetric(request, S3MetricType::Errors); if (error_report) error_report(proxy_configuration); + } /// Set response from string @@ -584,8 +531,6 @@ void PocoHTTPClient::makeRequestInternalImpl( if (status_code >= 500 && error_report) error_report(proxy_configuration); } - - /// expose stream, after that client reads data from that stream without built-in retries response->SetResponseBody(response_body_stream, session); } diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 5178d75e7b6..2a449458360 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -55,7 +55,6 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration size_t http_connection_pool_size = 0; /// See PoolBase::BehaviourOnLimit bool wait_on_pool_size_limit = true; - bool s3_use_adaptive_timeouts = true; std::function error_report; @@ -70,7 +69,6 @@ private: unsigned int s3_retry_attempts, bool enable_s3_requests_logging_, bool for_disk_s3_, - bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, std::function error_report_ @@ -171,8 +169,6 @@ private: Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; - ConnectionTimeouts getTimeouts(const String & method, bool first_attempt, bool first_byte) const; - protected: static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; @@ -182,7 +178,6 @@ protected: ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; - bool s3_use_adaptive_timeouts = true; bool enable_s3_requests_logging; bool for_disk_s3; diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 30da1c580c1..a16a1a41505 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -53,6 +53,7 @@ namespace public: UploadHelper( const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, @@ -61,6 +62,7 @@ namespace bool for_disk_s3_, const Poco::Logger * log_) : client_ptr(client_ptr_) + , client_with_long_timeout_ptr(client_with_long_timeout_ptr_) , dest_bucket(dest_bucket_) , dest_key(dest_key_) , request_settings(request_settings_) @@ -76,6 +78,7 @@ namespace protected: std::shared_ptr client_ptr; + std::shared_ptr client_with_long_timeout_ptr; const String & dest_bucket; const String & dest_key; const S3Settings::RequestSettings & request_settings; @@ -176,7 +179,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); - auto outcome = client_ptr->CompleteMultipartUpload(request); + auto outcome = client_with_long_timeout_ptr->CompleteMultipartUpload(request); if (outcome.IsSuccess()) { @@ -430,13 +433,14 @@ namespace size_t offset_, size_t size_, const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) + : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) , create_read_buffer(create_read_buffer_) , offset(offset_) , size(size_) @@ -598,6 +602,7 @@ namespace public: CopyFileHelper( const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & src_bucket_, const String & src_key_, size_t src_offset_, @@ -609,7 +614,7 @@ namespace const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) + : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) , src_bucket(src_bucket_) , src_key(src_key_) , offset(src_offset_) @@ -672,7 +677,7 @@ namespace /// 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 request.SetContentType("binary/octet-stream"); - client_ptr->setKMSHeaders(request); + client_with_long_timeout_ptr->setKMSHeaders(request); } void processCopyRequest(const S3::CopyObjectRequest & request) @@ -684,7 +689,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CopyObject); - auto outcome = client_ptr->CopyObject(request); + auto outcome = client_with_long_timeout_ptr->CopyObject(request); if (outcome.IsSuccess()) { LOG_TRACE( @@ -709,6 +714,7 @@ namespace offset, size, client_ptr, + client_with_long_timeout_ptr, dest_bucket, dest_key, request_settings, @@ -782,7 +788,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy); - auto outcome = client_ptr->UploadPartCopy(req); + auto outcome = client_with_long_timeout_ptr->UploadPartCopy(req); if (!outcome.IsSuccess()) { abortMultipartUpload(); @@ -800,6 +806,7 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, + const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, @@ -807,13 +814,14 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule, bool for_disk_s3) { - CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } void copyS3File( const std::shared_ptr & s3_client, + const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -828,7 +836,7 @@ void copyS3File( { if (settings.allow_native_copy) { - CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, read_settings, object_metadata, schedule, for_disk_s3}; + CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, read_settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } else @@ -837,7 +845,7 @@ void copyS3File( { return std::make_unique(s3_client, src_bucket, src_key, "", settings, read_settings); }; - copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 33e22fdfba2..1bcbfd7735e 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -27,9 +27,15 @@ using CreateReadBuffer = std::function()>; /// because it is a known issue, it is fallbacks to read-write copy /// (copyDataToS3File()). /// +/// s3_client_with_long_timeout (may be equal to s3_client) is used for native copy and +/// CompleteMultipartUpload requests. These requests need longer timeout because S3 servers often +/// block on them for multiple seconds without sending or receiving data from us (maybe the servers +/// are copying data internally, or maybe throttling, idk). +/// /// read_settings - is used for throttling in case of native copy is not possible void copyS3File( const std::shared_ptr & s3_client, + const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -52,6 +58,7 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, + const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index bff9ca6fa7b..c42f14e9a53 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -91,6 +91,7 @@ void doWriteRequest(std::shared_ptr client, const DB::S3:: DB::S3Settings::RequestSettings request_settings; request_settings.max_unexpected_write_error_retries = max_unexpected_write_error_retries; DB::WriteBufferFromS3 write_buffer( + client, client, uri.bucket, uri.key, @@ -170,7 +171,6 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeadersRead) "authorization: ... SignedHeaders=" "amz-sdk-invocation-id;" "amz-sdk-request;" - "clickhouse-request;" "content-type;" "host;" "x-amz-api-version;" @@ -216,7 +216,6 @@ TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersRead) "authorization: ... SignedHeaders=" "amz-sdk-invocation-id;" "amz-sdk-request;" - "clickhouse-request;" "content-type;" "host;" "x-amz-api-version;" diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 62d0c80f1f2..e1b9c17efe9 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -77,6 +77,7 @@ struct WriteBufferFromS3::PartData WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, + std::shared_ptr client_with_long_timeout_ptr_, const String & bucket_, const String & key_, size_t buf_size_, @@ -91,6 +92,7 @@ WriteBufferFromS3::WriteBufferFromS3( , upload_settings(request_settings.getUploadSettings()) , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) + , client_with_long_timeout_ptr(std::move(client_with_long_timeout_ptr_)) , object_metadata(std::move(object_metadata_)) , buffer_allocation_policy(ChooseBufferPolicy(upload_settings)) , task_tracker( @@ -564,7 +566,7 @@ void WriteBufferFromS3::completeMultipartUpload() ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); Stopwatch watch; - auto outcome = client_ptr->CompleteMultipartUpload(req); + auto outcome = client_with_long_timeout_ptr->CompleteMultipartUpload(req); watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 590342cc997..95148c49779 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -30,6 +30,8 @@ class WriteBufferFromS3 final : public WriteBufferFromFileBase public: WriteBufferFromS3( std::shared_ptr client_ptr_, + /// for CompleteMultipartUploadRequest, because it blocks on recv() for a few seconds on big uploads + std::shared_ptr client_with_long_timeout_ptr_, const String & bucket_, const String & key_, size_t buf_size_, @@ -88,6 +90,7 @@ private: const S3Settings::RequestSettings::PartUploadSettings & upload_settings; const WriteSettings write_settings; const std::shared_ptr client_ptr; + const std::shared_ptr client_with_long_timeout_ptr; const std::optional> object_metadata; Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index c82f97f8b20..21bdd9a6f26 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -549,6 +549,7 @@ public: getAsyncPolicy().setAutoExecute(false); return std::make_unique( + client, client, bucket, file_name, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index bdbba5abd96..80ee1e9339d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -825,6 +825,7 @@ public: write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique( configuration_.client, + configuration_.client_with_long_timeout, bucket, key, DBMS_DEFAULT_BUFFER_SIZE, @@ -1329,6 +1330,8 @@ void StorageS3::Configuration::connect(ContextPtr context) 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)), }); + + client_with_long_timeout = client->clone(std::nullopt, request_settings.long_request_timeout_ms); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 3f35c578e19..3330ac6c210 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -311,6 +311,7 @@ public: HTTPHeaderEntries headers_from_ast; std::shared_ptr client; + std::shared_ptr client_with_long_timeout; std::vector keys; }; diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 728972c948c..e3d577ca0b3 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -69,7 +69,8 @@ struct S3Settings ThrottlerPtr get_request_throttler; ThrottlerPtr put_request_throttler; size_t retry_attempts = 10; - size_t request_timeout_ms = 30000; + size_t request_timeout_ms = 3000; + size_t long_request_timeout_ms = 30000; // TODO: Take this from config like request_timeout_ms bool allow_native_copy = true; bool throw_on_zero_files_match = false; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml index 4210c13b727..206eb4f2bad 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml @@ -4,7 +4,6 @@ 1000000 - 1
diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml index 95a313ea4f2..556bf60d385 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml @@ -4,7 +4,6 @@ 5 - 0
diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml index 7b1f503ed55..b77e72d808b 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -7,18 +7,11 @@ - - s3 - http://minio1:9001/root/data/ - minio - minio123 - s3 http://resolver:8083/root/data/ minio minio123 - 1 @@ -30,16 +23,9 @@ - - -
- s3 -
-
-
- s3 + broken_s3 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index b000ccabcf4..d6bcb3fb8f4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -64,8 +64,6 @@ def test_upload_after_check_works(cluster, broken_s3): data String ) ENGINE=MergeTree() ORDER BY id - SETTINGS - storage_policy='broken_s3' """ ) @@ -80,7 +78,7 @@ def test_upload_after_check_works(cluster, broken_s3): assert "suddenly disappeared" in error, error -def get_multipart_counters(node, query_id, log_type="ExceptionWhileProcessing"): +def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): node.query("SYSTEM FLUSH LOGS") return [ int(x) @@ -89,25 +87,7 @@ def get_multipart_counters(node, query_id, log_type="ExceptionWhileProcessing"): SELECT ProfileEvents['S3CreateMultipartUpload'], ProfileEvents['S3UploadPart'], - ProfileEvents['S3WriteRequestsErrors'], - FROM system.query_log - WHERE query_id='{query_id}' - AND type='{log_type}' - """ - ).split() - if x - ] - - -def get_put_counters(node, query_id, log_type="ExceptionWhileProcessing"): - node.query("SYSTEM FLUSH LOGS") - return [ - int(x) - for x in node.query( - f""" - SELECT - ProfileEvents['S3PutObject'], - ProfileEvents['S3WriteRequestsErrors'], + ProfileEvents['S3WriteRequestsErrors'] FROM system.query_log WHERE query_id='{query_id}' AND type='{log_type}' @@ -149,12 +129,12 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - create_multipart, upload_parts, s3_errors = get_multipart_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( node, insert_query_id ) - assert create_multipart == 1 - assert upload_parts == 0 - assert s3_errors == 1 + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 0 + assert count_s3_errors == 1 # Add "lz4" compression method in the list after https://github.com/ClickHouse/ClickHouse/issues/50975 is fixed @@ -192,12 +172,12 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - create_multipart, upload_parts, s3_errors = get_multipart_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( node, insert_query_id ) - assert create_multipart == 1 - assert upload_parts >= 2 - assert s3_errors >= 2 + assert count_create_multi_part_uploads == 1 + assert count_upload_parts >= 2 + assert count_s3_errors >= 2 def test_when_s3_connection_refused_is_retried(cluster, broken_s3): @@ -227,12 +207,12 @@ def test_when_s3_connection_refused_is_retried(cluster, broken_s3): query_id=insert_query_id, ) - create_multipart, upload_parts, s3_errors = get_multipart_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( node, insert_query_id, log_type="QueryFinish" ) - assert create_multipart == 1 - assert upload_parts == 39 - assert s3_errors == 3 + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 broken_s3.setup_at_part_upload(count=1000, after=2, action="connection_refused") insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED_1" @@ -299,13 +279,13 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( query_id=insert_query_id, ) - create_multipart, upload_parts, s3_errors = get_multipart_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( node, insert_query_id, log_type="QueryFinish" ) - assert create_multipart == 1 - assert upload_parts == 39 - assert s3_errors == 3 + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 broken_s3.setup_at_part_upload( count=1000, @@ -381,13 +361,13 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( query_id=insert_query_id, ) - create_multipart, upload_parts, s3_errors = get_multipart_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( node, insert_query_id, log_type="QueryFinish" ) - assert create_multipart == 1 - assert upload_parts == 39 - assert s3_errors == 3 + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 broken_s3.setup_at_create_multi_part_upload( count=1000, @@ -458,13 +438,13 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): query_id=insert_query_id, ) - create_multipart, upload_parts, s3_errors = get_multipart_counters( + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( node, insert_query_id, log_type="QueryFinish" ) - assert create_multipart == 1 - assert upload_parts == 7 - assert s3_errors == 3 + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 7 + assert count_s3_errors == 3 broken_s3.setup_at_part_upload( count=1000, @@ -553,60 +533,3 @@ def test_query_is_canceled_with_inf_retries(cluster, broken_s3): retry_count=120, sleep_time=1, ) - - -@pytest.mark.parametrize("node_name", ["node", "node_with_inf_s3_retries"]) -def test_adaptive_timeouts(cluster, broken_s3, node_name): - node = cluster.instances[node_name] - - broken_s3.setup_fake_puts(part_length=1) - broken_s3.setup_slow_answers( - timeout=5, - count=1000000, - ) - - insert_query_id = f"TEST_ADAPTIVE_TIMEOUTS_{node_name}" - node.query( - f""" - INSERT INTO - TABLE FUNCTION s3( - 'http://resolver:8083/root/data/adaptive_timeouts', - 'minio', 'minio123', - 'CSV', auto, 'none' - ) - SELECT - * - FROM system.numbers - LIMIT 1 - SETTINGS - s3_request_timeout_ms=30000, - s3_check_objects_after_upload=0 - """, - query_id=insert_query_id, - ) - - broken_s3.reset() - - put_objects, s3_errors = get_put_counters( - node, insert_query_id, log_type="QueryFinish" - ) - - assert put_objects == 1 - - s3_use_adaptive_timeouts = node.query( - f""" - SELECT - value - FROM system.settings - WHERE - name='s3_use_adaptive_timeouts' - """ - ).strip() - - if node_name == "node_with_inf_s3_retries": - # first 2 attempts failed - assert s3_use_adaptive_timeouts == "1" - assert s3_errors == 1 - else: - assert s3_use_adaptive_timeouts == "0" - assert s3_errors == 0 diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml index 6303e9273fc..235b9a7b7a1 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml @@ -11,7 +11,6 @@ true 0 - 0 20000 @@ -34,7 +33,6 @@ true 1 - 0 1 20000 diff --git a/tests/integration/test_storage_s3/configs/defaultS3.xml b/tests/integration/test_storage_s3/configs/defaultS3.xml index 7dac6d9fbb5..37454ef6781 100644 --- a/tests/integration/test_storage_s3/configs/defaultS3.xml +++ b/tests/integration/test_storage_s3/configs/defaultS3.xml @@ -1,4 +1,9 @@ + + + 5 + + http://resolver:8080 diff --git a/tests/integration/test_storage_s3/configs/s3_retry.xml b/tests/integration/test_storage_s3/configs/s3_retry.xml index 3171da051d0..727e23273cf 100644 --- a/tests/integration/test_storage_s3/configs/s3_retry.xml +++ b/tests/integration/test_storage_s3/configs/s3_retry.xml @@ -1,9 +1,7 @@ - 1 - 10 - 5 + 5 diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 5ef781bdc9e..103dd30340c 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -4,7 +4,6 @@ import re import socket import struct import sys -import time def gen_n_digit_number(n): @@ -40,14 +39,14 @@ random.seed("Unstable server/1.0") # Generating some "random" data and append a line which contains sum of numbers in column 4. lines = ( - b"".join([gen_line() for _ in range(500000)]) + b"".join((gen_line() for _ in range(500000))) + f"0,0,0,{-sum_in_4_column}\n".encode() ) class RequestHandler(http.server.BaseHTTPRequestHandler): def do_HEAD(self): - if self.path == "/root/test.csv" or self.path == "/root/slow_send_test.csv": + if self.path == "/root/test.csv": self.from_bytes = 0 self.end_bytes = len(lines) self.size = self.end_bytes @@ -102,18 +101,6 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): print("Dropping connection") break - if self.path == "/root/slow_send_test.csv": - self.send_block_size = 81920 - - for c, i in enumerate( - range(self.from_bytes, self.end_bytes, self.send_block_size) - ): - self.wfile.write( - lines[i : min(i + self.send_block_size, self.end_bytes)] - ) - self.wfile.flush() - time.sleep(1) - elif self.path == "/": self.wfile.write(b"OK") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 835c8b908f0..3dd3c9e39d0 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -818,15 +818,6 @@ def test_storage_s3_get_unstable(started_cluster): assert result.splitlines() == ["500001,500000,0"] -def test_storage_s3_get_slow(started_cluster): - bucket = started_cluster.minio_bucket - instance = started_cluster.instances["dummy"] - table_format = "column1 Int64, column2 Int64, column3 Int64, column4 Int64" - get_query = f"SELECT count(), sum(column3), sum(column4) FROM s3('http://resolver:8081/{started_cluster.minio_bucket}/slow_send_test.csv', 'CSV', '{table_format}') FORMAT CSV" - result = run_query(instance, get_query) - assert result.splitlines() == ["500001,500000,0"] - - def test_storage_s3_put_uncompressed(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] From f999337daee891499a78cadec1cc562cf29ebcaa Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Mon, 20 Nov 2023 14:53:22 +0100 Subject: [PATCH 590/813] Revert "Revert "s3 adaptive timeouts"" --- base/poco/Net/src/HTTPServerSession.cpp | 1 - base/poco/Net/src/HTTPSession.cpp | 31 ++++- docs/en/operations/settings/settings.md | 7 + src/Backups/BackupIO_S3.cpp | 9 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 1 - src/Core/Settings.h | 3 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 58 +++----- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 15 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 8 +- src/IO/ConnectionTimeouts.cpp | 82 +++++++++++ src/IO/ConnectionTimeouts.h | 2 + src/IO/HTTPCommon.cpp | 12 +- src/IO/HTTPCommon.h | 2 + src/IO/ReadBufferFromS3.cpp | 24 ++-- src/IO/ReadBufferFromS3.h | 4 +- src/IO/S3/Client.cpp | 12 +- src/IO/S3/Client.h | 10 +- src/IO/S3/PocoHTTPClient.cpp | 111 +++++++++++---- src/IO/S3/PocoHTTPClient.h | 5 + src/IO/S3/copyS3File.cpp | 26 ++-- src/IO/S3/copyS3File.h | 7 - src/IO/S3/tests/gtest_aws_s3_client.cpp | 3 +- src/IO/WriteBufferFromS3.cpp | 4 +- src/IO/WriteBufferFromS3.h | 3 - src/IO/tests/gtest_writebuffer_s3.cpp | 1 - src/Storages/StorageS3.cpp | 3 - src/Storages/StorageS3.h | 1 - src/Storages/StorageS3Settings.h | 3 +- .../configs/inf_s3_retries.xml | 1 + .../configs/s3_retries.xml | 1 + .../configs/storage_conf.xml | 16 ++- .../test_checking_s3_blobs_paranoid/test.py | 129 ++++++++++++++---- .../configs/config.d/storage_conf.xml | 2 + .../test_storage_s3/configs/defaultS3.xml | 5 - .../test_storage_s3/configs/s3_retry.xml | 4 +- .../s3_mocks/unstable_server.py | 17 ++- tests/integration/test_storage_s3/test.py | 9 ++ 37 files changed, 430 insertions(+), 202 deletions(-) diff --git a/base/poco/Net/src/HTTPServerSession.cpp b/base/poco/Net/src/HTTPServerSession.cpp index f6d3c4e5b92..d4f2b24879e 100644 --- a/base/poco/Net/src/HTTPServerSession.cpp +++ b/base/poco/Net/src/HTTPServerSession.cpp @@ -26,7 +26,6 @@ HTTPServerSession::HTTPServerSession(const StreamSocket& socket, HTTPServerParam _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) { setTimeout(pParams->getTimeout()); - this->socket().setReceiveTimeout(pParams->getTimeout()); } diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index d2663baaf9f..8f951b3102c 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -93,9 +93,34 @@ void HTTPSession::setTimeout(const Poco::Timespan& timeout) void HTTPSession::setTimeout(const Poco::Timespan& connectionTimeout, const Poco::Timespan& sendTimeout, const Poco::Timespan& receiveTimeout) { - _connectionTimeout = connectionTimeout; - _sendTimeout = sendTimeout; - _receiveTimeout = receiveTimeout; + try + { + _connectionTimeout = connectionTimeout; + + if (_sendTimeout.totalMicroseconds() != sendTimeout.totalMicroseconds()) { + _sendTimeout = sendTimeout; + + if (connected()) + _socket.setSendTimeout(_sendTimeout); + } + + if (_receiveTimeout.totalMicroseconds() != receiveTimeout.totalMicroseconds()) { + _receiveTimeout = receiveTimeout; + + if (connected()) + _socket.setReceiveTimeout(_receiveTimeout); + } + } + catch (NetException &) + { +#ifndef NDEBUG + throw; +#else + // mute exceptions in release + // just in case when changing settings on socket is not allowed + // however it should be OK for timeouts +#endif + } } diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e61934d2168..edc1c9bdfd7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4826,3 +4826,10 @@ When set to `true` the metadata files are written with `VERSION_FULL_OBJECT_KEY` When set to `false` the metadata files are written with the previous format version, `VERSION_INLINE_DATA`. With that format only suffixes of object storage key names are are written to the metadata files. The prefix for all of object storage key names is set in configurations files at `storage_configuration.disks` section. Default value: `false`. + +## s3_use_adaptive_timeouts {#s3_use_adaptive_timeouts} + +When set to `true` than for all s3 requests first two attempts are made with low send and receive timeouts. +When set to `false` than all attempts are made with identical timeouts. + +Default value: `true`. diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index eb9dcf6b45a..ea3f57c27ff 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -55,7 +55,9 @@ namespace static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false, request_settings.get_request_throttler, request_settings.put_request_throttler, + /* for_disk_s3 = */ false, + request_settings.get_request_throttler, + request_settings.put_request_throttler, s3_uri.uri.getScheme()); client_configuration.endpointOverride = s3_uri.endpoint; @@ -167,7 +169,6 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s blob_path.size(), mode); copyS3File( - client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -229,7 +230,6 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src { LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( - client, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], @@ -268,7 +268,7 @@ void BackupWriterS3::copyFile(const String & destination, const String & source, void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { - copyDataToS3File(create_read_buffer, start_pos, length, client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, s3_settings.request_settings, {}, + copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, s3_settings.request_settings, {}, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); } @@ -298,7 +298,6 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) { return std::make_unique( client, - client, // already has long timeout s3_uri.bucket, fs::path(s3_uri.key) / file_name, DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 302e05c8418..bedde0d7b39 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -148,7 +148,6 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapsh const auto create_writer = [&](const auto & key) { return WriteBufferFromS3( - s3_client->client, s3_client->client, s3_client->uri.bucket, key, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ac4c6b6c17f..bb5e4322485 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,6 +94,7 @@ class IColumn; 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(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) \ @@ -104,7 +105,7 @@ class IColumn; 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, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 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, 3000, "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_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_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. Only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 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) \ diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 3af316bf0cf..308db389ee1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -155,7 +155,7 @@ private: bool S3ObjectStorage::exists(const StoredObject & object) const { auto settings_ptr = s3_settings.get(); - return S3::objectExists(*clients.get()->client, bucket, object.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + return S3::objectExists(*client.get(), bucket, object.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); } std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT @@ -174,7 +174,7 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT (const std::string & path, size_t read_until_position) -> std::unique_ptr { return std::make_unique( - clients.get()->client, + client.get(), bucket, path, version_id, @@ -224,7 +224,7 @@ std::unique_ptr S3ObjectStorage::readObject( /// NOLINT { auto settings_ptr = s3_settings.get(); return std::make_unique( - clients.get()->client, + client.get(), bucket, object.remote_path, version_id, @@ -249,10 +249,8 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); - auto clients_ = clients.get(); return std::make_unique( - clients_->client, - clients_->client_with_long_timeout, + client.get(), bucket, object.remote_path, buf_size, @@ -266,15 +264,12 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix) const { auto settings_ptr = s3_settings.get(); - auto client_ptr = clients.get()->client; - - return std::make_shared(bucket, path_prefix, client_ptr, settings_ptr->list_object_keys_size); + return std::make_shared(bucket, path_prefix, client.get(), settings_ptr->list_object_keys_size); } void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto settings_ptr = s3_settings.get(); - auto client_ptr = clients.get()->client; S3::ListObjectsV2Request request; request.SetBucket(bucket); @@ -289,7 +284,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet { ProfileEvents::increment(ProfileEvents::S3ListObjects); ProfileEvents::increment(ProfileEvents::DiskS3ListObjects); - outcome = client_ptr->ListObjectsV2(request); + outcome = client.get()->ListObjectsV2(request); throwIfError(outcome); auto result = outcome.GetResult(); @@ -320,14 +315,12 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exists) { - auto client_ptr = clients.get()->client; - ProfileEvents::increment(ProfileEvents::S3DeleteObjects); ProfileEvents::increment(ProfileEvents::DiskS3DeleteObjects); S3::DeleteObjectRequest request; request.SetBucket(bucket); request.SetKey(object.remote_path); - auto outcome = client_ptr->DeleteObject(request); + auto outcome = client.get()->DeleteObject(request); throwIfUnexpectedError(outcome, if_exists); @@ -346,7 +339,6 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e } else { - auto client_ptr = clients.get()->client; auto settings_ptr = s3_settings.get(); size_t chunk_size_limit = settings_ptr->objects_chunk_size_to_delete; @@ -375,7 +367,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e S3::DeleteObjectsRequest request; request.SetBucket(bucket); request.SetDelete(delkeys); - auto outcome = client_ptr->DeleteObjects(request); + auto outcome = client.get()->DeleteObjects(request); throwIfUnexpectedError(outcome, if_exists); @@ -407,7 +399,7 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*clients.get()->client, bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); + auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) return {}; @@ -423,7 +415,7 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*clients.get()->client, bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true); + auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true); ObjectMetadata result; result.size_bytes = object_info.size; @@ -444,12 +436,12 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT /// Shortcut for S3 if (auto * dest_s3 = dynamic_cast(&object_storage_to); dest_s3 != nullptr) { - auto clients_ = clients.get(); + auto client_ = client.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*client_, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(clients_->client, - clients_->client_with_long_timeout, + copyS3File( + client.get(), bucket, object_from.remote_path, 0, @@ -473,12 +465,11 @@ void S3ObjectStorage::copyObject( // NOLINT const WriteSettings &, std::optional object_to_attributes) { - auto clients_ = clients.get(); + auto client_ = client.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*client_, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(clients_->client, - clients_->client_with_long_timeout, + copyS3File(client_, bucket, object_from.remote_path, 0, @@ -499,31 +490,25 @@ void S3ObjectStorage::setNewSettings(std::unique_ptr && void S3ObjectStorage::shutdown() { - auto clients_ptr = clients.get(); /// This call stops any next retry attempts for ongoing S3 requests. /// If S3 request is failed and the method below is executed S3 client immediately returns the last failed S3 request outcome. /// If S3 is healthy nothing wrong will be happened and S3 requests will be processed in a regular way without errors. /// This should significantly speed up shutdown process if S3 is unhealthy. - const_cast(*clients_ptr->client).DisableRequestProcessing(); - const_cast(*clients_ptr->client_with_long_timeout).DisableRequestProcessing(); + const_cast(*client.get()).DisableRequestProcessing(); } void S3ObjectStorage::startup() { - auto clients_ptr = clients.get(); - /// Need to be enabled if it was disabled during shutdown() call. - const_cast(*clients_ptr->client).EnableRequestProcessing(); - const_cast(*clients_ptr->client_with_long_timeout).EnableRequestProcessing(); + const_cast(*client.get()).EnableRequestProcessing(); } void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, 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); - auto new_clients = std::make_unique(std::move(new_client), *new_s3_settings); s3_settings.set(std::move(new_s3_settings)); - clients.set(std::move(new_clients)); + client.set(std::move(new_client)); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( @@ -538,9 +523,6 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( endpoint, object_key_prefix); } -S3ObjectStorage::Clients::Clients(std::shared_ptr client_, const S3ObjectStorageSettings & settings) - : client(std::move(client_)), client_with_long_timeout(client->clone(std::nullopt, settings.request_settings.long_request_timeout_ms)) {} - ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string &) const { /// Path to store the new S3 object. diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index b1b3fb22366..7d14482311f 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -39,16 +39,6 @@ struct S3ObjectStorageSettings class S3ObjectStorage : public IObjectStorage { -public: - struct Clients - { - std::shared_ptr client; - std::shared_ptr client_with_long_timeout; - - Clients() = default; - Clients(std::shared_ptr client, const S3ObjectStorageSettings & settings); - }; - private: friend class S3PlainObjectStorage; @@ -63,7 +53,7 @@ private: String object_key_prefix_) : bucket(std::move(bucket_)) , object_key_prefix(std::move(object_key_prefix_)) - , clients(std::make_unique(std::move(client_), *s3_settings_)) + , client(std::move(client_)) , s3_settings(std::move(s3_settings_)) , s3_capabilities(s3_capabilities_) , version_id(std::move(version_id_)) @@ -184,7 +174,8 @@ private: std::string bucket; String object_key_prefix; - MultiVersion clients; + + MultiVersion client; MultiVersion s3_settings; S3Capabilities s3_capabilities; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index de88c876922..0232a6eb070 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -60,13 +60,15 @@ std::unique_ptr getClient( uri.uri.getScheme()); client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 1000); - client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 3000); + client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 30000); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100); client_configuration.endpointOverride = uri.endpoint; - client_configuration.http_keep_alive_timeout_ms - = config.getUInt(config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); + client_configuration.http_keep_alive_timeout_ms = config.getUInt( + config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); client_configuration.wait_on_pool_size_limit = false; + client_configuration.s3_use_adaptive_timeouts = config.getBool( + config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); /* * Override proxy configuration for backwards compatibility with old configuration format. diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 01fbaa4f817..970afc75ec3 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -133,4 +133,86 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings settings.http_receive_timeout); } +class SendReceiveTimeoutsForFirstAttempt +{ +private: + static constexpr size_t known_methods_count = 6; + using KnownMethodsArray = std::array; + static const KnownMethodsArray known_methods; + + /// HTTP_POST is used for CompleteMultipartUpload requests. Its latency could be high. + /// These requests need longer timeout, especially when minio is used. + /// The same assumption are made for HTTP_DELETE, HTTP_PATCH + /// That requests are more heavy that HTTP_GET, HTTP_HEAD, HTTP_PUT + + static constexpr Poco::Timestamp::TimeDiff first_byte_ms[known_methods_count][2] = + { + /* GET */ {200, 200}, + /* POST */ {200, 200}, + /* DELETE */ {200, 200}, + /* PUT */ {200, 200}, + /* HEAD */ {200, 200}, + /* PATCH */ {200, 200}, + }; + + static constexpr Poco::Timestamp::TimeDiff rest_bytes_ms[known_methods_count][2] = + { + /* GET */ {500, 500}, + /* POST */ {1000, 30000}, + /* DELETE */ {1000, 10000}, + /* PUT */ {1000, 3000}, + /* HEAD */ {500, 500}, + /* PATCH */ {1000, 10000}, + }; + + static_assert(sizeof(first_byte_ms) == sizeof(rest_bytes_ms)); + static_assert(sizeof(first_byte_ms) == known_methods_count * sizeof(Poco::Timestamp::TimeDiff) * 2); + + static size_t getMethodIndex(const String & method) + { + KnownMethodsArray::const_iterator it = std::find(known_methods.begin(), known_methods.end(), method); + chassert(it != known_methods.end()); + if (it == known_methods.end()) + return 0; + return std::distance(known_methods.begin(), it); + } + +public: + static std::pair getSendReceiveTimeout(const String & method, bool first_byte) + { + auto idx = getMethodIndex(method); + + if (first_byte) + return std::make_pair( + Poco::Timespan(first_byte_ms[idx][0] * 1000), + Poco::Timespan(first_byte_ms[idx][1] * 1000) + ); + + return std::make_pair( + Poco::Timespan(rest_bytes_ms[idx][0] * 1000), + Poco::Timespan(rest_bytes_ms[idx][1] * 1000) + ); + } +}; + +const SendReceiveTimeoutsForFirstAttempt::KnownMethodsArray SendReceiveTimeoutsForFirstAttempt::known_methods = +{ + "GET", "POST", "DELETE", "PUT", "HEAD", "PATCH" +}; + + +ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(const String & method, bool first_attempt, bool first_byte) const +{ + if (!first_attempt) + return *this; + + auto [send, recv] = SendReceiveTimeoutsForFirstAttempt::getSendReceiveTimeout(method, first_byte); + + auto aggressive = *this; + aggressive.send_timeout = saturate(send, send_timeout); + aggressive.receive_timeout = saturate(recv, receive_timeout); + + return aggressive; +} + } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 684af42827f..aabebdb836d 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -67,6 +67,8 @@ struct ConnectionTimeouts /// Timeouts for the case when we will try many addresses in a loop. static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings); static ConnectionTimeouts getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout); + + ConnectionTimeouts getAdaptiveTimeouts(const String & method, bool first_attempt, bool first_byte) const; }; } diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 65ffa51a466..cce394c67c9 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -50,12 +50,6 @@ namespace ErrorCodes namespace { - void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) - { - session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); - session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); - } - Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const ProxyConfiguration & proxy_configuration) { Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config; @@ -359,6 +353,12 @@ namespace }; } +void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) +{ + session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); + session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); +} + void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout) { if (!response.getKeepAlive()) diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index de62b5d5c16..c9968fc6915 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -113,4 +113,6 @@ std::istream * receiveResponse( void assertResponseIsOk( const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false); + +void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index f19978ccb47..c9c9319c44c 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -167,9 +167,9 @@ bool ReadBufferFromS3::nextImpl() } size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; !next_result; ++attempt) + for (size_t attempt = 1; !next_result; ++attempt) { - bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + bool last_attempt = attempt >= request_settings.max_single_read_retries; ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); @@ -177,7 +177,7 @@ bool ReadBufferFromS3::nextImpl() { if (!impl) { - impl = initialize(); + impl = initialize(attempt); if (use_external_buffer) { @@ -232,9 +232,9 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons { size_t initial_n = n; size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; n > 0; ++attempt) + for (size_t attempt = 1; n > 0; ++attempt) { - bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + bool last_attempt = attempt >= request_settings.max_single_read_retries; size_t bytes_copied = 0; ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); @@ -266,7 +266,7 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons try { - result = sendRequest(range_begin, range_begin + n - 1); + result = sendRequest(attempt, range_begin, range_begin + n - 1); std::istream & istr = result->GetBody(); copyFromIStreamWithProgressCallback(istr, to, n, progress_callback, &bytes_copied); @@ -304,8 +304,8 @@ bool ReadBufferFromS3::processException(Poco::Exception & e, size_t read_offset, LOG_DEBUG( log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, " - "Attempt: {}, Message: {}", - bucket, key, version_id.empty() ? "Latest" : version_id, read_offset, attempt, e.message()); + "Attempt: {}/{}, Message: {}", + bucket, key, version_id.empty() ? "Latest" : version_id, read_offset, attempt, request_settings.max_single_read_retries, e.message()); if (auto * s3_exception = dynamic_cast(&e)) @@ -463,7 +463,7 @@ ReadBufferFromS3::~ReadBufferFromS3() } } -std::unique_ptr ReadBufferFromS3::initialize() +std::unique_ptr ReadBufferFromS3::initialize(size_t attempt) { resetSessionIfNeeded(readAllRangeSuccessfully(), read_result); read_all_range_successfully = false; @@ -475,13 +475,13 @@ std::unique_ptr ReadBufferFromS3::initialize() if (read_until_position && offset >= read_until_position) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); - read_result = sendRequest(offset, read_until_position ? std::make_optional(read_until_position - 1) : std::nullopt); + read_result = sendRequest(attempt, offset, read_until_position ? std::make_optional(read_until_position - 1) : std::nullopt); size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; return std::make_unique(read_result->GetBody(), buffer_size); } -Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin, std::optional range_end_incl) const +Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, size_t range_begin, std::optional range_end_incl) const { S3::GetObjectRequest req; req.SetBucket(bucket); @@ -489,6 +489,8 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin if (!version_id.empty()) req.SetVersionId(version_id); + req.SetAdditionalCustomHeaderValue("clickhouse-request", fmt::format("attempt={}", attempt)); + if (range_end_incl) { req.SetRange(fmt::format("bytes={}-{}", range_begin, *range_end_incl)); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 0835e52a5b2..101e25f8b43 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -79,7 +79,7 @@ public: bool supportsReadAt() override { return true; } private: - std::unique_ptr initialize(); + std::unique_ptr initialize(size_t attempt); /// If true, if we destroy impl now, no work was wasted. Just for metrics. bool atEndOfRequestedRangeGuess(); @@ -88,7 +88,7 @@ private: /// Returns true if the error looks retriable. bool processException(Poco::Exception & e, size_t read_offset, size_t attempt) const; - Aws::S3::Model::GetObjectResult sendRequest(size_t range_begin, std::optional range_end_incl) const; + Aws::S3::Model::GetObjectResult sendRequest(size_t attempt, size_t range_begin, std::optional range_end_incl) const; bool readAllRangeSuccessfully() const; diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index ceb7d275299..4630e68fbb6 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -118,16 +118,9 @@ std::unique_ptr Client::create( new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, use_virtual_addressing)); } -std::unique_ptr Client::clone( - std::optional> override_retry_strategy, - std::optional override_request_timeout_ms) const +std::unique_ptr Client::clone() const { - PocoHTTPClientConfiguration new_configuration = client_configuration; - if (override_retry_strategy.has_value()) - new_configuration.retryStrategy = *override_retry_strategy; - if (override_request_timeout_ms.has_value()) - new_configuration.requestTimeoutMs = *override_request_timeout_ms; - return std::unique_ptr(new Client(*this, new_configuration)); + return std::unique_ptr(new Client(*this, client_configuration)); } namespace @@ -905,6 +898,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT s3_retry_attempts, enable_s3_requests_logging, for_disk_s3, + context->getGlobalContext()->getSettingsRef().s3_use_adaptive_timeouts, get_request_throttler, put_request_throttler, error_report); diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 48310bc21af..5ad57a9d827 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -118,15 +118,7 @@ public: Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, bool use_virtual_addressing); - /// Create a client with adjusted settings: - /// * override_retry_strategy can be used to disable retries to avoid nested retries when we have - /// a retry loop outside of S3 client. Specifically, for read and write buffers. Currently not - /// actually used. - /// * override_request_timeout_ms is used to increase timeout for CompleteMultipartUploadRequest - /// because it often sits idle for 10 seconds: https://github.com/ClickHouse/ClickHouse/pull/42321 - std::unique_ptr clone( - std::optional> override_retry_strategy = std::nullopt, - std::optional override_request_timeout_ms = std::nullopt) const; + std::unique_ptr clone() const; Client & operator=(const Client &) = delete; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index d0f248f48a6..4a1b6def133 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -99,6 +99,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( unsigned int s3_retry_attempts_, bool enable_s3_requests_logging_, bool for_disk_s3_, + bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, std::function error_report_) @@ -111,6 +112,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , for_disk_s3(for_disk_s3_) , get_request_throttler(get_request_throttler_) , put_request_throttler(put_request_throttler_) + , s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_) , error_report(error_report_) { } @@ -157,6 +159,7 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config Poco::Timespan(client_configuration.http_keep_alive_timeout_ms * 1000))) /// flag indicating whether keep-alive is enabled is set to each session upon creation , remote_host_filter(client_configuration.remote_host_filter) , s3_max_redirects(client_configuration.s3_max_redirects) + , s3_use_adaptive_timeouts(client_configuration.s3_use_adaptive_timeouts) , enable_s3_requests_logging(client_configuration.enable_s3_requests_logging) , for_disk_s3(client_configuration.for_disk_s3) , get_request_throttler(client_configuration.get_request_throttler) @@ -268,6 +271,38 @@ void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricT ProfileEvents::increment(disk_s3_events_map[static_cast(type)][static_cast(kind)], amount); } +String extractAttemptFromInfo(const Aws::String & request_info) +{ + static auto key = Aws::String("attempt="); + + auto key_begin = request_info.find(key, 0); + if (key_begin == Aws::String::npos) + return "1"; + + auto val_begin = key_begin + key.size(); + auto val_end = request_info.find(';', val_begin); + if (val_end == Aws::String::npos) + val_end = request_info.size(); + + return request_info.substr(val_begin, val_end-val_begin); +} + +String getOrEmpty(const Aws::Http::HeaderValueCollection & map, const String & key) +{ + auto it = map.find(key); + if (it == map.end()) + return {}; + return it->second; +} + +ConnectionTimeouts PocoHTTPClient::getTimeouts(const String & method, bool first_attempt, bool first_byte) const +{ + if (!s3_use_adaptive_timeouts) + return timeouts; + + return timeouts.getAdaptiveTimeouts(method, first_attempt, first_byte); +} + void PocoHTTPClient::makeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, @@ -282,6 +317,25 @@ void PocoHTTPClient::makeRequestInternal( makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); } +String getMethod(const Aws::Http::HttpRequest & request) +{ + switch (request.GetMethod()) + { + case Aws::Http::HttpMethod::HTTP_GET: + return Poco::Net::HTTPRequest::HTTP_GET; + case Aws::Http::HttpMethod::HTTP_POST: + return Poco::Net::HTTPRequest::HTTP_POST; + case Aws::Http::HttpMethod::HTTP_DELETE: + return Poco::Net::HTTPRequest::HTTP_DELETE; + case Aws::Http::HttpMethod::HTTP_PUT: + return Poco::Net::HTTPRequest::HTTP_PUT; + case Aws::Http::HttpMethod::HTTP_HEAD: + return Poco::Net::HTTPRequest::HTTP_HEAD; + case Aws::Http::HttpMethod::HTTP_PATCH: + return Poco::Net::HTTPRequest::HTTP_PATCH; + } +} + template void PocoHTTPClient::makeRequestInternalImpl( Aws::Http::HttpRequest & request, @@ -295,9 +349,14 @@ void PocoHTTPClient::makeRequestInternalImpl( Poco::Logger * log = &Poco::Logger::get("AWSClient"); auto uri = request.GetUri().GetURIString(); + auto method = getMethod(request); + + auto sdk_attempt = extractAttemptFromInfo(getOrEmpty(request.GetHeaders(), Aws::Http::SDK_REQUEST_HEADER)); + auto ch_attempt = extractAttemptFromInfo(getOrEmpty(request.GetHeaders(), "clickhouse-request")); + bool first_attempt = ch_attempt == "1" && sdk_attempt == "1"; if (enable_s3_requests_logging) - LOG_TEST(log, "Make request to: {}", uri); + LOG_TEST(log, "Make request to: {}, aws sdk attempt: {}, clickhouse attempt: {}", uri, sdk_attempt, ch_attempt); switch (request.GetMethod()) { @@ -348,17 +407,29 @@ void PocoHTTPClient::makeRequestInternalImpl( /// This can lead to request signature difference on S3 side. if constexpr (pooled) session = makePooledHTTPSession( - target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit, proxy_configuration); + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true), + http_connection_pool_size, + wait_on_pool_size_limit, + proxy_configuration); else - session = makeHTTPSession(target_uri, timeouts, proxy_configuration); + session = makeHTTPSession( + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true), + proxy_configuration); } else { if constexpr (pooled) session = makePooledHTTPSession( - target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit); + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true), + http_connection_pool_size, + wait_on_pool_size_limit); else - session = makeHTTPSession(target_uri, timeouts); + session = makeHTTPSession( + target_uri, + getTimeouts(method, first_attempt, /*first_byte*/ true)); } /// In case of error this address will be written to logs @@ -392,28 +463,7 @@ void PocoHTTPClient::makeRequestInternalImpl( path_and_query = "/"; poco_request.setURI(path_and_query); - - switch (request.GetMethod()) - { - case Aws::Http::HttpMethod::HTTP_GET: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_GET); - break; - case Aws::Http::HttpMethod::HTTP_POST: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_POST); - break; - case Aws::Http::HttpMethod::HTTP_DELETE: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); - break; - case Aws::Http::HttpMethod::HTTP_PUT: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); - break; - case Aws::Http::HttpMethod::HTTP_HEAD: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); - break; - case Aws::Http::HttpMethod::HTTP_PATCH: - poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); - break; - } + poco_request.setMethod(method); /// Headers coming from SDK are lower-cased. for (const auto & [header_name, header_value] : request.GetHeaders()) @@ -438,6 +488,7 @@ void PocoHTTPClient::makeRequestInternalImpl( request.GetContentBody()->clear(); request.GetContentBody()->seekg(0); + setTimeouts(*session, getTimeouts(method, first_attempt, /*first_byte*/ false)); auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); if (enable_s3_requests_logging) LOG_TEST(log, "Written {} bytes to request body", size); @@ -447,6 +498,8 @@ void PocoHTTPClient::makeRequestInternalImpl( LOG_TEST(log, "Receiving response..."); auto & response_body_stream = session->receiveResponse(poco_response); + setTimeouts(*session, getTimeouts(method, first_attempt, /*first_byte*/ false)); + watch.stop(); addMetric(request, S3MetricType::Microseconds, watch.elapsedMicroseconds()); @@ -498,6 +551,7 @@ void PocoHTTPClient::makeRequestInternalImpl( /// Request is successful but for some special requests we can have actual error message in body if (status_code >= SUCCESS_RESPONSE_MIN && status_code <= SUCCESS_RESPONSE_MAX && checkRequestCanReturn2xxAndErrorInBody(request)) { + /// reading the full response std::string response_string((std::istreambuf_iterator(response_body_stream)), std::istreambuf_iterator()); @@ -512,7 +566,6 @@ void PocoHTTPClient::makeRequestInternalImpl( addMetric(request, S3MetricType::Errors); if (error_report) error_report(proxy_configuration); - } /// Set response from string @@ -531,6 +584,8 @@ void PocoHTTPClient::makeRequestInternalImpl( if (status_code >= 500 && error_report) error_report(proxy_configuration); } + + /// expose stream, after that client reads data from that stream without built-in retries response->SetResponseBody(response_body_stream, session); } diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 2a449458360..5178d75e7b6 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -55,6 +55,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration size_t http_connection_pool_size = 0; /// See PoolBase::BehaviourOnLimit bool wait_on_pool_size_limit = true; + bool s3_use_adaptive_timeouts = true; std::function error_report; @@ -69,6 +70,7 @@ private: unsigned int s3_retry_attempts, bool enable_s3_requests_logging_, bool for_disk_s3_, + bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, std::function error_report_ @@ -169,6 +171,8 @@ private: Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; + ConnectionTimeouts getTimeouts(const String & method, bool first_attempt, bool first_byte) const; + protected: static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; @@ -178,6 +182,7 @@ protected: ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; + bool s3_use_adaptive_timeouts = true; bool enable_s3_requests_logging; bool for_disk_s3; diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index a16a1a41505..30da1c580c1 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -53,7 +53,6 @@ namespace public: UploadHelper( const std::shared_ptr & client_ptr_, - const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, @@ -62,7 +61,6 @@ namespace bool for_disk_s3_, const Poco::Logger * log_) : client_ptr(client_ptr_) - , client_with_long_timeout_ptr(client_with_long_timeout_ptr_) , dest_bucket(dest_bucket_) , dest_key(dest_key_) , request_settings(request_settings_) @@ -78,7 +76,6 @@ namespace protected: std::shared_ptr client_ptr; - std::shared_ptr client_with_long_timeout_ptr; const String & dest_bucket; const String & dest_key; const S3Settings::RequestSettings & request_settings; @@ -179,7 +176,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); - auto outcome = client_with_long_timeout_ptr->CompleteMultipartUpload(request); + auto outcome = client_ptr->CompleteMultipartUpload(request); if (outcome.IsSuccess()) { @@ -433,14 +430,13 @@ namespace size_t offset_, size_t size_, const std::shared_ptr & client_ptr_, - const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) + : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) , create_read_buffer(create_read_buffer_) , offset(offset_) , size(size_) @@ -602,7 +598,6 @@ namespace public: CopyFileHelper( const std::shared_ptr & client_ptr_, - const std::shared_ptr & client_with_long_timeout_ptr_, const String & src_bucket_, const String & src_key_, size_t src_offset_, @@ -614,7 +609,7 @@ namespace const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) + : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) , src_bucket(src_bucket_) , src_key(src_key_) , offset(src_offset_) @@ -677,7 +672,7 @@ namespace /// 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 request.SetContentType("binary/octet-stream"); - client_with_long_timeout_ptr->setKMSHeaders(request); + client_ptr->setKMSHeaders(request); } void processCopyRequest(const S3::CopyObjectRequest & request) @@ -689,7 +684,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CopyObject); - auto outcome = client_with_long_timeout_ptr->CopyObject(request); + auto outcome = client_ptr->CopyObject(request); if (outcome.IsSuccess()) { LOG_TRACE( @@ -714,7 +709,6 @@ namespace offset, size, client_ptr, - client_with_long_timeout_ptr, dest_bucket, dest_key, request_settings, @@ -788,7 +782,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy); - auto outcome = client_with_long_timeout_ptr->UploadPartCopy(req); + auto outcome = client_ptr->UploadPartCopy(req); if (!outcome.IsSuccess()) { abortMultipartUpload(); @@ -806,7 +800,6 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, - const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, @@ -814,14 +807,13 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule, bool for_disk_s3) { - CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } void copyS3File( const std::shared_ptr & s3_client, - const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -836,7 +828,7 @@ void copyS3File( { if (settings.allow_native_copy) { - CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, read_settings, object_metadata, schedule, for_disk_s3}; + CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, read_settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } else @@ -845,7 +837,7 @@ void copyS3File( { return std::make_unique(s3_client, src_bucket, src_key, "", settings, read_settings); }; - copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 1bcbfd7735e..33e22fdfba2 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -27,15 +27,9 @@ using CreateReadBuffer = std::function()>; /// because it is a known issue, it is fallbacks to read-write copy /// (copyDataToS3File()). /// -/// s3_client_with_long_timeout (may be equal to s3_client) is used for native copy and -/// CompleteMultipartUpload requests. These requests need longer timeout because S3 servers often -/// block on them for multiple seconds without sending or receiving data from us (maybe the servers -/// are copying data internally, or maybe throttling, idk). -/// /// read_settings - is used for throttling in case of native copy is not possible void copyS3File( const std::shared_ptr & s3_client, - const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -58,7 +52,6 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, - const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index c42f14e9a53..bff9ca6fa7b 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -91,7 +91,6 @@ void doWriteRequest(std::shared_ptr client, const DB::S3:: DB::S3Settings::RequestSettings request_settings; request_settings.max_unexpected_write_error_retries = max_unexpected_write_error_retries; DB::WriteBufferFromS3 write_buffer( - client, client, uri.bucket, uri.key, @@ -171,6 +170,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeadersRead) "authorization: ... SignedHeaders=" "amz-sdk-invocation-id;" "amz-sdk-request;" + "clickhouse-request;" "content-type;" "host;" "x-amz-api-version;" @@ -216,6 +216,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersRead) "authorization: ... SignedHeaders=" "amz-sdk-invocation-id;" "amz-sdk-request;" + "clickhouse-request;" "content-type;" "host;" "x-amz-api-version;" diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index e1b9c17efe9..62d0c80f1f2 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -77,7 +77,6 @@ struct WriteBufferFromS3::PartData WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, - std::shared_ptr client_with_long_timeout_ptr_, const String & bucket_, const String & key_, size_t buf_size_, @@ -92,7 +91,6 @@ WriteBufferFromS3::WriteBufferFromS3( , upload_settings(request_settings.getUploadSettings()) , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) - , client_with_long_timeout_ptr(std::move(client_with_long_timeout_ptr_)) , object_metadata(std::move(object_metadata_)) , buffer_allocation_policy(ChooseBufferPolicy(upload_settings)) , task_tracker( @@ -566,7 +564,7 @@ void WriteBufferFromS3::completeMultipartUpload() ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); Stopwatch watch; - auto outcome = client_with_long_timeout_ptr->CompleteMultipartUpload(req); + auto outcome = client_ptr->CompleteMultipartUpload(req); watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 95148c49779..590342cc997 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -30,8 +30,6 @@ class WriteBufferFromS3 final : public WriteBufferFromFileBase public: WriteBufferFromS3( std::shared_ptr client_ptr_, - /// for CompleteMultipartUploadRequest, because it blocks on recv() for a few seconds on big uploads - std::shared_ptr client_with_long_timeout_ptr_, const String & bucket_, const String & key_, size_t buf_size_, @@ -90,7 +88,6 @@ private: const S3Settings::RequestSettings::PartUploadSettings & upload_settings; const WriteSettings write_settings; const std::shared_ptr client_ptr; - const std::shared_ptr client_with_long_timeout_ptr; const std::optional> object_metadata; Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 21bdd9a6f26..c82f97f8b20 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -549,7 +549,6 @@ public: getAsyncPolicy().setAutoExecute(false); return std::make_unique( - client, client, bucket, file_name, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 80ee1e9339d..bdbba5abd96 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -825,7 +825,6 @@ public: write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique( configuration_.client, - configuration_.client_with_long_timeout, bucket, key, DBMS_DEFAULT_BUFFER_SIZE, @@ -1330,8 +1329,6 @@ void StorageS3::Configuration::connect(ContextPtr context) 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)), }); - - client_with_long_timeout = client->clone(std::nullopt, request_settings.long_request_timeout_ms); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 3330ac6c210..3f35c578e19 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -311,7 +311,6 @@ public: HTTPHeaderEntries headers_from_ast; std::shared_ptr client; - std::shared_ptr client_with_long_timeout; std::vector keys; }; diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index e3d577ca0b3..728972c948c 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -69,8 +69,7 @@ struct S3Settings ThrottlerPtr get_request_throttler; ThrottlerPtr put_request_throttler; size_t retry_attempts = 10; - size_t request_timeout_ms = 3000; - size_t long_request_timeout_ms = 30000; // TODO: Take this from config like request_timeout_ms + size_t request_timeout_ms = 30000; bool allow_native_copy = true; bool throw_on_zero_files_match = false; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml index 206eb4f2bad..4210c13b727 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml @@ -4,6 +4,7 @@ 1000000 + 1 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml index 556bf60d385..95a313ea4f2 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml @@ -4,6 +4,7 @@ 5 + 0 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml index b77e72d808b..7b1f503ed55 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -7,11 +7,18 @@ + + s3 + http://minio1:9001/root/data/ + minio + minio123 + s3 http://resolver:8083/root/data/ minio minio123 + 1 @@ -23,9 +30,16 @@ + + +
+ s3 +
+
+
- broken_s3 + s3 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index d6bcb3fb8f4..b000ccabcf4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -64,6 +64,8 @@ def test_upload_after_check_works(cluster, broken_s3): data String ) ENGINE=MergeTree() ORDER BY id + SETTINGS + storage_policy='broken_s3' """ ) @@ -78,7 +80,7 @@ def test_upload_after_check_works(cluster, broken_s3): assert "suddenly disappeared" in error, error -def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): +def get_multipart_counters(node, query_id, log_type="ExceptionWhileProcessing"): node.query("SYSTEM FLUSH LOGS") return [ int(x) @@ -87,7 +89,25 @@ def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): SELECT ProfileEvents['S3CreateMultipartUpload'], ProfileEvents['S3UploadPart'], - ProfileEvents['S3WriteRequestsErrors'] + ProfileEvents['S3WriteRequestsErrors'], + FROM system.query_log + WHERE query_id='{query_id}' + AND type='{log_type}' + """ + ).split() + if x + ] + + +def get_put_counters(node, query_id, log_type="ExceptionWhileProcessing"): + node.query("SYSTEM FLUSH LOGS") + return [ + int(x) + for x in node.query( + f""" + SELECT + ProfileEvents['S3PutObject'], + ProfileEvents['S3WriteRequestsErrors'], FROM system.query_log WHERE query_id='{query_id}' AND type='{log_type}' @@ -129,12 +149,12 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 0 - assert count_s3_errors == 1 + assert create_multipart == 1 + assert upload_parts == 0 + assert s3_errors == 1 # Add "lz4" compression method in the list after https://github.com/ClickHouse/ClickHouse/issues/50975 is fixed @@ -172,12 +192,12 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts >= 2 - assert count_s3_errors >= 2 + assert create_multipart == 1 + assert upload_parts >= 2 + assert s3_errors >= 2 def test_when_s3_connection_refused_is_retried(cluster, broken_s3): @@ -207,12 +227,12 @@ def test_when_s3_connection_refused_is_retried(cluster, broken_s3): query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 39 - assert count_s3_errors == 3 + assert create_multipart == 1 + assert upload_parts == 39 + assert s3_errors == 3 broken_s3.setup_at_part_upload(count=1000, after=2, action="connection_refused") insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED_1" @@ -279,13 +299,13 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 39 - assert count_s3_errors == 3 + assert create_multipart == 1 + assert upload_parts == 39 + assert s3_errors == 3 broken_s3.setup_at_part_upload( count=1000, @@ -361,13 +381,13 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 39 - assert count_s3_errors == 3 + assert create_multipart == 1 + assert upload_parts == 39 + assert s3_errors == 3 broken_s3.setup_at_create_multi_part_upload( count=1000, @@ -438,13 +458,13 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): query_id=insert_query_id, ) - count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id, log_type="QueryFinish" ) - assert count_create_multi_part_uploads == 1 - assert count_upload_parts == 7 - assert count_s3_errors == 3 + assert create_multipart == 1 + assert upload_parts == 7 + assert s3_errors == 3 broken_s3.setup_at_part_upload( count=1000, @@ -533,3 +553,60 @@ def test_query_is_canceled_with_inf_retries(cluster, broken_s3): retry_count=120, sleep_time=1, ) + + +@pytest.mark.parametrize("node_name", ["node", "node_with_inf_s3_retries"]) +def test_adaptive_timeouts(cluster, broken_s3, node_name): + node = cluster.instances[node_name] + + broken_s3.setup_fake_puts(part_length=1) + broken_s3.setup_slow_answers( + timeout=5, + count=1000000, + ) + + insert_query_id = f"TEST_ADAPTIVE_TIMEOUTS_{node_name}" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/adaptive_timeouts', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1 + SETTINGS + s3_request_timeout_ms=30000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + broken_s3.reset() + + put_objects, s3_errors = get_put_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert put_objects == 1 + + s3_use_adaptive_timeouts = node.query( + f""" + SELECT + value + FROM system.settings + WHERE + name='s3_use_adaptive_timeouts' + """ + ).strip() + + if node_name == "node_with_inf_s3_retries": + # first 2 attempts failed + assert s3_use_adaptive_timeouts == "1" + assert s3_errors == 1 + else: + assert s3_use_adaptive_timeouts == "0" + assert s3_errors == 0 diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml index 235b9a7b7a1..6303e9273fc 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml @@ -11,6 +11,7 @@ true 0 + 0 20000 @@ -33,6 +34,7 @@ true 1 + 0 1 20000 diff --git a/tests/integration/test_storage_s3/configs/defaultS3.xml b/tests/integration/test_storage_s3/configs/defaultS3.xml index 37454ef6781..7dac6d9fbb5 100644 --- a/tests/integration/test_storage_s3/configs/defaultS3.xml +++ b/tests/integration/test_storage_s3/configs/defaultS3.xml @@ -1,9 +1,4 @@ - - - 5 - - http://resolver:8080 diff --git a/tests/integration/test_storage_s3/configs/s3_retry.xml b/tests/integration/test_storage_s3/configs/s3_retry.xml index 727e23273cf..3171da051d0 100644 --- a/tests/integration/test_storage_s3/configs/s3_retry.xml +++ b/tests/integration/test_storage_s3/configs/s3_retry.xml @@ -1,7 +1,9 @@ - 5 + 1 + 10 + 5 diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 103dd30340c..5ef781bdc9e 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -4,6 +4,7 @@ import re import socket import struct import sys +import time def gen_n_digit_number(n): @@ -39,14 +40,14 @@ random.seed("Unstable server/1.0") # Generating some "random" data and append a line which contains sum of numbers in column 4. lines = ( - b"".join((gen_line() for _ in range(500000))) + b"".join([gen_line() for _ in range(500000)]) + f"0,0,0,{-sum_in_4_column}\n".encode() ) class RequestHandler(http.server.BaseHTTPRequestHandler): def do_HEAD(self): - if self.path == "/root/test.csv": + if self.path == "/root/test.csv" or self.path == "/root/slow_send_test.csv": self.from_bytes = 0 self.end_bytes = len(lines) self.size = self.end_bytes @@ -101,6 +102,18 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): print("Dropping connection") break + if self.path == "/root/slow_send_test.csv": + self.send_block_size = 81920 + + for c, i in enumerate( + range(self.from_bytes, self.end_bytes, self.send_block_size) + ): + self.wfile.write( + lines[i : min(i + self.send_block_size, self.end_bytes)] + ) + self.wfile.flush() + time.sleep(1) + elif self.path == "/": self.wfile.write(b"OK") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 3dd3c9e39d0..835c8b908f0 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -818,6 +818,15 @@ def test_storage_s3_get_unstable(started_cluster): assert result.splitlines() == ["500001,500000,0"] +def test_storage_s3_get_slow(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + table_format = "column1 Int64, column2 Int64, column3 Int64, column4 Int64" + get_query = f"SELECT count(), sum(column3), sum(column4) FROM s3('http://resolver:8081/{started_cluster.minio_bucket}/slow_send_test.csv', 'CSV', '{table_format}') FORMAT CSV" + result = run_query(instance, get_query) + assert result.splitlines() == ["500001,500000,0"] + + def test_storage_s3_put_uncompressed(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] From 4c7daf51675fda3f1ef02481b25ba9f572132f58 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 20 Nov 2023 14:55:23 +0100 Subject: [PATCH 591/813] Fix --- tests/integration/test_storage_s3_queue/test.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index ec27b732634..b1163a549b1 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -717,6 +717,8 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 300 + row_num = 50 + total_rows = row_num * files_to_generate for instance in [node, node_2]: create_table( @@ -734,7 +736,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): create_mv(instance, table_name, dst_table_name) total_values = generate_random_files( - started_cluster, files_path, files_to_generate, row_num=50 + started_cluster, files_path, files_to_generate, row_num=row_num ) def get_count(node, table_name): @@ -743,13 +745,13 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): for _ in range(150): if ( get_count(node, dst_table_name) + get_count(node_2, dst_table_name) - ) == files_to_generate: + ) == total_rows: break time.sleep(1) if ( get_count(node, dst_table_name) + get_count(node_2, dst_table_name) - ) != files_to_generate: + ) != total_rows: info = node.query( f"SELECT * FROM system.s3queue WHERE zookeeper_path like '%{table_name}' ORDER BY file_name FORMAT Vertical" ) @@ -762,7 +764,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): list(map(int, l.split())) for l in run_query(node_2, get_query).splitlines() ] - assert len(res1) + len(res2) == files_to_generate + assert len(res1) + len(res2) == total_rows # Checking that all engines have made progress assert len(res1) > 0 @@ -774,7 +776,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): time.sleep(10) assert ( get_count(node, dst_table_name) + get_count(node_2, dst_table_name) - ) == files_to_generate + ) == total_rows def test_max_set_age(started_cluster): From a7fc8d4b997359da08cc46fbf66f8aad1de42ed9 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 20 Nov 2023 15:04:14 +0100 Subject: [PATCH 592/813] test_merge_tree_s3 counts errors, turn off s3_use_adaptive_timeouts --- tests/integration/test_merge_tree_s3/configs/config.d/users.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/users.xml b/tests/integration/test_merge_tree_s3/configs/config.d/users.xml index 3daa6f06a78..79e5091b28a 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/users.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/users.xml @@ -3,6 +3,7 @@ 1 20 + 0 From 6da51942ebe3e01b3ff2c565e87dc97c3441fc79 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 20 Nov 2023 11:57:46 +0100 Subject: [PATCH 593/813] Follow up the fix from #44311 --- tests/ci/build_check.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 3a20ca846a1..34a5956b777 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -128,18 +128,16 @@ def check_for_success_run( version: ClickHouseVersion, ) -> None: # TODO: Remove after S3 artifacts - # the final empty argument is necessary for distinguish build and build_suffix - logged_prefix = "/".join((S3_BUILDS_BUCKET, s3_prefix, "")) - logging.info("Checking for artifacts in %s", logged_prefix) + logging.info("Checking for artifacts %s in bucket %s", s3_prefix, S3_BUILDS_BUCKET) try: # Performance artifacts are now part of regular build, so we're safe build_results = s3_helper.list_prefix(s3_prefix) except Exception as ex: - logging.info("Got exception while listing %s: %s\nRerun", logged_prefix, ex) + logging.info("Got exception while listing %s: %s\nRerun", s3_prefix, ex) return if build_results is None or len(build_results) == 0: - logging.info("Nothing found in %s, rerun", logged_prefix) + logging.info("Nothing found in %s, rerun", s3_prefix) return logging.info("Some build results found:\n%s", build_results) @@ -254,7 +252,9 @@ def main(): # If this is rerun, then we try to find already created artifacts and just # put them as github actions artifact (result) - check_for_success_run(s3_helper, s3_path_prefix, build_name, version) + # The s3_path_prefix has additional "/" in the end to prevent finding + # e.g. `binary_darwin_aarch64/clickhouse` for `binary_darwin` + check_for_success_run(s3_helper, f"{s3_path_prefix}/", build_name, version) docker_image = get_image_with_version(IMAGES_PATH, IMAGE_NAME) image_version = docker_image.version From 94824a01945edf76cc8a7e9af3611f97b9620ff9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 20 Nov 2023 14:21:38 +0100 Subject: [PATCH 594/813] Fix shellcheck for time-trace --- utils/prepare-time-trace/prepare-time-trace.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/utils/prepare-time-trace/prepare-time-trace.sh b/utils/prepare-time-trace/prepare-time-trace.sh index 5f4aad4c0b9..812928e8bd8 100755 --- a/utils/prepare-time-trace/prepare-time-trace.sh +++ b/utils/prepare-time-trace/prepare-time-trace.sh @@ -8,7 +8,7 @@ # See also https://docs.google.com/document/d/1CvAClvFfyA5R-PhYUmn5OOQtYMH4h6I0nSsKchNAySU/preview -< Date: Mon, 20 Nov 2023 14:22:02 +0100 Subject: [PATCH 595/813] Fix logging for profile JSON files --- tests/ci/build_check.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 34a5956b777..adbd67bd95a 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -426,7 +426,9 @@ FORMAT JSONCompactEachRow""" url = f"https://{ci_logs_credentials.host}/" profiles_dir = temp_path / "profiles_source" profiles_dir.mkdir(parents=True, exist_ok=True) - logging.info("Processing profile JSON files from {GIT_REPO_ROOT}/build_docker") + logging.info( + "Processing profile JSON files from %s", repo_path / "build_docker" + ) git_runner( "./utils/prepare-time-trace/prepare-time-trace.sh " f"build_docker {profiles_dir.absolute()}" From 8fef863b95a6812097355558c57d04231624c030 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 20 Nov 2023 21:46:32 +0800 Subject: [PATCH 596/813] Fix flaky and slow tests. --- .../01710_projection_aggregation_in_order.sql | 6 ++---- .../0_stateless/02516_projections_with_rollup.sql | 12 +++++------- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql b/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql index e4fb1816c89..06f192adb57 100644 --- a/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql +++ b/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql @@ -1,5 +1,3 @@ --- Tags: disabled --- FIXME https://github.com/ClickHouse/ClickHouse/issues/49552 -- Test that check the correctness of the result for optimize_aggregation_in_order and projections, -- not that this optimization will take place. @@ -20,7 +18,7 @@ CREATE TABLE normal ) ) ENGINE = MergeTree -ORDER BY (key, ts); +ORDER BY tuple(); INSERT INTO normal SELECT number, @@ -52,7 +50,7 @@ CREATE TABLE agg ) ) ENGINE = MergeTree -ORDER BY (key, ts); +ORDER BY tuple(); INSERT INTO agg SELECT 1, diff --git a/tests/queries/0_stateless/02516_projections_with_rollup.sql b/tests/queries/0_stateless/02516_projections_with_rollup.sql index 038caf59264..a87621073af 100644 --- a/tests/queries/0_stateless/02516_projections_with_rollup.sql +++ b/tests/queries/0_stateless/02516_projections_with_rollup.sql @@ -1,6 +1,3 @@ --- Tags: disabled --- FIXME https://github.com/ClickHouse/ClickHouse/issues/49552 - DROP TABLE IF EXISTS video_log; DROP TABLE IF EXISTS video_log_result__fuzz_0; DROP TABLE IF EXISTS rng; @@ -16,7 +13,8 @@ CREATE TABLE video_log ) ENGINE = MergeTree PARTITION BY toDate(datetime) -ORDER BY (user_id, device_id); +ORDER BY (user_id, device_id) +SETTINGS index_granularity_bytes=10485760, index_granularity=8192; CREATE TABLE video_log_result__fuzz_0 ( @@ -62,7 +60,7 @@ LIMIT 10; ALTER TABLE video_log ADD PROJECTION p_norm ( - SELECT + SELECT datetime, device_id, bytes, @@ -77,12 +75,12 @@ SETTINGS mutations_sync = 1; ALTER TABLE video_log ADD PROJECTION p_agg ( - SELECT + SELECT toStartOfHour(datetime) AS hour, domain, sum(bytes), avg(duration) - GROUP BY + GROUP BY hour, domain ); From 6fa67cf95dff4f3f582d2801b38bc4ea598b94b7 Mon Sep 17 00:00:00 2001 From: edef Date: Mon, 20 Nov 2023 14:59:28 +0000 Subject: [PATCH 597/813] Test reading arrow::LargeListArray --- .../02911_arrow_large_list.reference | 4 +++ .../0_stateless/02911_arrow_large_list.sh | 31 +++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 tests/queries/0_stateless/02911_arrow_large_list.reference create mode 100755 tests/queries/0_stateless/02911_arrow_large_list.sh diff --git a/tests/queries/0_stateless/02911_arrow_large_list.reference b/tests/queries/0_stateless/02911_arrow_large_list.reference new file mode 100644 index 00000000000..a6fbcce8c06 --- /dev/null +++ b/tests/queries/0_stateless/02911_arrow_large_list.reference @@ -0,0 +1,4 @@ +a +Array(Nullable(String)) +['00000','00001','00002'] +['10000','10001','10002'] diff --git a/tests/queries/0_stateless/02911_arrow_large_list.sh b/tests/queries/0_stateless/02911_arrow_large_list.sh new file mode 100755 index 00000000000..9b1c9a9d0ed --- /dev/null +++ b/tests/queries/0_stateless/02911_arrow_large_list.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# ## generate arrow file with python +# import pyarrow as pa +# schema = pa.schema([ pa.field('a', pa.large_list(pa.utf8())) ]) +# a = pa.array([["00000", "00001", "00002"], ["10000", "10001", "10002"]]) +# with pa.OSFile('arraydata.arrow', 'wb') as sink: +# with pa.ipc.new_file(sink, schema=schema) as writer: +# batch = pa.record_batch([a], schema=schema) +# writer.write(batch) + +# cat arraydata.arrow | base64 + +cat < Date: Mon, 20 Nov 2023 16:04:21 +0100 Subject: [PATCH 598/813] Skip test in fast tests --- tests/queries/0_stateless/02918_sqlite_path_check.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02918_sqlite_path_check.sh b/tests/queries/0_stateless/02918_sqlite_path_check.sh index 1f250387a71..798efda6ec1 100755 --- a/tests/queries/0_stateless/02918_sqlite_path_check.sh +++ b/tests/queries/0_stateless/02918_sqlite_path_check.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: Fast tests don't build external libraries (SQLite) CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 59c251bf88628951b2da442780ac91b19e9a3600 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 20 Nov 2023 16:17:31 +0100 Subject: [PATCH 599/813] Allow to use concat with a single argument --- src/Functions/concat.cpp | 6 ++++-- tests/queries/0_stateless/00727_concat.reference | 2 ++ tests/queries/0_stateless/00727_concat.sql | 3 ++- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index f426f662868..6403c4b8416 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -207,6 +207,8 @@ public: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { + if (arguments.size() == 1) + return FunctionFactory::instance().getImpl("toString", context)->build(arguments); if (std::ranges::all_of(arguments, [](const auto & elem) { return isArray(elem.type); })) return FunctionFactory::instance().getImpl("arrayConcat", context)->build(arguments); if (std::ranges::all_of(arguments, [](const auto & elem) { return isMap(elem.type); })) @@ -221,10 +223,10 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() < 2) + if (arguments.empty()) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be at least 2.", + "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", getName(), arguments.size()); diff --git a/tests/queries/0_stateless/00727_concat.reference b/tests/queries/0_stateless/00727_concat.reference index 1e102051fd0..9659405ea3b 100644 --- a/tests/queries/0_stateless/00727_concat.reference +++ b/tests/queries/0_stateless/00727_concat.reference @@ -64,4 +64,6 @@ Three arguments test 42144255 42144 42144255 +42 +foo Testing the alias diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index edeaf9340dd..94cdc83d51f 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -82,8 +82,9 @@ SELECT concat(materialize(42 :: Int32), materialize(144 :: UInt64)); SELECT concat(materialize(42 :: Int32), materialize(144 :: UInt64), materialize(255 :: UInt32)); SELECT concat(42, 144); SELECT concat(42, 144, 255); +SELECT concat(42); +SELECT concat('foo'); SELECT CONCAT('Testing the ', 'alias'); SELECT concat(); -- { serverError 42 } -SELECT concat(1); -- { serverError 42 } From 9f96b5897979effb67ff9b1eb233622494bb5e1c Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 20 Nov 2023 16:22:47 +0100 Subject: [PATCH 600/813] Update docs, add more tests. --- docs/en/sql-reference/functions/string-functions.md | 2 +- tests/queries/0_stateless/00727_concat.reference | 3 +++ tests/queries/0_stateless/00727_concat.sql | 4 ++++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 4b6e0356301..1940993ce0b 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -439,7 +439,7 @@ concat(s1, s2, ...) **Arguments** -At least two values of arbitrary type. +At least one value of arbitrary type. Arguments which are not of types [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md) are converted to strings using their default serialization. As this decreases performance, it is not recommended to use non-String/FixedString arguments. diff --git a/tests/queries/0_stateless/00727_concat.reference b/tests/queries/0_stateless/00727_concat.reference index 9659405ea3b..0ce6ac247c5 100644 --- a/tests/queries/0_stateless/00727_concat.reference +++ b/tests/queries/0_stateless/00727_concat.reference @@ -64,6 +64,9 @@ Three arguments test 42144255 42144 42144255 +-- Single argument tests 42 foo +\N +\N Testing the alias diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index 94cdc83d51f..2d0e6fe6a1f 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -82,8 +82,12 @@ SELECT concat(materialize(42 :: Int32), materialize(144 :: UInt64)); SELECT concat(materialize(42 :: Int32), materialize(144 :: UInt64), materialize(255 :: UInt32)); SELECT concat(42, 144); SELECT concat(42, 144, 255); + +SELECT '-- Single argument tests'; SELECT concat(42); SELECT concat('foo'); +SELECT concat(NULL); +SELECT concat(materialize(NULL :: Nullable(UInt64))); SELECT CONCAT('Testing the ', 'alias'); From 9b5e180995c39b4f6c6549a500d0a05f749c4257 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Nov 2023 16:27:02 +0100 Subject: [PATCH 601/813] Fix --- src/Storages/S3Queue/StorageS3Queue.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 72e74d3c2a0..5d1e69f8b15 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -118,7 +118,11 @@ StorageS3Queue::StorageS3Queue( , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { - if (configuration.url.key.ends_with('/')) + if (configuration.url.key.empty()) + { + configuration.url.key = "/*"; + } + else if (configuration.url.key.ends_with('/')) { configuration.url.key += '*'; } From 23ea802bd7fe333ab72265d9bb2cac49b83e2495 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 20 Nov 2023 16:28:04 +0100 Subject: [PATCH 602/813] Add more concat tests --- tests/queries/0_stateless/00727_concat.reference | 2 ++ tests/queries/0_stateless/00727_concat.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/00727_concat.reference b/tests/queries/0_stateless/00727_concat.reference index 0ce6ac247c5..6fb23c072d3 100644 --- a/tests/queries/0_stateless/00727_concat.reference +++ b/tests/queries/0_stateless/00727_concat.reference @@ -66,6 +66,8 @@ Three arguments test 42144255 -- Single argument tests 42 +42 +foo foo \N \N diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index 2d0e6fe6a1f..f5048dcaaae 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -85,7 +85,9 @@ SELECT concat(42, 144, 255); SELECT '-- Single argument tests'; SELECT concat(42); +SELECT concat(materialize(42)); SELECT concat('foo'); +SELECT concat(materialize('foo')); SELECT concat(NULL); SELECT concat(materialize(NULL :: Nullable(UInt64))); From be0f0b68795273c0fabe336e1b8055a057dc15b1 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Mon, 20 Nov 2023 23:34:52 +0800 Subject: [PATCH 603/813] Fix code style --- .../MergeTree/tests/gtest_combine_filters.cpp | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/tests/gtest_combine_filters.cpp b/src/Storages/MergeTree/tests/gtest_combine_filters.cpp index 91e9e53c230..84ddec04c24 100644 --- a/src/Storages/MergeTree/tests/gtest_combine_filters.cpp +++ b/src/Storages/MergeTree/tests/gtest_combine_filters.cpp @@ -37,17 +37,26 @@ bool testCombineFilters(size_t size) auto result = combineFilters(first_filter, second_filter); - if (result->size() != size) return false; + if (result->size() != size) + { + return false; + } for (size_t i = 0; i < size; i++) { if (i % 4 == 0) { - if (result->get64(i) != 1) return false; + if (result->get64(i) != 1) + { + return false; + } } else { - if (result->get64(i) != 0) return false; + if (result->get64(i) != 0) + { + return false; + } } } @@ -72,7 +81,7 @@ bool testCombineColumns(size_t size) auto & column_data = column->getData(); non_zero_count = 0; - for (size_t i = 0; i < len; non_zero_count++, i+=non_zero_count) + for (size_t i = 0; i < len; non_zero_count++, i += non_zero_count) { column_data[i] = 1; } @@ -119,7 +128,10 @@ bool testCombineColumns(size_t size) } else { - if (result_data[i] != 0) return false; + if (result_data[i] != 0) + { + return false; + } } } From d84d5692ef3d78195559ae9988da61ea72026e00 Mon Sep 17 00:00:00 2001 From: Kevin Mingtarja Date: Mon, 20 Nov 2023 23:52:22 +0800 Subject: [PATCH 604/813] Enable implicit constraint for VersionedCollapsing --- .../MergeTree/registerStorageMergeTree.cpp | 4 ++- ...onstraints_for_collapsing_engine.reference | 2 ++ ...olumn_constraints_for_collapsing_engine.sh | 32 +++++++++++++++++-- 3 files changed, 34 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 9285dfcdd91..ce394d682fc 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -620,7 +620,9 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.query.columns_list && args.query.columns_list->constraints) for (auto & constraint : args.query.columns_list->constraints->children) constraints.push_back(constraint); - if (merging_params.mode == MergeTreeData::MergingParams::Collapsing && storage_settings->add_implicit_sign_column_constraint_for_collapsing_engine) + if ((merging_params.mode == MergeTreeData::MergingParams::Collapsing || + merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) && + storage_settings->add_implicit_sign_column_constraint_for_collapsing_engine) { auto sign_column_check_constraint = std::make_unique(); sign_column_check_constraint->name = "check_sign_column"; diff --git a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference index 5c6c001014d..87fb3b0e8c4 100644 --- a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference +++ b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference @@ -1,2 +1,4 @@ 1 2504 1 ok +1 200 1 1 +ok \ No newline at end of file diff --git a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh index bee12afc511..43594a45a1e 100755 --- a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh +++ b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.sh @@ -4,13 +4,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +EXCEPTION_TEXT="VIOLATED_CONSTRAINT" EXCEPTION_SUCCESS_TEXT=ok -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS collapsing_merge_tree;" # CollapsingSortedAlgorithm::merge() also has a check for sign column value # optimize_on_insert = 0 is required to avoid this automatic merge behavior -$CLICKHOUSE_CLIENT --query="SET optimize_on_insert = 0;" +$CLICKHOUSE_CLIENT --query="SET optimize_on_insert=0;" + +# CollapsingMergeTree +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS collapsing_merge_tree;" $CLICKHOUSE_CLIENT --query="CREATE TABLE collapsing_merge_tree ( Key UInt32, @@ -26,6 +29,29 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM collapsing_merge_tree;" # Should throw an exception $CLICKHOUSE_CLIENT --query="INSERT INTO collapsing_merge_tree VALUES (1, 2504, 5);" 2>&1 \ - | grep -q VIOLATED_CONSTRAINT && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" $CLICKHOUSE_CLIENT --query="DROP TABLE collapsing_merge_tree;" + + +# VersionedCollapsingMergeTree +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS versioned_collapsing_merge_tree;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE versioned_collapsing_merge_tree +( + Key UInt32, + Count UInt8, + Sign Int8, + Version UInt8 +) +ENGINE=VersionedCollapsingMergeTree(Sign, Version) ORDER BY Key +SETTINGS add_implicit_sign_column_constraint_for_collapsing_engine=1;" + +# Should succeed +$CLICKHOUSE_CLIENT --query="INSERT INTO versioned_collapsing_merge_tree VALUES (1, 2504, 1, 1);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM versioned_collapsing_merge_tree;" + +# Should throw an exception +$CLICKHOUSE_CLIENT --query="INSERT INTO versioned_collapsing_merge_tree VALUES (1, 2504, 5, 1);" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" + +$CLICKHOUSE_CLIENT --query="DROP TABLE versioned_collapsing_merge_tree;" From 1d668264e11eafa184dd16a927a1897b2297dfd0 Mon Sep 17 00:00:00 2001 From: Kevin Mingtarja Date: Mon, 20 Nov 2023 23:54:36 +0800 Subject: [PATCH 605/813] Fix style --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index ce394d682fc..3310b8cb72d 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -620,8 +620,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.query.columns_list && args.query.columns_list->constraints) for (auto & constraint : args.query.columns_list->constraints->children) constraints.push_back(constraint); - if ((merging_params.mode == MergeTreeData::MergingParams::Collapsing || - merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) && + if ((merging_params.mode == MergeTreeData::MergingParams::Collapsing || + merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) && storage_settings->add_implicit_sign_column_constraint_for_collapsing_engine) { auto sign_column_check_constraint = std::make_unique(); From 38133692bb3fb997c6a960c6a460b49748683f98 Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Fri, 10 Nov 2023 14:40:55 +0100 Subject: [PATCH 606/813] Add total_bytes for RocksDB storage Show total_bytes in system tables for RocksDB storage Previously it did not work because total_rows was calculated only when optimize_trivial_approximate_count_query=1 --- src/Core/Settings.h | 2 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 31 ++++++++++++------- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 3 ++ .../02892_rocksdb_trivial_count.reference | 5 +++ .../02892_rocksdb_trivial_count.sql | 4 ++- 5 files changed, 32 insertions(+), 13 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 951c5dac663..4ef651c3708 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -545,7 +545,7 @@ class IColumn; M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ - M(Bool, optimize_trivial_approximate_count_query, false, "Use an approximate value for trivial count optimization of storages that support such estimations.", 0) \ + M(Bool, optimize_trivial_approximate_count_query, true, "Use an approximate value for trivial count optimization of storages that support such estimations.", 0) \ M(Bool, optimize_count_from_files, true, "Optimize counting rows from files in supported input formats", 0) \ M(Bool, use_cache_for_count_from_files, true, "Use cache to count the number of rows in files", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 42519c84f35..fd563ccf62b 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -703,17 +703,26 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory) std::optional StorageEmbeddedRocksDB::totalRows(const Settings & settings) const { - if (settings.optimize_trivial_approximate_count_query) - { - std::shared_lock lock(rocksdb_ptr_mx); - if (!rocksdb_ptr) - return {}; - UInt64 estimated_rows; - if (!rocksdb_ptr->GetIntProperty("rocksdb.estimate-num-keys", &estimated_rows)) - return {}; - return estimated_rows; - } - return {}; + if (!settings.optimize_trivial_approximate_count_query) + return {}; + std::shared_lock lock(rocksdb_ptr_mx); + if (!rocksdb_ptr) + return {}; + UInt64 estimated_rows; + if (!rocksdb_ptr->GetIntProperty("rocksdb.estimate-num-keys", &estimated_rows)) + return {}; + return estimated_rows; +} + +std::optional StorageEmbeddedRocksDB::totalBytes(const Settings & /*settings*/) const +{ + std::shared_lock lock(rocksdb_ptr_mx); + if (!rocksdb_ptr) + return {}; + UInt64 estimated_bytes; + if (!rocksdb_ptr->GetIntProperty("rocksdb.estimate-live-data-size", &estimated_bytes)) + return {}; + return estimated_bytes; } } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 11eba607c3a..4f987e1308a 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace rocksdb @@ -89,6 +90,8 @@ public: std::optional totalRows(const Settings & settings) const override; + std::optional totalBytes(const Settings & settings) const override; + private: const String primary_key; using RocksDBPtr = std::unique_ptr; diff --git a/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference b/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference index 9289ddcee34..7f298ea71ad 100644 --- a/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference +++ b/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference @@ -1 +1,6 @@ +-- { echoOn } +SELECT count() FROM dict SETTINGS optimize_trivial_approximate_count_query = 0, max_rows_to_read = 1; -- { serverError TOO_MANY_ROWS } +SELECT count() FROM dict SETTINGS max_rows_to_read = 1; +121 +SELECT total_rows FROM system.tables WHERE database = currentDatabase() AND name = 'dict'; 121 diff --git a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql index 0cdf2d1b2b2..b5780954d74 100644 --- a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql +++ b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql @@ -2,5 +2,7 @@ CREATE TABLE dict (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY key; INSERT INTO dict SELECT number, toString(number) FROM numbers(121); +-- { echoOn } SELECT count() FROM dict SETTINGS optimize_trivial_approximate_count_query = 0, max_rows_to_read = 1; -- { serverError TOO_MANY_ROWS } -SELECT count() FROM dict SETTINGS optimize_trivial_approximate_count_query = 1, max_rows_to_read = 1; +SELECT count() FROM dict SETTINGS max_rows_to_read = 1; +SELECT total_rows FROM system.tables WHERE database = currentDatabase() AND name = 'dict'; From d5a4580236b98bbcf1ecfed4d61b6b32c8de34ec Mon Sep 17 00:00:00 2001 From: Kevin Mingtarja Date: Tue, 21 Nov 2023 00:13:53 +0800 Subject: [PATCH 607/813] Add newline in test reference --- ...icit_sign_column_constraints_for_collapsing_engine.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference index 87fb3b0e8c4..323b12c173a 100644 --- a/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference +++ b/tests/queries/0_stateless/02918_implicit_sign_column_constraints_for_collapsing_engine.reference @@ -1,4 +1,4 @@ 1 2504 1 ok 1 200 1 1 -ok \ No newline at end of file +ok From 6d14350886655c8730c77547773e73499ce06883 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 20 Nov 2023 17:26:31 +0100 Subject: [PATCH 608/813] Allow to use any integer types with fromDaysSinceYearZero --- src/Functions/fromDaysSinceYearZero.cpp | 29 ++++++--------- .../02907_fromDaysSinceYearZero.reference | 32 ++++++++-------- .../02907_fromDaysSinceYearZero.sql | 37 +++++++++---------- 3 files changed, 46 insertions(+), 52 deletions(-) diff --git a/src/Functions/fromDaysSinceYearZero.cpp b/src/Functions/fromDaysSinceYearZero.cpp index 36a05f1cbf9..4aa96fc8fbb 100644 --- a/src/Functions/fromDaysSinceYearZero.cpp +++ b/src/Functions/fromDaysSinceYearZero.cpp @@ -1,20 +1,14 @@ #include -#include -#include -#include -#include -#include -#include -#include -#include #include #include -#include -#include +#include +#include +#include +#include +#include #include #include -#include #include #include @@ -23,7 +17,8 @@ namespace DB { namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int ARGUMENT_OUT_OF_BOUND; } namespace @@ -44,7 +39,6 @@ struct DateTraits32 template class FunctionFromDaysSinceYearZero : public IFunction { - public: static constexpr auto name = Traits::name; using RawReturnType = typename Traits::ReturnDataType::FieldType; @@ -58,9 +52,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - FunctionArgumentDescriptors args{ - {"days", &isNativeUInt, nullptr, "UInt*"} - }; + FunctionArgumentDescriptors args{{"days", &isNativeInteger, nullptr, "Integer"}}; validateFunctionArgumentTypes(*this, arguments, args); @@ -84,7 +76,8 @@ public: return false; }; - const bool success = try_type(UInt8{}) || try_type(UInt16{}) || try_type(UInt32{}) || try_type(UInt64{}); + const bool success = try_type(UInt8{}) || try_type(UInt16{}) || try_type(UInt32{}) || try_type(UInt64{}) || try_type(Int8{}) + || try_type(Int16{}) || try_type(Int32{}) || try_type(Int64{}); if (!success) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column while execute function {}", getName()); @@ -104,6 +97,8 @@ public: for (size_t i = 0; i < rows_count; ++i) { auto raw_value = src_data[i]; + if (raw_value < 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Expected a non-negative integer, got: {}", std::to_string(raw_value)); auto value = static_cast(raw_value); dst_data[i] = static_cast(value - ToDaysSinceYearZeroImpl::DAYS_BETWEEN_YEARS_0_AND_1970); } diff --git a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference index ac0f4662db2..d6f19af007e 100644 --- a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference +++ b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference @@ -1,21 +1,21 @@ -- negative tests -- const and non-const arguments -719527 2149-06-06 2149-06-06 -719528 1970-01-01 1970-01-01 -719529 1970-01-02 1970-01-02 -785062 2149-06-05 2149-06-05 -785063 2149-06-06 2149-06-06 -785064 1970-01-01 1970-01-01 -693960 2299-12-31 2299-12-31 -693961 1900-01-01 1900-01-01 -693962 1900-01-02 1900-01-02 -840056 2299-12-30 2299-12-30 -840057 2299-12-31 2299-12-31 -840058 2299-12-31 2299-12-31 --- integer types != UInt32 -255 1974-06-12 2299-12-31 -65535 1973-09-29 2299-12-31 -719529 1970-01-02 1970-01-02 +719527 719527 2149-06-06 2149-06-06 2149-06-06 2149-06-06 +719528 719528 1970-01-01 1970-01-01 1970-01-01 1970-01-01 +719529 719529 1970-01-02 1970-01-02 1970-01-02 1970-01-02 +785062 785062 2149-06-05 2149-06-05 2149-06-05 2149-06-05 +785063 785063 2149-06-06 2149-06-06 2149-06-06 2149-06-06 +785064 785064 1970-01-01 1970-01-01 1970-01-01 1970-01-01 +693960 693960 2299-12-31 2299-12-31 2299-12-31 2299-12-31 +693961 693961 1900-01-01 1900-01-01 1900-01-01 1900-01-01 +693962 693962 1900-01-02 1900-01-02 1900-01-02 1900-01-02 +840056 840056 2299-12-30 2299-12-30 2299-12-30 2299-12-30 +840057 840057 2299-12-31 2299-12-31 2299-12-31 2299-12-31 +840058 840058 2299-12-31 2299-12-31 2299-12-31 2299-12-31 +-- integer types != (U)Int32 +255 127 1974-06-12 2299-12-31 1974-02-04 2299-12-31 +65535 32767 1973-09-29 2299-12-31 2063-06-17 2299-12-31 +719529 719529 1970-01-02 1970-01-02 1970-01-02 1970-01-02 -- NULL handling \N \N -- Alias diff --git a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql index 83cfa01d5ed..22838a008be 100644 --- a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql +++ b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql @@ -7,29 +7,28 @@ SELECT fromDaysSinceYearZero(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_ SELECT fromDaysSinceYearZero32(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT fromDaysSinceYearZero('needs a number'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT fromDaysSinceYearZero32('needs a number'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT fromDaysSinceYearZero(-3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT fromDaysSinceYearZero32(-3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT fromDaysSinceYearZero(-3); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT fromDaysSinceYearZero32(-3); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT '-- const and non-const arguments'; +SELECT 719527 AS x, toInt32(719527) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -- outside Date's range +SELECT 719528 AS x, toInt32(719528) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); +SELECT 719529 AS x, toInt32(719529) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); +SELECT 785062 AS x, toInt32(785062) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); +SELECT 785063 AS x, toInt32(785063) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); +SELECT 785064 AS x, toInt32(785064) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -- outside Date's range -SELECT 719527 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)); -- outside Date's range -SELECT 719528 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)); -SELECT 719529 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)); -SELECT 785062 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)); -SELECT 785063 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)); -SELECT 785064 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)); -- outside Date's range +SELECT 693960 AS x, toInt32(693960) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -- outside Date32's range +SELECT 693961 AS x, toInt32(693961) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); +SELECT 693962 AS x, toInt32(693962) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); +SELECT 840056 AS x, toInt32(840056) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); +SELECT 840057 AS x, toInt32(840057) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); +SELECT 840058 AS x, toInt32(840058) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -- outside Date32's range -SELECT 693960 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)); -- outside Date32's range -SELECT 693961 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)); -SELECT 693962 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)); -SELECT 840056 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)); -SELECT 840057 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)); -SELECT 840058 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)); -- outside Date32's range - -SELECT '-- integer types != UInt32'; -SELECT toUInt8(255) AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x); -- outside Date's range for all UInt8-s -SELECT toUInt16(65535) AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x); -- outside Date's range for all UInt16-s -SELECT toUInt64(719529) AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x); -- something useful +SELECT '-- integer types != (U)Int32'; +SELECT toUInt8(255) AS x, toInt8(127) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x), fromDaysSinceYearZero(y), fromDaysSinceYearZero32(y); -- outside Date's range for all (U)Int8-s +SELECT toUInt16(65535) AS x, toInt16(32767) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x), fromDaysSinceYearZero(y), fromDaysSinceYearZero32(y); -- outside Date's range for all (U)Int16-s +SELECT toUInt64(719529) AS x, toInt64(719529) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x), fromDaysSinceYearZero(y), fromDaysSinceYearZero32(y); -- something useful SELECT '-- NULL handling'; SELECT fromDaysSinceYearZero(NULL), fromDaysSinceYearZero32(NULL); From cd4e674aa3a09e898cda275cced5c66b05cd747c Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 20 Nov 2023 17:31:47 +0100 Subject: [PATCH 609/813] Simplify fromDaysSinceYearZero tests a bit --- .../02907_fromDaysSinceYearZero.sql | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql index 22838a008be..053cc8563c6 100644 --- a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql +++ b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql @@ -11,19 +11,19 @@ SELECT fromDaysSinceYearZero(-3); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT fromDaysSinceYearZero32(-3); -- { serverError ARGUMENT_OUT_OF_BOUND } SELECT '-- const and non-const arguments'; -SELECT 719527 AS x, toInt32(719527) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -- outside Date's range -SELECT 719528 AS x, toInt32(719528) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -SELECT 719529 AS x, toInt32(719529) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -SELECT 785062 AS x, toInt32(785062) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -SELECT 785063 AS x, toInt32(785063) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -SELECT 785064 AS x, toInt32(785064) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -- outside Date's range +SELECT 719527 AS x, toInt32(x) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -- outside Date's range +SELECT 719528 AS x, toInt32(x) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); +SELECT 719529 AS x, toInt32(x) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); +SELECT 785062 AS x, toInt32(x) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); +SELECT 785063 AS x, toInt32(x) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); +SELECT 785064 AS x, toInt32(x) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)), fromDaysSinceYearZero(y), fromDaysSinceYearZero(materialize(y)); -- outside Date's range -SELECT 693960 AS x, toInt32(693960) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -- outside Date32's range -SELECT 693961 AS x, toInt32(693961) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -SELECT 693962 AS x, toInt32(693962) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -SELECT 840056 AS x, toInt32(840056) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -SELECT 840057 AS x, toInt32(840057) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -SELECT 840058 AS x, toInt32(840058) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -- outside Date32's range +SELECT 693960 AS x, toInt32(x) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -- outside Date32's range +SELECT 693961 AS x, toInt32(x) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); +SELECT 693962 AS x, toInt32(x) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); +SELECT 840056 AS x, toInt32(x) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); +SELECT 840057 AS x, toInt32(x) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); +SELECT 840058 AS x, toInt32(x) AS y, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)), fromDaysSinceYearZero32(y), fromDaysSinceYearZero32(materialize(y)); -- outside Date32's range SELECT '-- integer types != (U)Int32'; SELECT toUInt8(255) AS x, toInt8(127) AS y, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x), fromDaysSinceYearZero(y), fromDaysSinceYearZero32(y); -- outside Date's range for all (U)Int8-s From e5129990ed4a593acc88bcee448b9a7382f53833 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 20 Nov 2023 13:38:32 -0300 Subject: [PATCH 610/813] sign all aws headers --- src/IO/S3/Client.cpp | 50 +++++++++++++++++-------- src/IO/S3/Client.h | 32 ++++++++-------- src/IO/S3/PocoHTTPClient.cpp | 12 +++++- src/IO/S3/tests/gtest_aws_s3_client.cpp | 10 ++++- 4 files changed, 70 insertions(+), 34 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 4630e68fbb6..7302e129b4b 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -103,6 +103,20 @@ void verifyClientConfiguration(const Aws::Client::ClientConfiguration & client_c assert_cast(*client_config.retryStrategy); } +void addAdditionalAMZHeadersToCanonicalHeadersList( + Aws::AmazonWebServiceRequest & request, + const HTTPHeaderEntries & extra_headers +) +{ + for (const auto & [name, value] : extra_headers) + { + if (name.starts_with("x-amz-")) + { + request.SetAdditionalCustomHeaderValue(name, value); + } + } +} + } std::unique_ptr Client::create( @@ -265,12 +279,14 @@ template void Client::setKMSHeaders(CreateMultipar template void Client::setKMSHeaders(CopyObjectRequest & request) const; template void Client::setKMSHeaders(PutObjectRequest & request) const; -Model::HeadObjectOutcome Client::HeadObject(const HeadObjectRequest & request) const +Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const { const auto & bucket = request.GetBucket(); request.setApiMode(api_mode); + addAdditionalAMZHeadersToCanonicalHeadersList(request, client_configuration.extra_headers); + if (auto region = getRegionForBucket(bucket); !region.empty()) { if (!detect_region) @@ -346,36 +362,36 @@ Model::HeadObjectOutcome Client::HeadObject(const HeadObjectRequest & request) c /// For each request, we wrap the request functions from Aws::S3::Client with doRequest /// doRequest calls virtuall function from Aws::S3::Client while DB::S3::Client has not virtual calls for each request type -Model::ListObjectsV2Outcome Client::ListObjectsV2(const ListObjectsV2Request & request) const +Model::ListObjectsV2Outcome Client::ListObjectsV2(ListObjectsV2Request & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); } -Model::ListObjectsOutcome Client::ListObjects(const ListObjectsRequest & request) const +Model::ListObjectsOutcome Client::ListObjects(ListObjectsRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); } -Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) const +Model::GetObjectOutcome Client::GetObject(GetObjectRequest & request) const { return doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); }); } -Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(const AbortMultipartUploadRequest & request) const +Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(AbortMultipartUploadRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::AbortMultipartUploadRequest & req) { return AbortMultipartUpload(req); }); } -Model::CreateMultipartUploadOutcome Client::CreateMultipartUpload(const CreateMultipartUploadRequest & request) const +Model::CreateMultipartUploadOutcome Client::CreateMultipartUpload(CreateMultipartUploadRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::CreateMultipartUploadRequest & req) { return CreateMultipartUpload(req); }); } -Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const CompleteMultipartUploadRequest & request) const +Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(CompleteMultipartUploadRequest & request) const { auto outcome = doRequestWithRetryNetworkErrors( request, [this](const Model::CompleteMultipartUploadRequest & req) { return CompleteMultipartUpload(req); }); @@ -422,31 +438,31 @@ Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const Comp return outcome; } -Model::CopyObjectOutcome Client::CopyObject(const CopyObjectRequest & request) const +Model::CopyObjectOutcome Client::CopyObject(CopyObjectRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); } -Model::PutObjectOutcome Client::PutObject(const PutObjectRequest & request) const +Model::PutObjectOutcome Client::PutObject(PutObjectRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); } -Model::UploadPartOutcome Client::UploadPart(const UploadPartRequest & request) const +Model::UploadPartOutcome Client::UploadPart(UploadPartRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); } -Model::UploadPartCopyOutcome Client::UploadPartCopy(const UploadPartCopyRequest & request) const +Model::UploadPartCopyOutcome Client::UploadPartCopy(UploadPartCopyRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); } -Model::DeleteObjectOutcome Client::DeleteObject(const DeleteObjectRequest & request) const +Model::DeleteObjectOutcome Client::DeleteObject(DeleteObjectRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); @@ -458,7 +474,7 @@ Model::DeleteObjectsOutcome Client::DeleteObjects(const DeleteObjectsRequest & r request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); } -Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & request) const +Client::ComposeObjectOutcome Client::ComposeObject(ComposeObjectRequest & request) const { auto request_fn = [this](const ComposeObjectRequest & req) { @@ -490,8 +506,9 @@ Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & template std::invoke_result_t -Client::doRequest(const RequestType & request, RequestFn request_fn) const +Client::doRequest(RequestType & request, RequestFn request_fn) const { + addAdditionalAMZHeadersToCanonicalHeadersList(request, client_configuration.extra_headers); const auto & bucket = request.GetBucket(); request.setApiMode(api_mode); @@ -568,8 +585,9 @@ Client::doRequest(const RequestType & request, RequestFn request_fn) const template std::invoke_result_t -Client::doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const +Client::doRequestWithRetryNetworkErrors(RequestType & request, RequestFn request_fn) const { + addAdditionalAMZHeadersToCanonicalHeadersList(request, client_configuration.extra_headers); auto with_retries = [this, request_fn_ = std::move(request_fn)] (const RequestType & request_) { chassert(client_configuration.retryStrategy); @@ -666,6 +684,8 @@ std::string Client::getRegionForBucket(const std::string & bucket, bool force_de Aws::S3::Model::HeadBucketRequest req; req.SetBucket(bucket); + addAdditionalAMZHeadersToCanonicalHeadersList(req, client_configuration.extra_headers); + std::string region; auto outcome = HeadBucket(req); if (outcome.IsSuccess()) diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 5ad57a9d827..cf0d53d9c72 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -177,24 +177,24 @@ public: template void setKMSHeaders(RequestType & request) const; - Model::HeadObjectOutcome HeadObject(const HeadObjectRequest & request) const; - Model::ListObjectsV2Outcome ListObjectsV2(const ListObjectsV2Request & request) const; - Model::ListObjectsOutcome ListObjects(const ListObjectsRequest & request) const; - Model::GetObjectOutcome GetObject(const GetObjectRequest & request) const; + Model::HeadObjectOutcome HeadObject(HeadObjectRequest & request) const; + Model::ListObjectsV2Outcome ListObjectsV2(ListObjectsV2Request & request) const; + Model::ListObjectsOutcome ListObjects(ListObjectsRequest & request) const; + Model::GetObjectOutcome GetObject(GetObjectRequest & request) const; - Model::AbortMultipartUploadOutcome AbortMultipartUpload(const AbortMultipartUploadRequest & request) const; - Model::CreateMultipartUploadOutcome CreateMultipartUpload(const CreateMultipartUploadRequest & request) const; - Model::CompleteMultipartUploadOutcome CompleteMultipartUpload(const CompleteMultipartUploadRequest & request) const; - Model::UploadPartOutcome UploadPart(const UploadPartRequest & request) const; - Model::UploadPartCopyOutcome UploadPartCopy(const UploadPartCopyRequest & request) const; + Model::AbortMultipartUploadOutcome AbortMultipartUpload(AbortMultipartUploadRequest & request) const; + Model::CreateMultipartUploadOutcome CreateMultipartUpload(CreateMultipartUploadRequest & request) const; + Model::CompleteMultipartUploadOutcome CompleteMultipartUpload(CompleteMultipartUploadRequest & request) const; + Model::UploadPartOutcome UploadPart(UploadPartRequest & request) const; + Model::UploadPartCopyOutcome UploadPartCopy(UploadPartCopyRequest & request) const; - Model::CopyObjectOutcome CopyObject(const CopyObjectRequest & request) const; - Model::PutObjectOutcome PutObject(const PutObjectRequest & request) const; - Model::DeleteObjectOutcome DeleteObject(const DeleteObjectRequest & request) const; - Model::DeleteObjectsOutcome DeleteObjects(const DeleteObjectsRequest & request) const; + Model::CopyObjectOutcome CopyObject(CopyObjectRequest & request) const; + Model::PutObjectOutcome PutObject(PutObjectRequest & request) const; + Model::DeleteObjectOutcome DeleteObject(DeleteObjectRequest & request) const; + Model::DeleteObjectsOutcome DeleteObjects(DeleteObjectsRequest & request) const; using ComposeObjectOutcome = Aws::Utils::Outcome; - ComposeObjectOutcome ComposeObject(const ComposeObjectRequest & request) const; + ComposeObjectOutcome ComposeObject(ComposeObjectRequest & request) const; using Aws::S3::S3Client::EnableRequestProcessing; using Aws::S3::S3Client::DisableRequestProcessing; @@ -236,11 +236,11 @@ private: template std::invoke_result_t - doRequest(const RequestType & request, RequestFn request_fn) const; + doRequest(RequestType & request, RequestFn request_fn) const; template std::invoke_result_t - doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const; + doRequestWithRetryNetworkErrors(RequestType & request, RequestFn request_fn) const; void updateURIForBucket(const std::string & bucket, S3::URI new_uri) const; std::optional getURIFromError(const Aws::S3::S3Error & error) const; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 4a1b6def133..a9dfd03a5e3 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -469,7 +469,17 @@ void PocoHTTPClient::makeRequestInternalImpl( for (const auto & [header_name, header_value] : request.GetHeaders()) poco_request.set(header_name, header_value); for (const auto & [header_name, header_value] : extra_headers) - poco_request.set(boost::algorithm::to_lower_copy(header_name), header_value); + { + // AWS S3 canonical headers must include `Host`, `Content-Type` and any `x-amz-*`. + // These headers will be signed. Custom S3 headers specified in ClickHouse storage conf are added in `extra_headers`. + // At this point in the stack trace, request has already been signed and any `x-amz-*` extra headers was already added + // to the canonical headers list. Therefore, we should not add them again to the request. + // https://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-header-based-auth.html + if (!header_name.starts_with("x-amz-")) + { + poco_request.set(boost::algorithm::to_lower_copy(header_name), header_value); + } + } Poco::Net::HTTPResponse poco_response; diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index bff9ca6fa7b..64c8c7759c9 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -175,7 +175,10 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeadersRead) "host;" "x-amz-api-version;" "x-amz-content-sha256;" - "x-amz-date, ...\n" + "x-amz-date;" + "x-amz-server-side-encryption-customer-algorithm;" + "x-amz-server-side-encryption-customer-key;" + "x-amz-server-side-encryption-customer-key-md5, ...\n" "x-amz-server-side-encryption-customer-algorithm: AES256\n" "x-amz-server-side-encryption-customer-key: Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=\n" "x-amz-server-side-encryption-customer-key-md5: fMNuOw6OLU5GG2vc6RTA+g==\n"); @@ -196,7 +199,10 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeadersWrite) "content-type;" "host;" "x-amz-content-sha256;" - "x-amz-date, ...\n" + "x-amz-date;" + "x-amz-server-side-encryption-customer-algorithm;" + "x-amz-server-side-encryption-customer-key;" + "x-amz-server-side-encryption-customer-key-md5, ...\n" "x-amz-server-side-encryption-customer-algorithm: AES256\n" "x-amz-server-side-encryption-customer-key: Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=\n" "x-amz-server-side-encryption-customer-key-md5: fMNuOw6OLU5GG2vc6RTA+g==\n"); From 3544ee1e5f2826d21a8fa68ed4036f02f1dab595 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 20 Nov 2023 13:52:18 -0300 Subject: [PATCH 611/813] fix build by removing some const specifiers --- src/IO/S3/Client.cpp | 2 +- src/IO/S3/copyS3File.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 7302e129b4b..28a55580dc5 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -468,7 +468,7 @@ Model::DeleteObjectOutcome Client::DeleteObject(DeleteObjectRequest & request) c request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); } -Model::DeleteObjectsOutcome Client::DeleteObjects(const DeleteObjectsRequest & request) const +Model::DeleteObjectsOutcome Client::DeleteObjects(DeleteObjectsRequest & request) const { return doRequestWithRetryNetworkErrors( request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 30da1c580c1..a7b98bfb445 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -488,7 +488,7 @@ namespace client_ptr->setKMSHeaders(request); } - void processPutRequest(const S3::PutObjectRequest & request) + void processPutRequest(S3::PutObjectRequest & request) { size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries, 1UL); for (size_t retries = 1;; ++retries) @@ -675,7 +675,7 @@ namespace client_ptr->setKMSHeaders(request); } - void processCopyRequest(const S3::CopyObjectRequest & request) + void processCopyRequest(S3::CopyObjectRequest & request) { size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries, 1UL); for (size_t retries = 1;; ++retries) From ff438112e95568bb322c9ff8283e6b7f2e1b412a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 20 Nov 2023 18:08:39 +0100 Subject: [PATCH 612/813] Style --- tests/queries/0_stateless/02918_sqlite_path_check.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02918_sqlite_path_check.sh b/tests/queries/0_stateless/02918_sqlite_path_check.sh index 798efda6ec1..fa74b9ecfc8 100755 --- a/tests/queries/0_stateless/02918_sqlite_path_check.sh +++ b/tests/queries/0_stateless/02918_sqlite_path_check.sh @@ -12,4 +12,4 @@ function get_exception_message() } get_exception_message "Select * from sqlite('/etc/passwd', 'something');" -get_exception_message "Select * from sqlite('../../../../etc/passwd', 'something'); +get_exception_message "Select * from sqlite('../../../../etc/passwd', 'something');" From cbb2e02c0342c7703e8cc069378e2e12b9bf930e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Nov 2023 11:41:40 +0000 Subject: [PATCH 613/813] Analyzer: partition pruning for S3 --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 16 +- src/Storages/StorageS3.cpp | 173 +++++++++++++----- src/Storages/StorageS3.h | 12 -- src/Storages/VirtualColumnUtils.cpp | 18 ++ src/Storages/VirtualColumnUtils.h | 1 + tests/analyzer_tech_debt.txt | 1 - .../02302_s3_file_pruning.reference | 10 + .../0_stateless/02302_s3_file_pruning.sql | 12 +- 8 files changed, 173 insertions(+), 70 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d1a285b8818..0bc02ab7395 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -778,23 +778,11 @@ std::optional> MergeTreeDataSelectExecutor::filterPar { if (!filter_dag) return {}; - auto sample = data.getSampleBlockWithVirtualColumns(); - std::unordered_set allowed_inputs; - for (const auto * input : filter_dag->getInputs()) - if (sample.has(input->result_name)) - allowed_inputs.insert(input); - - if (allowed_inputs.empty()) + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(sample, filter_dag, context); + if (!dag) return {}; - auto atoms = filter_dag->extractConjunctionAtoms(filter_dag->getOutputs().at(0)); - atoms = ActionsDAG::filterNodesByAllowedInputs(std::move(atoms), allowed_inputs); - if (atoms.empty()) - return {}; - - auto dag = ActionsDAG::buildFilterActionsDAG(atoms, {}, context); - auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */); VirtualColumnUtils::filterBlockWithQuery(dag, virtual_columns_block, context); return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index bdbba5abd96..2a67c9303e8 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -44,6 +44,8 @@ #include #include +#include +#include #include @@ -126,6 +128,117 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; } +static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, const Strings & paths) +{ + Block virtual_columns_block; + { + for (const auto & column : virtual_columns) + virtual_columns_block.insert({column.type->createColumn(), column.type, column.name}); + + virtual_columns_block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); + + for (size_t i = 0; i != paths.size(); ++i) + { + const auto & path = paths[i]; + if (virtual_columns_block.has("_path")) + virtual_columns_block.getByName("_path").column->assumeMutableRef().insert(path); + + if (virtual_columns_block.has("_file")) + { + auto pos = path.find_last_of('/'); + String file; + if (pos != std::string::npos) + file = path.substr(pos + 1); + else + file = path; + + virtual_columns_block.getByName("_file").column->assumeMutableRef().insert(file); + } + + virtual_columns_block.getByName("_idx").column->assumeMutableRef().insert(i); + } + } + return virtual_columns_block; +} + +static Block renameColumnsInBlock(const Block & source_block, const std::unordered_map & rename_map) +{ + auto columns = source_block.getColumnsWithTypeAndName(); + for (auto & col : columns) + { + auto it = rename_map.find(col.name); + if (it != rename_map.end()) + col.name = it->second; + } + return Block(std::move(columns)); +} + +static ActionsDAGPtr getFilterForPartitionPruning(const SelectQueryInfo & query_info, + const NamesAndTypesList & virtual_columns, + NameToNameMap & column_rename, + ContextPtr context) +{ + if (!query_info.query_tree || !query_info.planner_context) + return nullptr; + + const auto * query_node = query_info.query_tree->as(); + if (!query_node || !query_node->getWhere()) + return nullptr; + + Block header = getBlockWithVirtuals(virtual_columns, "", {}); + + const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); + for (const auto & [column_name, column_identifier] : table_expression_data.getColumnNameToIdentifier()) + column_rename.emplace(column_name, column_identifier); + + header = renameColumnsInBlock(header, column_rename); + auto filter_dag = buildActionsDAGFromExpressionNode( + query_node->getWhere(), header.getColumnsWithTypeAndName(), query_info.planner_context); + + if (filter_dag) + return VirtualColumnUtils::splitFilterDagForAllowedInputs(header, filter_dag, context); + return {}; +} + +static void filterKeysForPartitionPruning(std::vector & keys, const String & bucket, const NamesAndTypesList & virtual_columns, const SelectQueryInfo & query_info, ContextPtr context) +{ + ASTPtr filter_ast; + if (!keys.empty()) + filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query_info.query, virtual_columns, fs::path(bucket) / keys[0], context); + + if (filter_ast) + { + std::vector paths; + paths.reserve(keys.size()); + for (const auto & key : keys) + paths.push_back(fs::path(bucket) / key); + + VirtualColumnUtils::filterByPathOrFile(keys, paths, query_info.query, virtual_columns, context, filter_ast); + LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied AST partition pruning {} from {} keys left", keys.size(), paths.size()); + return; + } + + NameToNameMap column_rename; + auto filter_actions = getFilterForPartitionPruning(query_info, virtual_columns, column_rename, context); + if (filter_actions) + { + auto block = getBlockWithVirtuals(virtual_columns, bucket, keys); + block = renameColumnsInBlock(block, column_rename); + + VirtualColumnUtils::filterBlockWithQuery(filter_actions, block, context); + + String key_column_name = "_key"; + if (auto it = column_rename.find("_key"); it != column_rename.end()) + key_column_name = it->second; + auto filtered_keys = VirtualColumnUtils::extractSingleValueFromBlock(block, key_column_name); + LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied partition pruning {} from {} keys left", filtered_keys.size(), keys.size()); + keys.clear(); + keys.reserve(filtered_keys.size()); + for (auto && key : filtered_keys) + keys.emplace_back(key); + } +} + class IOutputFormat; using OutputFormatPtr = std::shared_ptr; @@ -390,7 +503,7 @@ size_t StorageS3Source::DisclosedGlobIterator::estimatedKeysCount() return pimpl->objectsCount(); } -class StorageS3Source::KeysIterator::Impl : WithContext +class StorageS3Source::KeysIterator::Impl { public: explicit Impl( @@ -399,35 +512,15 @@ public: const std::vector & keys_, const String & bucket_, const S3Settings::RequestSettings & request_settings_, - ASTPtr query_, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, KeysWithInfo * read_keys_, std::function file_progress_callback_) - : WithContext(context_) - , keys(keys_) + : keys(keys_) , client(client_.clone()) , version_id(version_id_) , bucket(bucket_) , request_settings(request_settings_) - , query(query_) - , virtual_columns(virtual_columns_) , file_progress_callback(file_progress_callback_) { - ASTPtr filter_ast; - if (!keys.empty()) - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(bucket) / keys[0], getContext()); - - if (filter_ast) - { - std::vector paths; - paths.reserve(keys.size()); - for (const auto & key : keys) - paths.push_back(fs::path(bucket) / key); - - VirtualColumnUtils::filterByPathOrFile(keys, paths, query, virtual_columns, getContext(), filter_ast); - } - if (read_keys_) { for (const auto & key : keys) @@ -463,8 +556,6 @@ private: String version_id; String bucket; S3Settings::RequestSettings request_settings; - ASTPtr query; - NamesAndTypesList virtual_columns; std::function file_progress_callback; }; @@ -474,14 +565,11 @@ StorageS3Source::KeysIterator::KeysIterator( const std::vector & keys_, const String & bucket_, const S3Settings::RequestSettings & request_settings_, - ASTPtr query, - const NamesAndTypesList & virtual_columns_, - ContextPtr context, KeysWithInfo * read_keys, std::function file_progress_callback_) : pimpl(std::make_shared( client_, version_id_, keys_, bucket_, request_settings_, - query, virtual_columns_, context, read_keys, file_progress_callback_)) + read_keys, file_progress_callback_)) { } @@ -965,8 +1053,6 @@ private: const String key; const std::optional format_settings; - ExpressionActionsPtr partition_by_expr; - static void validateBucket(const String & str) { S3::URI::validateBucket(str, {}); @@ -1038,15 +1124,17 @@ StorageS3::StorageS3( virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } -std::shared_ptr StorageS3::createFileIterator( - const Configuration & configuration, +static std::shared_ptr createFileIterator( + const StorageS3::Configuration & configuration, bool distributed_processing, ContextPtr local_context, - ASTPtr query, + const SelectQueryInfo & query_info, const NamesAndTypesList & virtual_columns, - KeysWithInfo * read_keys, - std::function file_progress_callback) + StorageS3::KeysWithInfo * read_keys = nullptr, + std::function file_progress_callback = {}) { + ASTPtr query = query_info.query; + if (distributed_processing) { return std::make_shared(local_context->getReadTaskCallback(), local_context->getSettingsRef().max_threads); @@ -1060,10 +1148,11 @@ std::shared_ptr StorageS3::createFileIterator( } else { + Strings keys = configuration.keys; + filterKeysForPartitionPruning(keys, configuration.url.bucket, virtual_columns, query_info, local_context); return std::make_shared( - *configuration.client, configuration.url.version_id, configuration.keys, - configuration.url.bucket, configuration.request_settings, query, - virtual_columns, local_context, read_keys, file_progress_callback); + *configuration.client, configuration.url.version_id, keys, + configuration.url.bucket, configuration.request_settings, read_keys, file_progress_callback); } } @@ -1096,10 +1185,10 @@ Pipe StorageS3::read( if (partition_by && query_configuration.withWildcard()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); - Pipes pipes; + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); std::shared_ptr iterator_wrapper = createFileIterator( - query_configuration, distributed_processing, local_context, query_info.query, virtual_columns, nullptr, local_context->getFileProgressCallback()); + query_configuration, distributed_processing, local_context, query_info, virtual_columns, nullptr, local_context->getFileProgressCallback()); size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); if (estimated_keys_count > 1) @@ -1108,7 +1197,6 @@ Pipe StorageS3::read( /// Disclosed glob iterator can underestimate the amount of keys in some cases. We will keep one stream for this particular case. num_streams = 1; - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; @@ -1116,6 +1204,7 @@ Pipe StorageS3::read( const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads); + Pipes pipes; pipes.reserve(num_streams); for (size_t i = 0; i < num_streams; ++i) { @@ -1580,7 +1669,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( { KeysWithInfo read_keys; - auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, &read_keys); + auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, &read_keys); std::optional columns_from_cache; if (ctx->getSettingsRef().schema_inference_use_cache_for_s3) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 3f35c578e19..4f16be0a077 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -102,9 +102,6 @@ public: const std::vector & keys_, const String & bucket_, const S3Settings::RequestSettings & request_settings_, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - ContextPtr context, KeysWithInfo * read_keys = nullptr, std::function progress_callback_ = {}); @@ -400,15 +397,6 @@ private: std::optional format_settings; ASTPtr partition_by; - static std::shared_ptr createFileIterator( - const Configuration & configuration, - bool distributed_processing, - ContextPtr local_context, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - KeysWithInfo * read_keys = nullptr, - std::function progress_callback = {}); - static ColumnsDescription getTableStructureFromDataImpl( const Configuration & configuration, const std::optional & format_settings, diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 219043f25c6..7aec5ce0d78 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -445,6 +445,24 @@ void addRequestedPathAndFileVirtualsToChunk( } } +ActionsDAGPtr splitFilterDagForAllowedInputs(const Block & header, const ActionsDAGPtr & filter_dag, ContextPtr context) +{ + std::unordered_set allowed_inputs; + for (const auto * input : filter_dag->getInputs()) + if (header.has(input->result_name)) + allowed_inputs.insert(input); + + if (allowed_inputs.empty()) + return {}; + + auto atoms = filter_dag->extractConjunctionAtoms(filter_dag->getOutputs().at(0)); + atoms = ActionsDAG::filterNodesByAllowedInputs(std::move(atoms), allowed_inputs); + if (atoms.empty()) + return {}; + + return ActionsDAG::buildFilterActionsDAG(atoms, {}, context); +} + } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index a21f2b05552..a717d2a0197 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -34,6 +34,7 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block /// If `expression_ast` is passed, use it to filter block. void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast = {}); void filterBlockWithQuery(ActionsDAGPtr dag, Block & block, ContextPtr context); +ActionsDAGPtr splitFilterDagForAllowedInputs(const Block & header, const ActionsDAGPtr & filter_dag, ContextPtr context); /// Extract from the input stream a set of `name` column values template diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 392c4e2e313..e0055a9d617 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -24,7 +24,6 @@ 01952_optimize_distributed_group_by_sharding_key 02139_MV_with_scalar_subquery 02174_cte_scalar_cache_mv -02302_s3_file_pruning 02352_grouby_shadows_arg 02354_annoy 02428_parameterized_view diff --git a/tests/queries/0_stateless/02302_s3_file_pruning.reference b/tests/queries/0_stateless/02302_s3_file_pruning.reference index f8d2bdd0612..7e69bdd55db 100644 --- a/tests/queries/0_stateless/02302_s3_file_pruning.reference +++ b/tests/queries/0_stateless/02302_s3_file_pruning.reference @@ -24,4 +24,14 @@ insert into test_02302 select 1 settings s3_create_new_file_on_insert = true; insert into test_02302 select 2 settings s3_create_new_file_on_insert = true; select * from test_02302 where _file like '%1'; 1 +select _file, * from test_02302 where _file like '%1'; +test_02302.1 1 +set max_rows_to_read = 2; +select * from test_02302 where (_file like '%.1' OR _file like '%.2') AND a > 1; +2 +set max_rows_to_read = 999; +select 'a1' as _file, * from test_02302 where _file like '%1' ORDER BY a; +a1 0 +a1 1 +a1 2 drop table test_02302; diff --git a/tests/queries/0_stateless/02302_s3_file_pruning.sql b/tests/queries/0_stateless/02302_s3_file_pruning.sql index 624a87506d1..93fc8a1bc25 100644 --- a/tests/queries/0_stateless/02302_s3_file_pruning.sql +++ b/tests/queries/0_stateless/02302_s3_file_pruning.sql @@ -1,5 +1,5 @@ -- Tags: no-parallel, no-fasttest --- Tag no-fasttest: Depends on AWS +-- Tag no-fasttest: Depends on S3 -- { echo } drop table if exists test_02302; @@ -32,4 +32,14 @@ insert into test_02302 select 1 settings s3_create_new_file_on_insert = true; insert into test_02302 select 2 settings s3_create_new_file_on_insert = true; select * from test_02302 where _file like '%1'; + +select _file, * from test_02302 where _file like '%1'; + +set max_rows_to_read = 2; +select * from test_02302 where (_file like '%.1' OR _file like '%.2') AND a > 1; + +set max_rows_to_read = 999; + +select 'a1' as _file, * from test_02302 where _file like '%1' ORDER BY a; + drop table test_02302; From 1cddfb1e6b775b452715d68ee566e56794081dfc Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Nov 2023 11:42:23 +0000 Subject: [PATCH 614/813] rewrite getBlockWithVirtuals for S3Storage --- src/Storages/StorageS3.cpp | 61 +++++++++++++++++++++++++------------- 1 file changed, 41 insertions(+), 20 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2a67c9303e8..2f1288c9067 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -128,36 +128,57 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; } -static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, const Strings & paths) +static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, const String & bucket, const Strings & keys) { Block virtual_columns_block; + fs::path bucket_path(bucket); + + for (const auto & [column_name, column_type] : virtual_columns) { - for (const auto & column : virtual_columns) - virtual_columns_block.insert({column.type->createColumn(), column.type, column.name}); - - virtual_columns_block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); - - for (size_t i = 0; i != paths.size(); ++i) + if (column_name == "_path") { - const auto & path = paths[i]; - if (virtual_columns_block.has("_path")) - virtual_columns_block.getByName("_path").column->assumeMutableRef().insert(path); - - if (virtual_columns_block.has("_file")) + auto column = column_type->createColumn(); + for (const auto & key : keys) + column->insert((bucket_path / key).string()); + virtual_columns_block.insert({std::move(column), column_type, column_name}); + } + else if (column_name == "_file") + { + auto column = column_type->createColumn(); + for (const auto & key : keys) { - auto pos = path.find_last_of('/'); - String file; + auto pos = key.find_last_of('/'); if (pos != std::string::npos) - file = path.substr(pos + 1); + column->insert(key.substr(pos + 1)); else - file = path; - - virtual_columns_block.getByName("_file").column->assumeMutableRef().insert(file); + column->insert(key); } - - virtual_columns_block.getByName("_idx").column->assumeMutableRef().insert(i); + virtual_columns_block.insert({std::move(column), column_type, column_name}); + } + else if (column_name == "_key") + { + auto column = column_type->createColumn(); + for (const auto & key : keys) + column->insert(key); + virtual_columns_block.insert({std::move(column), column_type, column_name}); + } + else + { + auto column = column_type->createColumn(); + column->insertManyDefaults(keys.size()); + virtual_columns_block.insert({std::move(column), column_type, column_name}); } } + + /// Column _key is mandatory and may not be in virtual_columns list + if (!virtual_columns_block.has("_key")) + { + auto column_type = std::make_shared(); + auto column = column_type->createColumn(); for (const auto & key : keys) + column->insert(key); + virtual_columns_block.insert({std::move(column), column_type, "_key"}); + } + return virtual_columns_block; } From a915eeded805b736a2e70e46648f7048292c7a75 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Nov 2023 17:49:16 +0000 Subject: [PATCH 615/813] StorageS3 use filters from SourceStepWithFilter --- src/Storages/StorageS3.cpp | 148 +++++++++++++++++++++++-------------- src/Storages/StorageS3.h | 4 +- 2 files changed, 96 insertions(+), 56 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2f1288c9067..3e12660997f 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -42,6 +42,8 @@ #include #include #include +#include + #include #include @@ -128,6 +130,48 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; } + +class ReadFromStorageS3Step : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromStorageS3Step"; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + void applyFilters() override; + + ReadFromStorageS3Step( + Block sample_block, + const Names & column_names_, + StorageSnapshotPtr storage_snapshot_, + StorageS3 & storage_, + SelectQueryInfo query_info_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) + , column_names(column_names_) + , storage_snapshot(std::move(storage_snapshot_)) + , storage(storage_) + , query_info(std::move(query_info_)) + , local_context(std::move(context_)) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + { + } + +private: + Names column_names; + StorageSnapshotPtr storage_snapshot; + StorageS3 & storage; + SelectQueryInfo query_info; + ContextPtr local_context; + + size_t max_block_size; + size_t num_streams; +}; + + static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, const String & bucket, const Strings & keys) { Block virtual_columns_block; @@ -182,50 +226,17 @@ static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, con return virtual_columns_block; } -static Block renameColumnsInBlock(const Block & source_block, const std::unordered_map & rename_map) +static void filterKeysForPartitionPruning(std::vector & keys, + const String & bucket, + const NamesAndTypesList & virtual_columns, + const SelectQueryInfo & query_info, + const std::vector & filter_dags, + ContextPtr context) { - auto columns = source_block.getColumnsWithTypeAndName(); - for (auto & col : columns) - { - auto it = rename_map.find(col.name); - if (it != rename_map.end()) - col.name = it->second; - } - return Block(std::move(columns)); -} + if (keys.empty()) + return; -static ActionsDAGPtr getFilterForPartitionPruning(const SelectQueryInfo & query_info, - const NamesAndTypesList & virtual_columns, - NameToNameMap & column_rename, - ContextPtr context) -{ - if (!query_info.query_tree || !query_info.planner_context) - return nullptr; - - const auto * query_node = query_info.query_tree->as(); - if (!query_node || !query_node->getWhere()) - return nullptr; - - Block header = getBlockWithVirtuals(virtual_columns, "", {}); - - const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); - for (const auto & [column_name, column_identifier] : table_expression_data.getColumnNameToIdentifier()) - column_rename.emplace(column_name, column_identifier); - - header = renameColumnsInBlock(header, column_rename); - auto filter_dag = buildActionsDAGFromExpressionNode( - query_node->getWhere(), header.getColumnsWithTypeAndName(), query_info.planner_context); - - if (filter_dag) - return VirtualColumnUtils::splitFilterDagForAllowedInputs(header, filter_dag, context); - return {}; -} - -static void filterKeysForPartitionPruning(std::vector & keys, const String & bucket, const NamesAndTypesList & virtual_columns, const SelectQueryInfo & query_info, ContextPtr context) -{ - ASTPtr filter_ast; - if (!keys.empty()) - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query_info.query, virtual_columns, fs::path(bucket) / keys[0], context); + ASTPtr filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query_info.query, virtual_columns, fs::path(bucket) / keys[0], context); if (filter_ast) { @@ -240,12 +251,13 @@ static void filterKeysForPartitionPruning(std::vector & keys, const Stri } NameToNameMap column_rename; - auto filter_actions = getFilterForPartitionPruning(query_info, virtual_columns, column_rename, context); - if (filter_actions) + for (const auto & filter_dag : filter_dags) { auto block = getBlockWithVirtuals(virtual_columns, bucket, keys); - block = renameColumnsInBlock(block, column_rename); + auto filter_actions = VirtualColumnUtils::splitFilterDagForAllowedInputs(block, filter_dag, context); + if (!filter_actions) + continue; VirtualColumnUtils::filterBlockWithQuery(filter_actions, block, context); String key_column_name = "_key"; @@ -1150,6 +1162,7 @@ static std::shared_ptr createFileIterator( bool distributed_processing, ContextPtr local_context, const SelectQueryInfo & query_info, + const std::vector & filter_dags, const NamesAndTypesList & virtual_columns, StorageS3::KeysWithInfo * read_keys = nullptr, std::function file_progress_callback = {}) @@ -1170,7 +1183,7 @@ static std::shared_ptr createFileIterator( else { Strings keys = configuration.keys; - filterKeysForPartitionPruning(keys, configuration.url.bucket, virtual_columns, query_info, local_context); + filterKeysForPartitionPruning(keys, configuration.url.bucket, virtual_columns, query_info, filter_dags, local_context); return std::make_shared( *configuration.client, configuration.url.version_id, keys, configuration.url.bucket, configuration.request_settings, read_keys, file_progress_callback); @@ -1192,7 +1205,8 @@ bool StorageS3::parallelizeOutputAfterReading(ContextPtr context) const return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration.format, context); } -Pipe StorageS3::read( +void StorageS3::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -1201,15 +1215,34 @@ Pipe StorageS3::read( size_t max_block_size, size_t num_streams) { - auto query_configuration = updateConfigurationAndGetCopy(local_context); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), virtual_columns); - if (partition_by && query_configuration.withWildcard()) + auto reading = std::make_unique( + read_from_format_info.source_header, + column_names, + storage_snapshot, + *this, + query_info, + local_context, + max_block_size, + num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + auto query_configuration = storage.updateConfigurationAndGetCopy(local_context); + + if (storage.partition_by && query_configuration.withWildcard()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); + auto virtual_columns = storage.getVirtuals(); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, storage.supportsSubsetOfColumns(local_context), virtual_columns); std::shared_ptr iterator_wrapper = createFileIterator( - query_configuration, distributed_processing, local_context, query_info, virtual_columns, nullptr, local_context->getFileProgressCallback()); + query_configuration, storage.distributed_processing, local_context, query_info, filter_dags, + virtual_columns, nullptr, local_context->getFileProgressCallback()); size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); if (estimated_keys_count > 1) @@ -1232,9 +1265,9 @@ Pipe StorageS3::read( pipes.emplace_back(std::make_shared( read_from_format_info, query_configuration.format, - getName(), + storage.getName(), local_context, - format_settings, + storage.format_settings, max_block_size, query_configuration.request_settings, query_configuration.compression_method, @@ -1248,7 +1281,12 @@ Pipe StorageS3::read( query_info)); } - return Pipe::unitePipes(std::move(pipes)); + pipeline.init(Pipe::unitePipes(std::move(pipes))); +} + + +void ReadFromStorageS3Step::applyFilters() +{ } SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) @@ -1690,7 +1728,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( { KeysWithInfo read_keys; - auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, &read_keys); + auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, {}, &read_keys); std::optional columns_from_cache; if (ctx->getSettingsRef().schema_inference_use_cache_for_s3) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 4f16be0a077..f3b82cd2849 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -327,7 +327,8 @@ public: return name; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -387,6 +388,7 @@ private: friend class StorageS3Cluster; friend class TableFunctionS3Cluster; friend class StorageS3Queue; + friend class ReadFromStorageS3Step; Configuration configuration; std::mutex configuration_update_mutex; From 95e9a274176c46066afb8cbd1c125b0cf2cc96a4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Nov 2023 17:59:07 +0000 Subject: [PATCH 616/813] Remove ast based code from filterKeysForPartitionPruning --- src/Storages/StorageS3.cpp | 35 +++++++---------------------------- 1 file changed, 7 insertions(+), 28 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3e12660997f..0dc163981e9 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -229,30 +229,14 @@ static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, con static void filterKeysForPartitionPruning(std::vector & keys, const String & bucket, const NamesAndTypesList & virtual_columns, - const SelectQueryInfo & query_info, const std::vector & filter_dags, ContextPtr context) { - if (keys.empty()) - return; - - ASTPtr filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query_info.query, virtual_columns, fs::path(bucket) / keys[0], context); - - if (filter_ast) - { - std::vector paths; - paths.reserve(keys.size()); - for (const auto & key : keys) - paths.push_back(fs::path(bucket) / key); - - VirtualColumnUtils::filterByPathOrFile(keys, paths, query_info.query, virtual_columns, context, filter_ast); - LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied AST partition pruning {} from {} keys left", keys.size(), paths.size()); - return; - } - - NameToNameMap column_rename; for (const auto & filter_dag : filter_dags) { + if (keys.empty()) + break; + auto block = getBlockWithVirtuals(virtual_columns, bucket, keys); auto filter_actions = VirtualColumnUtils::splitFilterDagForAllowedInputs(block, filter_dag, context); @@ -260,10 +244,7 @@ static void filterKeysForPartitionPruning(std::vector & keys, continue; VirtualColumnUtils::filterBlockWithQuery(filter_actions, block, context); - String key_column_name = "_key"; - if (auto it = column_rename.find("_key"); it != column_rename.end()) - key_column_name = it->second; - auto filtered_keys = VirtualColumnUtils::extractSingleValueFromBlock(block, key_column_name); + std::unordered_set filtered_keys = VirtualColumnUtils::extractSingleValueFromBlock(block, "_key"); LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied partition pruning {} from {} keys left", filtered_keys.size(), keys.size()); keys.clear(); keys.reserve(filtered_keys.size()); @@ -1161,14 +1142,12 @@ static std::shared_ptr createFileIterator( const StorageS3::Configuration & configuration, bool distributed_processing, ContextPtr local_context, - const SelectQueryInfo & query_info, + ASTPtr query, const std::vector & filter_dags, const NamesAndTypesList & virtual_columns, StorageS3::KeysWithInfo * read_keys = nullptr, std::function file_progress_callback = {}) { - ASTPtr query = query_info.query; - if (distributed_processing) { return std::make_shared(local_context->getReadTaskCallback(), local_context->getSettingsRef().max_threads); @@ -1183,7 +1162,7 @@ static std::shared_ptr createFileIterator( else { Strings keys = configuration.keys; - filterKeysForPartitionPruning(keys, configuration.url.bucket, virtual_columns, query_info, filter_dags, local_context); + filterKeysForPartitionPruning(keys, configuration.url.bucket, virtual_columns, filter_dags, local_context); return std::make_shared( *configuration.client, configuration.url.version_id, keys, configuration.url.bucket, configuration.request_settings, read_keys, file_progress_callback); @@ -1241,7 +1220,7 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, storage.supportsSubsetOfColumns(local_context), virtual_columns); std::shared_ptr iterator_wrapper = createFileIterator( - query_configuration, storage.distributed_processing, local_context, query_info, filter_dags, + query_configuration, storage.distributed_processing, local_context, query_info.query, filter_dags, virtual_columns, nullptr, local_context->getFileProgressCallback()); size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); From 31a6c7c1c49da5f9ea42f97bee7e617cebabaebe Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Nov 2023 18:08:45 +0000 Subject: [PATCH 617/813] Style changes around filterKeysForPartitionPruning --- src/Storages/StorageS3.cpp | 32 +++++++++++++++---------------- src/Storages/VirtualColumnUtils.h | 2 ++ 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 0dc163981e9..594f973bc0d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -172,7 +172,7 @@ private: }; -static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, const String & bucket, const Strings & keys) +static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, const String & bucket, const std::unordered_set & keys) { Block virtual_columns_block; fs::path bucket_path(bucket); @@ -226,31 +226,31 @@ static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, con return virtual_columns_block; } -static void filterKeysForPartitionPruning(std::vector & keys, - const String & bucket, - const NamesAndTypesList & virtual_columns, - const std::vector & filter_dags, - ContextPtr context) +static std::vector filterKeysForPartitionPruning( + const std::vector & keys, + const String & bucket, + const NamesAndTypesList & virtual_columns, + const std::vector & filter_dags, + ContextPtr context) { + std::unordered_set result_keys(keys.begin(), keys.end()); for (const auto & filter_dag : filter_dags) { - if (keys.empty()) + if (result_keys.empty()) break; - auto block = getBlockWithVirtuals(virtual_columns, bucket, keys); + auto block = getBlockWithVirtuals(virtual_columns, bucket, result_keys); auto filter_actions = VirtualColumnUtils::splitFilterDagForAllowedInputs(block, filter_dag, context); if (!filter_actions) continue; VirtualColumnUtils::filterBlockWithQuery(filter_actions, block, context); - std::unordered_set filtered_keys = VirtualColumnUtils::extractSingleValueFromBlock(block, "_key"); - LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied partition pruning {} from {} keys left", filtered_keys.size(), keys.size()); - keys.clear(); - keys.reserve(filtered_keys.size()); - for (auto && key : filtered_keys) - keys.emplace_back(key); + result_keys = VirtualColumnUtils::extractSingleValueFromBlock(block, "_key"); } + + LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied partition pruning {} from {} keys left", result_keys.size(), keys.size()); + return std::vector(result_keys.begin(), result_keys.end()); } class IOutputFormat; @@ -1161,8 +1161,7 @@ static std::shared_ptr createFileIterator( } else { - Strings keys = configuration.keys; - filterKeysForPartitionPruning(keys, configuration.url.bucket, virtual_columns, filter_dags, local_context); + Strings keys = filterKeysForPartitionPruning(configuration.keys, configuration.url.bucket, virtual_columns, filter_dags, local_context); return std::make_shared( *configuration.client, configuration.url.version_id, keys, configuration.url.bucket, configuration.request_settings, read_keys, file_progress_callback); @@ -1266,6 +1265,7 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, void ReadFromStorageS3Step::applyFilters() { + /// We will use filter_dags in filterKeysForPartitionPruning called from initializePipeline, nothing to do here } SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index a717d2a0197..d8d4b44b4ff 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -34,6 +34,8 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block /// If `expression_ast` is passed, use it to filter block. void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast = {}); void filterBlockWithQuery(ActionsDAGPtr dag, Block & block, ContextPtr context); + +/// Extract subset of filter_dag that can be evaluated using only columns from header ActionsDAGPtr splitFilterDagForAllowedInputs(const Block & header, const ActionsDAGPtr & filter_dag, ContextPtr context); /// Extract from the input stream a set of `name` column values From 897cd06bcf2a7e75998133cad9a5ad668366231b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 20 Nov 2023 19:15:49 +0100 Subject: [PATCH 618/813] Fix dropping tables in test "test_create_or_drop_tables_during_backup". --- .../test_backup_restore_on_cluster/test_concurrency.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index aea82c6b559..ab37846db9a 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -214,7 +214,13 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): while time.time() < end_time: table_name = f"mydb.tbl{randint(1, num_nodes)}" node = nodes[randint(0, num_nodes - 1)] - node.query(f"DROP TABLE IF EXISTS {table_name} SYNC") + # "DROP TABLE IF EXISTS" still can throw some errors (e.g. "WRITE locking attempt on node0 has timed out!") + # So we use query_and_get_answer_with_error() to ignore any errors. + # `lock_acquire_timeout` is also reduced because we don't wait our test to wait too long. + node.query_and_get_answer_with_error( + f"DROP TABLE IF EXISTS {table_name} SYNC", + settings={"lock_acquire_timeout": 10}, + ) def rename_tables(): while time.time() < end_time: From 852e983fe3086a6a6aa7b2ba3461b124fe30a6f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Nov 2023 21:34:22 +0100 Subject: [PATCH 619/813] Follow-up --- packages/clickhouse-server.yaml | 2 -- 1 file changed, 2 deletions(-) diff --git a/packages/clickhouse-server.yaml b/packages/clickhouse-server.yaml index 5e2bc7c7412..7894129b8e3 100644 --- a/packages/clickhouse-server.yaml +++ b/packages/clickhouse-server.yaml @@ -52,8 +52,6 @@ contents: dst: /lib/systemd/system/clickhouse-server.service - src: root/usr/bin/clickhouse-copier dst: /usr/bin/clickhouse-copier -- src: root/usr/bin/clickhouse-report - dst: /usr/bin/clickhouse-report - src: root/usr/bin/clickhouse-server dst: /usr/bin/clickhouse-server # clickhouse-keeper part From 3c2cf5dc7018b23cbb465ed9f906768fb1314a19 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Nov 2023 21:52:47 +0100 Subject: [PATCH 620/813] Enable Analyzer in Stress and Fuzz tests --- docker/test/fuzzer/query-fuzzer-tweaks-users.xml | 5 ----- docker/test/stateless/stress_tests.lib | 15 --------------- 2 files changed, 20 deletions(-) diff --git a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml index ecd7aae2e4a..023f257253a 100644 --- a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml +++ b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml @@ -23,11 +23,6 @@ 10G - - - - - 200 diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 551461b6eca..8f89c1b80dd 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -140,21 +140,6 @@ EOL --> $PWD -EOL - - # Analyzer is not yet ready for testing - cat > /etc/clickhouse-server/users.d/no_analyzer.xml < - - - - - - - - - - EOL } From ff6dfd2490cee6300147ddea4f5eb1ffcfacb08b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 20 Nov 2023 21:47:17 +0000 Subject: [PATCH 621/813] Run CI for PRs with missing documentation if 'can be tested' label is present --- tests/ci/run_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index db98a2c1ab5..231e2617a3f 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -150,7 +150,7 @@ def main(): DOCS_NAME, pr_info, ) - sys.exit(1) + sys.exit(0) if description_error: print( From 524edd8460431b447c4bcd2f932bed489f7a83f6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 21 Nov 2023 00:08:20 +0100 Subject: [PATCH 622/813] turn test off --- tests/integration/test_merge_tree_s3/test.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 3b2f1c0f6a6..0856b1681f8 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -851,6 +851,8 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): def test_s3_engine_heavy_write_check_mem( cluster, broken_s3, node_name, in_flight_memory ): + pytest.skip("Disabled, will be fixed after https://github.com/ClickHouse/ClickHouse/issues/51152") + in_flight = in_flight_memory[0] memory = in_flight_memory[1] @@ -870,12 +872,16 @@ def test_s3_engine_heavy_write_check_mem( ) broken_s3.setup_fake_multpartuploads() - broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=15, count=10) + slow_responces = 10 + slow_timeout = 15 + broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=slow_timeout, count=slow_responces) query_id = f"INSERT_INTO_S3_ENGINE_QUERY_ID_{in_flight}" node.query( "INSERT INTO s3_test SELECT number, toString(number) FROM numbers(50000000)" - f" SETTINGS max_memory_usage={2*memory}" + f" SETTINGS " + f" max_memory_usage={2*memory}" + # f", max_threads=1" # ParallelFormattingOutputFormat consumption depends on it f", s3_max_inflight_parts_for_one_file={in_flight}", query_id=query_id, ) @@ -892,7 +898,8 @@ def test_s3_engine_heavy_write_check_mem( assert int(memory_usage) < 1.2 * memory assert int(memory_usage) > 0.8 * memory - assert int(wait_inflight) > in_flight * 1000 * 1000 + # The more in_flight value is the less time CH waits. + assert int(wait_inflight) / 1000 / 1000 > slow_responces * slow_timeout / in_flight check_no_objects_after_drop(cluster, node_name=node_name) From 3eef0601e3404a9b1238f24f0f2deab571e4a17c Mon Sep 17 00:00:00 2001 From: santrancisco Date: Tue, 21 Nov 2023 16:29:40 +1100 Subject: [PATCH 623/813] Fix file path validation for DatabaseFileSystem --- src/Databases/DatabaseFilesystem.cpp | 6 ++++-- .../02921_database_filesystem_path_check.reference | 2 ++ .../02921_database_filesystem_path_check.sh | 13 +++++++++++++ 3 files changed, 19 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02921_database_filesystem_path_check.reference create mode 100755 tests/queries/0_stateless/02921_database_filesystem_path_check.sh diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 49f260034db..b6a5f95a5f7 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -40,13 +40,15 @@ DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path { path = user_files_path / path; } - else if (!is_local && !pathStartsWith(fs::path(path), user_files_path)) + + path = fs::absolute(path).lexically_normal(); + + if (!is_local && !pathStartsWith(fs::path(path), user_files_path)) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path must be inside user-files path: {}", user_files_path.string()); } - path = fs::absolute(path).lexically_normal(); if (!fs::exists(path)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path does not exist: {}", path); } diff --git a/tests/queries/0_stateless/02921_database_filesystem_path_check.reference b/tests/queries/0_stateless/02921_database_filesystem_path_check.reference new file mode 100644 index 00000000000..21ebc92b43f --- /dev/null +++ b/tests/queries/0_stateless/02921_database_filesystem_path_check.reference @@ -0,0 +1,2 @@ +Path must be inside user-files path +Path must be inside user-files path \ No newline at end of file diff --git a/tests/queries/0_stateless/02921_database_filesystem_path_check.sh b/tests/queries/0_stateless/02921_database_filesystem_path_check.sh new file mode 100755 index 00000000000..79315d89f07 --- /dev/null +++ b/tests/queries/0_stateless/02921_database_filesystem_path_check.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function get_exception_message() +{ + $CLICKHOUSE_CLIENT --query "$1" |& grep -o 'Path must.*path' +} + +get_exception_message "create database db_filesystem ENGINE=Filesystem('/etc');" +get_exception_message "create database db_filesystem ENGINE=Filesystem('../../../../../../../../etc')';" \ No newline at end of file From f4d936fc554ae8af7a2cf01721995aa8e5b143a7 Mon Sep 17 00:00:00 2001 From: santrancisco Date: Tue, 21 Nov 2023 17:39:12 +1100 Subject: [PATCH 624/813] Fix style --- src/Databases/DatabaseFilesystem.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index b6a5f95a5f7..ca1b5b27a59 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -40,9 +40,9 @@ DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path { path = user_files_path / path; } - + path = fs::absolute(path).lexically_normal(); - + if (!is_local && !pathStartsWith(fs::path(path), user_files_path)) { throw Exception(ErrorCodes::BAD_ARGUMENTS, From 15234474d7a570f6b4fc0c4eae5cfb3718449316 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Nov 2023 10:03:57 +0000 Subject: [PATCH 625/813] Implement system table blob_storage_log --- .../settings.md | 2 +- .../system-tables/blob_storage_log.md | 59 +++++++++++ programs/server/config.xml | 10 ++ src/Backups/BackupIO_S3.cpp | 39 +++++++- src/Backups/BackupIO_S3.h | 6 +- src/Common/SystemLogBase.cpp | 1 + src/Common/SystemLogBase.h | 3 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 5 +- src/Coordination/Standalone/Context.cpp | 5 + src/Coordination/Standalone/Context.h | 2 + src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- .../ObjectStorages/DiskObjectStorage.cpp | 9 +- .../DiskObjectStorageTransaction.cpp | 4 +- src/Disks/ObjectStorages/IObjectStorage.h | 4 - .../MetadataStorageFromDisk.cpp | 2 +- .../MetadataStorageFromPlainObjectStorage.cpp | 2 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 28 +++++- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 6 +- .../ObjectStorages/S3/registerDiskS3.cpp | 11 ++- src/Disks/ObjectStorages/StoredObject.h | 21 +--- ...etadataStorageFromStaticFilesWebServer.cpp | 2 +- src/IO/S3/BlobStorageLogWriter.cpp | 72 ++++++++++++++ src/IO/S3/BlobStorageLogWriter.h | 57 +++++++++++ src/IO/S3/copyS3File.cpp | 53 ++++++++-- src/IO/S3/copyS3File.h | 3 + src/IO/S3/tests/gtest_aws_s3_client.cpp | 3 +- src/IO/WriteBufferFromS3.cpp | 26 ++++- src/IO/WriteBufferFromS3.h | 4 + src/IO/tests/gtest_writebuffer_s3.cpp | 3 +- src/Interpreters/BlobStorageLog.cpp | 92 +++++++++++++++++ src/Interpreters/BlobStorageLog.h | 57 +++++++++++ src/Interpreters/Context.cpp | 11 ++- src/Interpreters/Context.h | 2 + src/Interpreters/InterpreterSystemQuery.cpp | 1 + src/Interpreters/SystemLog.cpp | 4 + src/Interpreters/SystemLog.h | 3 + src/Storages/S3Queue/StorageS3Queue.cpp | 7 +- src/Storages/S3Queue/StorageS3Queue.h | 3 + src/Storages/StorageAzureBlob.cpp | 2 +- src/Storages/StorageS3.cpp | 19 +++- src/Storages/StorageS3.h | 1 + tests/config/config.d/blob_storage_log.xml | 9 ++ tests/config/install.sh | 1 + .../configs/blob_log.xml | 9 ++ .../test_backup_restore_s3/test.py | 28 +++++- .../configs/config.d/blob_log.xml | 9 ++ tests/integration/test_merge_tree_s3/test.py | 99 ++++++++++++++++--- .../test_storage_s3/configs/blob_log.xml | 9 ++ tests/integration/test_storage_s3/test.py | 58 +++++++++-- 49 files changed, 782 insertions(+), 86 deletions(-) create mode 100644 docs/en/operations/system-tables/blob_storage_log.md create mode 100644 src/IO/S3/BlobStorageLogWriter.cpp create mode 100644 src/IO/S3/BlobStorageLogWriter.h create mode 100644 src/Interpreters/BlobStorageLog.cpp create mode 100644 src/Interpreters/BlobStorageLog.h create mode 100644 tests/config/config.d/blob_storage_log.xml create mode 100644 tests/integration/test_backup_restore_s3/configs/blob_log.xml create mode 100644 tests/integration/test_merge_tree_s3/configs/config.d/blob_log.xml create mode 100644 tests/integration/test_storage_s3/configs/blob_log.xml diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index cfc5a939a0e..3e4f1f4313f 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2740,7 +2740,7 @@ ClickHouse will use it to form the proxy URI using the following template: `{pro 10 - + http://resolver:8080/hostname diff --git a/docs/en/operations/system-tables/blob_storage_log.md b/docs/en/operations/system-tables/blob_storage_log.md new file mode 100644 index 00000000000..db08b0c583d --- /dev/null +++ b/docs/en/operations/system-tables/blob_storage_log.md @@ -0,0 +1,59 @@ +--- +slug: /en/operations/system-tables/blob_storage_log +--- +# Blob Storage Operations Log + +Contains logging entries with information about various blob storage operations such as uploads and deletes. + +Columns: + +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Date of the event. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time of the event. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Time of the event with microseconds precision. +- `event_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Type of the event. Possible values: + - `'Upload'` + - `'Delete'` + - `'MultiPartUploadCreate'` + - `'MultiPartUploadWrite'` + - `'MultiPartUploadComplete'` + - `'MultiPartUploadAbort'` +- `query_id` ([String](../../sql-reference/data-types/string.md)) — Identifier of the query associated with the event, if any. +- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Identifier of the thread performing the operation. +- `thread_name` ([String](../../sql-reference/data-types/string.md)) — Name of the thread performing the operation. +- `disk_name` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Name of the associated disk. +- `bucket` ([String](../../sql-reference/data-types/string.md)) — Name of the bucket. +- `remote_path` ([String](../../sql-reference/data-types/string.md)) — Path to the remote resource. +- `local_path` ([String](../../sql-reference/data-types/string.md)) — Path to the metadata file on the local system, which references the remote resource. +- `data_size` ([UInt32](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Size of the data involved in the upload event. +- `error` ([String](../../sql-reference/data-types/string.md)) — Error message associated with the event, if any. + +**Example** + +Suppose a blob storage operation uploads a file, and an event is logged: + +```sql +SELECT * FROM system.blob_storage_log WHERE query_id = '7afe0450-504d-4e4b-9a80-cd9826047972' ORDER BY event_date, event_time_microseconds \G +``` + +```text +Row 1: +────── +event_date: 2023-10-31 +event_time: 2023-10-31 16:03:40 +event_time_microseconds: 2023-10-31 16:03:40.481437 +event_type: Upload +query_id: 7afe0450-504d-4e4b-9a80-cd9826047972 +thread_id: 2381740 +disk_name: disk_s3 +bucket: bucket1 +remote_path: rrr/kxo/tbnqtrghgtnxkzgtcrlutwuslgawe +local_path: store/654/6549e8b3-d753-4447-8047-d462df6e6dbe/tmp_insert_all_1_1_0/checksums.txt +data_size: 259 +error: +``` + +In this example, upload operation was associated with the `INSERT` query with ID `7afe0450-504d-4e4b-9a80-cd9826047972`. The local metadata file `store/654/6549e8b3-d753-4447-8047-d462df6e6dbe/tmp_insert_all_1_1_0/checksums.txt` refers to remote path `rrr/kxo/tbnqtrghgtnxkzgtcrlutwuslgawe` in bucket `bucket1` on disk `disk_s3`, with a size of 259 bytes. + +**See Also** + +- [External Disks for Storing Data](../../operations/storing-data.md) diff --git a/programs/server/config.xml b/programs/server/config.xml index 7800aa51166..4d7d9ab4d5a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1248,6 +1248,16 @@ 7500 + + + system + blob_storage_log
+ toYYYYMM(event_date) + 7500 + event_date + INTERVAL 30 DAY +
+ + + system + s3queue_log
+ toYYYYMM(event_date) + 7500 +
+ - false + true *_function.*ml diff --git a/tests/integration/test_dictionaries_wait_for_load/configs/no_dictionaries_lazy_load.xml b/tests/integration/test_dictionaries_wait_for_load/configs/no_dictionaries_lazy_load.xml new file mode 100644 index 00000000000..aaae3e0c4c1 --- /dev/null +++ b/tests/integration/test_dictionaries_wait_for_load/configs/no_dictionaries_lazy_load.xml @@ -0,0 +1,3 @@ + + 0 + diff --git a/tests/integration/test_dictionaries_wait_for_load/configs/wait_for_dictionaries_load.xml b/tests/integration/test_dictionaries_wait_for_load/configs/wait_for_dictionaries_load.xml deleted file mode 100644 index a446b730123..00000000000 --- a/tests/integration/test_dictionaries_wait_for_load/configs/wait_for_dictionaries_load.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_dictionaries_wait_for_load/test.py b/tests/integration/test_dictionaries_wait_for_load/test.py index 975e9ca3e56..b30cc61abce 100644 --- a/tests/integration/test_dictionaries_wait_for_load/test.py +++ b/tests/integration/test_dictionaries_wait_for_load/test.py @@ -10,11 +10,14 @@ DICTIONARY_FILES = [ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", - main_configs=["configs/wait_for_dictionaries_load.xml"], + main_configs=["configs/no_dictionaries_lazy_load.xml"], dictionaries=DICTIONARY_FILES, ) -node0 = cluster.add_instance("node0", dictionaries=DICTIONARY_FILES) +node0 = cluster.add_instance( + "node0", + dictionaries=DICTIONARY_FILES, +) @pytest.fixture(scope="module", autouse=True) @@ -33,9 +36,13 @@ def get_status(instance, dictionary_name): def test_wait_for_dictionaries_load(): - assert get_status(node0, "long_loading_dictionary") == "NOT_LOADED" - assert get_status(node1, "long_loading_dictionary") == "LOADED" assert node1.query("SELECT * FROM dictionary(long_loading_dictionary)") == TSV( [[1, "aa"], [2, "bb"]] ) + + assert get_status(node0, "long_loading_dictionary") == "NOT_LOADED" + assert node0.query("SELECT * FROM dictionary(long_loading_dictionary)") == TSV( + [[1, "aa"], [2, "bb"]] + ) + assert get_status(node0, "long_loading_dictionary") == "LOADED" From 9b258f4b09df177f58e1f944bf3667a1e4b71072 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 22 Nov 2023 21:09:07 +0100 Subject: [PATCH 747/813] Do not symbolize traces for debug/sanitizer builds for sending to cloud debug/sanitizer builds is very slow and symbolizing can take awhile, for example this increases the time for `system flush logs`, and likely make `02152_http_external_tables_memory_tracking` flaky again (#53215): azat@s1:~/ch/tmp$ zstd -cdq clickhouse-server.log.zst | grep -a -e 2dd61ba3-5a26-4b38-8979-af82cf3ff8bd -e 75b3cbcb-1d09-44ac-a82b-317b4fabfea9 -e 75ad1065-51cc-4c94-95a2-f9dd22981edd -e dc991967-4443-458b-84f0-2646a8d32a76 | grep trace_log -A1 2023.11.22 05:46:32.872164 [ 59150 ] {2dd61ba3-5a26-4b38-8979-af82cf3ff8bd} SystemLogQueue (system.trace_log): Requested flush up to offset 308544 2023.11.22 05:47:23.352098 [ 59150 ] {2dd61ba3-5a26-4b38-8979-af82cf3ff8bd} SystemLogQueue (system.crash_log): Requested flush up to offset 0 -- 2023.11.22 05:47:46.158400 [ 59158 ] {75b3cbcb-1d09-44ac-a82b-317b4fabfea9} SystemLogQueue (system.trace_log): Requested flush up to offset 328185 2023.11.22 05:48:38.392275 [ 59158 ] {75b3cbcb-1d09-44ac-a82b-317b4fabfea9} SystemLogQueue (system.crash_log): Requested flush up to offset 0 -- 2023.11.22 05:49:07.348590 [ 59150 ] {75ad1065-51cc-4c94-95a2-f9dd22981edd} SystemLogQueue (system.trace_log): Requested flush up to offset 347744 2023.11.22 05:50:00.265529 [ 59150 ] {75ad1065-51cc-4c94-95a2-f9dd22981edd} SystemLogQueue (system.crash_log): Requested flush up to offset 0 -- 2023.11.22 05:50:25.743286 [ 59158 ] {dc991967-4443-458b-84f0-2646a8d32a76} SystemLogQueue (system.trace_log): Requested flush up to offset 367101 2023.11.22 05:51:15.567347 [ 59158 ] {dc991967-4443-458b-84f0-2646a8d32a76} SystemLogQueue (system.crash_log): Requested flush up to offset 0 Signed-off-by: Azat Khuzhin --- docker/test/base/setup_export_logs.sh | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index ec24b237752..dbb46c3ba43 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -126,6 +126,9 @@ function setup_logs_replication # It's doesn't make sense to try creating tables if SYNC fails echo "SYSTEM SYNC DATABASE REPLICA default" | clickhouse-client "${CONNECTION_ARGS[@]}" || return 0 + debug_or_sanitizer_build=$(clickhouse-client -q "WITH ((SELECT value FROM system.build_options WHERE name='BUILD_TYPE') AS build, (SELECT value FROM system.build_options WHERE name='CXX_FLAGS') as flags) SELECT build='Debug' OR flags LIKE '%fsanitize%'") + echo "Build is debug or sanitizer: $debug_or_sanitizer_build" + # For each system log table: echo 'Create %_log tables' clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table @@ -133,7 +136,14 @@ function setup_logs_replication if [[ "$table" = "trace_log" ]] then EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS_TRACE_LOG}" - EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_TRACE_LOG}" + # Do not try to resolve stack traces in case of debug/sanitizers + # build, since it is too slow (flushing of trace_log can take ~1min + # with such MV attached) + if [[ "$debug_or_sanitizer_build" = 1 ]]; then + EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}" + else + EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_TRACE_LOG}" + fi else EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS}" EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}" From 9c7b0f33fc80a0e2b25112ef42a7d69bd2035794 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 14:49:51 +0100 Subject: [PATCH 748/813] Fix bad test `00002_log_and_exception_messages_formatting` --- ...nd_exception_messages_formatting.reference | 2 +- ..._log_and_exception_messages_formatting.sql | 50 ++++++++++++------- 2 files changed, 34 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index 00a2cd14700..cd9f0142d45 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -3,7 +3,7 @@ runtime exceptions 0.05 unknown runtime exceptions 0.01 messages shorter than 10 1 messages shorter than 16 3 -exceptions shorter than 30 3 +exceptions shorter than 30 3 [] noisy messages 0.3 noisy Trace messages 0.16 noisy Debug messages 0.09 diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 63432f127aa..d1ac8773aca 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -13,11 +13,11 @@ select 'runtime messages', greatest(coalesce(sum(length(message_format_string) = where message not like '% Received from %clickhouse-staging.com:9440%'; -- Check the same for exceptions. The value was 0.03 -select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs +select 'runtime exceptions', greatest(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs where (message like '%DB::Exception%' or message like '%Coordination::Exception%') and message not like '% Received from %clickhouse-staging.com:9440%'; -select 'unknown runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.01) from logs where +select 'unknown runtime exceptions', greatest(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.01) from logs where (message like '%DB::Exception%' or message like '%Coordination::Exception%') and message not like '% Received from %' and message not like '%(SYNTAX_ERROR)%'; @@ -50,10 +50,15 @@ create temporary table known_short_messages (s String) as select * from (select ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. -select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_string), 1) from logs where length(message_format_string) < 10 and message_format_string not in known_short_messages; +select 'messages shorter than 10', + greatest(uniqExact(message_format_string), 1) + from logs + where length(message_format_string) < 10 and message_format_string not in known_short_messages; -- Same as above. Feel free to update the threshold or remove this query if really necessary -select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 3) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; +select 'messages shorter than 16', + greatest(uniqExact(message_format_string), 3) + from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; -- Unlike above, here we look at length of the formatted message, not format string. Most short format strings are fine because they end up decorated with context from outer or inner exceptions, e.g.: -- "Expected end of line" -> "Code: 117. DB::Exception: Expected end of line: (in file/uri /var/lib/clickhouse/user_files/data_02118): (at row 1)" @@ -62,42 +67,53 @@ select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_st -- This table currently doesn't have enough information to do this reliably, so we just regex search for " (ERROR_NAME_IN_CAPS)" and hope that's good enough. -- For the "Code: 123. DB::Exception: " part, we just subtract 26 instead of searching for it. Because sometimes it's not at the start, e.g.: -- "Unexpected error, will try to restart main thread: Code: 341. DB::Exception: Unexpected error: Code: 57. DB::Exception:[...]" -select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 3) from logs - where message ilike '%DB::Exception%' and if(length(regexpExtract(message, '(.*)\\([A-Z0-9_]+\\)')) as pref > 0, pref, length(message)) < 30 + 26 and message_format_string not in known_short_messages; +select 'exceptions shorter than 30', + greatest(uniqExact(message_format_string), 3) AS c, + c = 3 ? [] : groupUniqArray(message_format_string) + from logs + where message ilike '%DB::Exception%' and if(length(extract(message, '(.*)\\([A-Z0-9_]+\\)')) as pref > 0, pref, length(message)) < 30 + 26 and message_format_string not in known_short_messages; -- Avoid too noisy messages: top 1 message frequency must be less than 30%. We should reduce the threshold -select 'noisy messages', max2((select count() from logs group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.30); +select 'noisy messages', + greatest((select count() from logs group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.30); -- Same as above, but excluding Test level (actually finds top 1 Trace message) with ('Access granted: {}{}', '{} -> {}') as frequent_in_tests -select 'noisy Trace messages', max2((select count() from logs where level!='Test' and message_format_string not in frequent_in_tests - group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.16); +select 'noisy Trace messages', + greatest((select count() from logs where level!='Test' and message_format_string not in frequent_in_tests + group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.16); -- Same as above for Debug -select 'noisy Debug messages', max2((select count() from logs where level <= 'Debug' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.09); +select 'noisy Debug messages', + greatest((select count() from logs where level <= 'Debug' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.09); -- Same as above for Info -select 'noisy Info messages', max2((select count() from logs where level <= 'Information' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.05); +select 'noisy Info messages', + greatest((select count() from logs where level <= 'Information' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.05); -- Same as above for Warning with ('Not enabled four letter command {}') as frequent_in_tests -select 'noisy Warning messages', max2(coalesce((select countOrDefault() from logs where level = 'Warning' and message_format_string not in frequent_in_tests +select 'noisy Warning messages', + greatest(coalesce((select count() from logs where level = 'Warning' and message_format_string not in frequent_in_tests group by message_format_string order by count() desc limit 1), 0) / (select count() from logs), 0.01); -- Same as above for Error -select 'noisy Error messages', max2(coalesce((select countOrDefault() from logs where level = 'Error' group by message_format_string order by count() desc limit 1), 0) / (select count() from logs), 0.02); +select 'noisy Error messages', + greatest(coalesce((select count() from logs where level = 'Error' group by message_format_string order by count() desc limit 1), 0) / (select count() from logs), 0.02); select 'no Fatal messages', count() from logs where level = 'Fatal'; -- Avoid too noisy messages: limit the number of messages with high frequency -select 'number of too noisy messages', max2(count(), 3) from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.10); -select 'number of noisy messages', max2(count(), 10) from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.05); +select 'number of too noisy messages', + greatest(count(), 3) from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.10); +select 'number of noisy messages', + greatest(count(), 10) from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.05); -- Each message matches its pattern (returns 0 rows) --- FIXME maybe we should make it stricter ('Code:%Exception: '||s||'%'), but it's not easy because of addMessage -select 'incorrect patterns', max2(countDistinct(message_format_string), 15) from ( +-- Note: maybe we should make it stricter ('Code:%Exception: '||s||'%'), but it's not easy because of addMessage +select 'incorrect patterns', greatest(uniqExact(message_format_string), 15) from ( select message_format_string, any(message) as any_message from logs where ((rand() % 8) = 0) and message not like (replaceRegexpAll(message_format_string, '{[:.0-9dfx]*}', '%') as s) From eabdc76e7c648c84c46fff8a0dbf7fd1a82f4bba Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 23 Nov 2023 15:25:58 +0100 Subject: [PATCH 749/813] Fix test test_replicated_merge_tree_encryption_codec/test.py::test_different_keys --- .../test_replicated_merge_tree_encryption_codec/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_merge_tree_encryption_codec/test.py b/tests/integration/test_replicated_merge_tree_encryption_codec/test.py index d2dbc7c5466..39eb4ee02eb 100644 --- a/tests/integration/test_replicated_merge_tree_encryption_codec/test.py +++ b/tests/integration/test_replicated_merge_tree_encryption_codec/test.py @@ -91,7 +91,9 @@ def test_different_keys(): copy_keys(node2, "key_b") create_table() - insert_data() + # Insert two blocks without duplicated blocks to force each replica to actually fetch parts from another replica. + node1.query("INSERT INTO tbl VALUES (1, 'str1')") + node2.query("INSERT INTO tbl VALUES (2, 'str2')") node1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") assert "BAD_DECRYPT" in node1.query_and_get_error("SELECT * FROM tbl") From 956a8add35264e18bdaf8ce0039ad0d5a538f5b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tom=C3=A1=C5=A1=20Hromada?= Date: Thu, 23 Nov 2023 15:57:19 +0100 Subject: [PATCH 750/813] Update other-functions.md Fix `initializeAgregation` typo --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 35fd5089bf0..4c103274f43 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1556,7 +1556,7 @@ initializeAggregation (aggregate_function, arg1, arg2, ..., argN) - Result of aggregation for every row passed to the function. -The return type is the same as the return type of function, that `initializeAgregation` takes as first argument. +The return type is the same as the return type of function, that `initializeAggregation` takes as first argument. **Example** From d81fb9d1175e8dd9080500ac399bcfb93ab10111 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 23 Nov 2023 16:38:15 +0100 Subject: [PATCH 751/813] Remove partial results from build matrix for stress tests The feature had been reverted in #55893 Signed-off-by: Azat Khuzhin --- tests/ci/stress.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index ae918363df7..14b958e5d7a 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -59,12 +59,6 @@ def get_options(i: int, upgrade_check: bool) -> str: client_options.append("implicit_transaction=1") client_options.append("throw_on_unsupported_query_inside_transaction=0") - if random.random() < 0.1: - client_options.append("allow_experimental_partial_result=1") - client_options.append( - f"partial_result_update_duration_ms={random.randint(10, 1000)}" - ) - if random.random() < 0.1: client_options.append("optimize_trivial_approximate_count_query=1") From 5abed8f8a8091633069b4c1f6883100fd8b41fd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 17:00:02 +0100 Subject: [PATCH 752/813] Better exit codes from the server --- programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index efb5ccb3203..53e2db91f27 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1970,7 +1970,7 @@ catch (...) { /// Poco does not provide stacktrace. tryLogCurrentException("Application"); - throw; + return getCurrentExceptionCode(); } std::unique_ptr Server::buildProtocolStackFromConfig( From bd4ffec361234da945136cd33039e756085dc8ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 17:00:57 +0100 Subject: [PATCH 753/813] Better exit codes from the server --- programs/keeper/Keeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 6df1bbaa329..8a4ec646ce1 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -556,7 +556,7 @@ catch (...) { /// Poco does not provide stacktrace. tryLogCurrentException("Application"); - throw; + return getCurrentExceptionCode(); } From d59b322619e985a314b3afa2d9441c9f5ae1cda5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 17:18:11 +0100 Subject: [PATCH 754/813] Add a test --- .../0_stateless/02922_server_exit_code.reference | 1 + tests/queries/0_stateless/02922_server_exit_code.sh | 12 ++++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/02922_server_exit_code.reference create mode 100755 tests/queries/0_stateless/02922_server_exit_code.sh diff --git a/tests/queries/0_stateless/02922_server_exit_code.reference b/tests/queries/0_stateless/02922_server_exit_code.reference new file mode 100644 index 00000000000..7326d960397 --- /dev/null +++ b/tests/queries/0_stateless/02922_server_exit_code.reference @@ -0,0 +1 @@ +Ok diff --git a/tests/queries/0_stateless/02922_server_exit_code.sh b/tests/queries/0_stateless/02922_server_exit_code.sh new file mode 100755 index 00000000000..60049902410 --- /dev/null +++ b/tests/queries/0_stateless/02922_server_exit_code.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# 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. + +${CLICKHOUSE_SERVER_BINARY} -- --path /dev/null 2>/dev/null; [[ "$?" == "$((400 % 256))" ]] && echo 'Ok' || echo 'Fail' From 64d6fe352010fc0dc23ab5c22a493bd2379943a2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 23 Nov 2023 16:48:40 +0000 Subject: [PATCH 755/813] Mark select() as harmful function --- base/harmful/harmful.c | 1 + 1 file changed, 1 insertion(+) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 78796ca0c05..74fcf769683 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -145,6 +145,7 @@ TRAP(qecvt) TRAP(qfcvt) TRAP(register_printf_function) TRAP(seed48) +TRAP(select) //TRAP(setenv) TRAP(setfsent) TRAP(setgrent) From 137bf2cf9c9d8e192a18a05aba22fe240665d639 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 17:51:55 +0100 Subject: [PATCH 756/813] Whitespaces --- programs/server/config.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 7003111c193..10db7444316 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -830,13 +830,13 @@ And also (and which is more important), the initial_user will be used as current user for the query. - Right now the protocol is pretty simple and it only takes into account: + Right now the protocol is pretty simple, and it only takes into account: - cluster name - query - Also it will be nice if the following will be implemented: - - source hostname (see interserver_http_host), but then it will depends from DNS, - it can use IP address instead, but then the you need to get correct on the initiator node. + Also, it will be nice if the following will be implemented: + - source hostname (see interserver_http_host), but then it will depend on DNS, + it can use IP address instead, but then you need to get correct on the initiator node. - target hostname / ip address (same notes as for source hostname) - time-based security tokens --> From b62e9d3a1a96603f110700eda87fb7c8f025a3c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 18:13:12 +0100 Subject: [PATCH 757/813] Address review comments --- programs/client/Client.cpp | 3 +-- programs/keeper/Keeper.cpp | 3 ++- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 3 ++- src/Client/ClientBase.cpp | 2 +- src/Client/ClientBase.h | 2 +- 6 files changed, 8 insertions(+), 7 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 7f48a9987c7..d2527ad0c98 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -325,7 +325,7 @@ try processConfig(); adjustSettings(); - initTtyBuffer(toProgressOption(config().getString("progress", "default"))); + initTTYBuffer(toProgressOption(config().getString("progress", "default"))); { // All that just to set DB::CurrentThread::get().getGlobalContext() @@ -1459,7 +1459,6 @@ int mainEntryClickHouseClient(int argc, char ** argv) DB::Client client; // Initialize command line options client.init(argc, argv); - /// Initialize config file return client.run(); } catch (const DB::Exception & e) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 8a4ec646ce1..e04e669abae 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -556,7 +556,8 @@ catch (...) { /// Poco does not provide stacktrace. tryLogCurrentException("Application"); - return getCurrentExceptionCode(); + auto code = getCurrentExceptionCode(); + return code ? code : -1; } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 80ab173c150..f3b84fa3eb1 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -495,7 +495,7 @@ try processConfig(); adjustSettings(); - initTtyBuffer(toProgressOption(config().getString("progress", "default"))); + initTTYBuffer(toProgressOption(config().getString("progress", "default"))); applyCmdSettings(global_context); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 53e2db91f27..eeaceb8efe5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1970,7 +1970,8 @@ catch (...) { /// Poco does not provide stacktrace. tryLogCurrentException("Application"); - return getCurrentExceptionCode(); + auto code = getCurrentExceptionCode(); + return code ? code : -1; } std::unique_ptr Server::buildProtocolStackFromConfig( diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index bb10b104d3e..b513e623829 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -722,7 +722,7 @@ void ClientBase::adjustSettings() global_context->setSettings(settings); } -void ClientBase::initTtyBuffer(ProgressOption progress) +void ClientBase::initTTYBuffer(ProgressOption progress) { if (tty_buf) return; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 9fde23cf775..4eed8e0ace3 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -185,7 +185,7 @@ protected: /// Adjust some settings after command line options and config had been processed. void adjustSettings(); - void initTtyBuffer(ProgressOption progress); + void initTTYBuffer(ProgressOption progress); /// Should be one of the first, to be destroyed the last, /// since other members can use them. From 673dec6b608f4d8be995cf0b614b372852e62e75 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 23 Nov 2023 17:30:59 +0000 Subject: [PATCH 758/813] Add pselect() to harmful functions as well --- base/harmful/harmful.c | 1 + 1 file changed, 1 insertion(+) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 74fcf769683..6526b7e577e 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -146,6 +146,7 @@ TRAP(qfcvt) TRAP(register_printf_function) TRAP(seed48) TRAP(select) +TRAP(pselect) //TRAP(setenv) TRAP(setfsent) TRAP(setgrent) From 77f27cf10ff17a479c27adf1c1fa6757437ee708 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 21:02:31 +0300 Subject: [PATCH 759/813] Revert "Resubmit 01600_parts_types_metrics test (possibly without flakiness)" --- .../01600_parts_types_metrics.reference | 3 -- .../0_stateless/01600_parts_types_metrics.sh | 49 ------------------- 2 files changed, 52 deletions(-) delete mode 100644 tests/queries/0_stateless/01600_parts_types_metrics.reference delete mode 100755 tests/queries/0_stateless/01600_parts_types_metrics.sh diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.reference b/tests/queries/0_stateless/01600_parts_types_metrics.reference deleted file mode 100644 index e8183f05f5d..00000000000 --- a/tests/queries/0_stateless/01600_parts_types_metrics.reference +++ /dev/null @@ -1,3 +0,0 @@ -1 -1 -1 diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.sh b/tests/queries/0_stateless/01600_parts_types_metrics.sh deleted file mode 100755 index f0914b53748..00000000000 --- a/tests/queries/0_stateless/01600_parts_types_metrics.sh +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-s3-storage, no-asan, long, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -set -e -set -o pipefail - -# NOTE: database = $CLICKHOUSE_DATABASE is unwanted -verify_sql="SELECT - (SELECT sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics) = - (SELECT countIf(part_type = 'Compact'), countIf(part_type = 'Wide') - FROM (SELECT part_type FROM system.parts UNION ALL SELECT part_type FROM system.projection_parts))" - -# The query is not atomic - it can compare states between system.parts and system.metrics from different points in time. -# So, there is inherent race condition (especially in fasttest that runs tests in parallel). -# -# But it should get the expected result eventually. -# In case of test failure, this code will do infinite loop and timeout. -verify() -{ - for ((i = 0; i < 100; ++i)); do - result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" ) - if [ "$result" = "1" ]; then - echo 1 - return - fi - sleep 0.1 - done -} - -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE IF EXISTS data_01600" -# Compact - (5..10] -# Wide - >10 -$CLICKHOUSE_CLIENT --query="CREATE TABLE data_01600 (part_type String, key Int) ENGINE = MergeTree PARTITION BY part_type ORDER BY key SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=10, index_granularity = 8192, index_granularity_bytes = '10Mi'" - -# Compact -$CLICKHOUSE_CLIENT --query="INSERT INTO data_01600 SELECT 'Compact', number FROM system.numbers LIMIT 6" -verify - -# Wide -$CLICKHOUSE_CLIENT --query="INSERT INTO data_01600 SELECT 'Wide', number FROM system.numbers LIMIT 11 OFFSET 6" -verify - -# DROP and check -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE data_01600" -verify From 36d27d171360471f3bf3fd5b1fb0ef8ac4989216 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Nov 2023 19:28:58 +0100 Subject: [PATCH 760/813] Improve the cherry-pick PR description --- tests/ci/cherry_pick.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 9ee63a98f94..e36c31b2089 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -67,15 +67,17 @@ The check results does not matter at this step - you can safely ignore them. ### Note This pull-request will be merged automatically as it reaches the mergeable state, \ -**do not merge it manually**. +**do not merge it manually**. It's 100% safe, but completely meaningless. ### If the PR was closed and then reopened -If it stuck, check {pr_url} for `{backport_created_label}` and delete it if \ -necessary. Manually merging will do nothing, since `{backport_created_label}` \ -prevents the original PR {pr_url} from being processed. +If it stuck (e.g. for a day), check {pr_url} for `{backport_created_label}` *label* and \ +delete it if necessary. Manually merging will do nothing, since \ +`{backport_created_label}` *label* prevents the original PR {pr_url} from being \ +processed. -If you want to recreate the PR: delete the `{label_cherrypick}` label and delete this branch. +If the cherry-pick PR is completely screwed, and you want to recreate it: delete the \ +`{label_cherrypick}` label and delete this branch. You may also need to delete the `{backport_created_label}` label from the original PR. """ BACKPORT_DESCRIPTION = """This pull-request is a last step of an automated \ From aee3c9651b67562889d703a44f8b3acd886db99a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Nov 2023 18:18:57 +0000 Subject: [PATCH 761/813] Fix ubsan bug --- src/Functions/fromDaysSinceYearZero.cpp | 4 +++- .../queries/0_stateless/02907_fromDaysSinceYearZero.reference | 3 +++ tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql | 4 ++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Functions/fromDaysSinceYearZero.cpp b/src/Functions/fromDaysSinceYearZero.cpp index 628949570a0..a21d0cc25bf 100644 --- a/src/Functions/fromDaysSinceYearZero.cpp +++ b/src/Functions/fromDaysSinceYearZero.cpp @@ -97,7 +97,9 @@ public: auto value = src_data[i]; if (value < 0) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Expected a non-negative integer, got: {}", std::to_string(value)); - dst_data[i] = static_cast(value) - static_cast(ToDaysSinceYearZeroImpl::DAYS_BETWEEN_YEARS_0_AND_1970); + /// prevent potential signed integer overflows (aka. undefined behavior) with Date32 results + auto value_uint64 = static_cast(value); /// NOLINT(bugprone-signed-char-misuse,cert-str34-c) + dst_data[i] = static_cast(value_uint64 - ToDaysSinceYearZeroImpl::DAYS_BETWEEN_YEARS_0_AND_1970); } } }; diff --git a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference index 6f62c3f8403..999505f54c9 100644 --- a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference +++ b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference @@ -18,5 +18,8 @@ 719529 719529 1970-01-02 1970-01-02 1970-01-02 1970-01-02 -- NULL handling \N \N +-- ubsan bug +2299-12-31 +2299-12-31 -- Alias 1973-10-01 diff --git a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql index 44fd498fce8..9f356080fe8 100644 --- a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql +++ b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql @@ -33,5 +33,9 @@ SELECT toUInt64(719529) AS u, toInt64(719529) AS s, fromDaysSinceYearZero(u), fr SELECT '-- NULL handling'; SELECT fromDaysSinceYearZero(NULL), fromDaysSinceYearZero32(NULL); +SELECT '-- ubsan bugs'; +SELECT fromDaysSinceYearZero32(2147483648); +SELECT fromDaysSinceYearZero32(3); + SELECT '-- Alias'; SELECT FROM_DAYS(1); From 77d48e4d55ee5d001d597c653c15aaf199aeb560 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Nov 2023 14:51:49 +0000 Subject: [PATCH 762/813] "optimise" --> "optimize". Sorry, friends from the UK. --- src/Core/Protocol.h | 2 +- src/Core/Settings.h | 4 ++-- src/Functions/greatCircleDistance.cpp | 2 +- src/Functions/randDistribution.cpp | 2 +- src/Interpreters/TreeRewriter.cpp | 2 +- src/Processors/QueryPlan/AggregatingStep.h | 2 +- src/Processors/QueryPlan/MergingAggregatedStep.cpp | 2 +- src/Processors/QueryPlan/Optimizations/Optimizations.h | 2 +- .../QueryPlan/Optimizations/QueryPlanOptimizationSettings.h | 4 ++-- .../QueryPlan/Optimizations/optimizeReadInOrder.cpp | 2 +- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h | 2 +- 13 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index cc642a7f501..441e22f4a16 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -97,7 +97,7 @@ namespace Protocol }; /// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10 - /// would always be true because of compiler optimisation. That would lead to out-of-bounds error + /// would always be true because of compiler optimization. That would lead to out-of-bounds error /// if the packet is invalid. /// See https://www.securecoding.cert.org/confluence/display/cplusplus/INT36-CPP.+Do+not+use+out-of-range+enumeration+values inline const char * toString(UInt64 packet) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6dac828aafa..af536cc2846 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -687,8 +687,8 @@ class IColumn; M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \ M(Bool, query_plan_execute_functions_after_sorting, true, "Allow to re-order functions after sorting", 0) \ M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \ - M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimisation", 0) \ - M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimisation", 0) \ + M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimization", 0) \ + M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimization", 0) \ M(Bool, query_plan_remove_redundant_sorting, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries", 0) \ M(Bool, query_plan_remove_redundant_distinct, true, "Remove redundant Distinct step in query plan", 0) \ M(Bool, query_plan_enable_multithreading_after_window_functions, true, "Enable multithreading after evaluating window functions to allow parallel stream processing", 0) \ diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index db1f777fa91..d1d1a101187 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -35,7 +35,7 @@ namespace ErrorCodes * https://github.com/sphinxsearch/sphinx/blob/409f2c2b5b2ff70b04e38f92b6b1a890326bad65/src/sphinxexpr.cpp#L3825. * Andrey Aksenov, the author of original code, permitted to use this code in ClickHouse under the Apache 2.0 license. * Presentation about this code from Highload++ Siberia 2019 is here https://github.com/ClickHouse/ClickHouse/files/3324740/1_._._GEODIST_._.pdf - * The main idea of this implementation is optimisations based on Taylor series, trigonometric identity + * The main idea of this implementation is optimizations based on Taylor series, trigonometric identity * and calculated constants once for cosine, arcsine(sqrt) and look up table. */ diff --git a/src/Functions/randDistribution.cpp b/src/Functions/randDistribution.cpp index 725ae0b4e64..db101486de8 100644 --- a/src/Functions/randDistribution.cpp +++ b/src/Functions/randDistribution.cpp @@ -196,7 +196,7 @@ struct PoissonDistribution * Accepts only constant arguments * Similar to the functions rand and rand64 an additional 'tag' argument could be added to the * end of arguments list (this argument will be ignored) which will guarantee that functions are not sticked together - * during optimisations. + * during optimizations. * Example: SELECT randNormal(0, 1, 1), randNormal(0, 1, 2) FROM numbers(10) * This query will return two different columns */ diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 6172ef06b9a..c3cc1c8edaf 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -992,7 +992,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (required.contains(name)) { - /// Optimisation: do not add columns needed only in JOIN ON section. + /// Optimization: do not add columns needed only in JOIN ON section. if (columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name)) analyzed_join->addJoinedColumn(joined_column); diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 3d128d788ac..f446ecec846 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -91,7 +91,7 @@ private: bool storage_has_evenly_distributed_read; bool group_by_use_nulls; - /// Both sort descriptions are needed for aggregate-in-order optimisation. + /// Both sort descriptions are needed for aggregate-in-order optimization. /// Both sort descriptions are subset of GROUP BY key columns (or monotonic functions over it). /// Sort description for merging is a sort description for input and a prefix of group_by_sort_description. /// group_by_sort_description contains all GROUP BY keys and is used for final merging of aggregated data. diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index c724de02de8..476f5541812 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -77,7 +77,7 @@ void MergingAggregatedStep::applyOrder(SortDescription sort_description, DataStr input_stream.sort_scope = sort_scope; input_stream.sort_description = sort_description; - /// Columns might be reordered during optimisation, so we better to update sort description. + /// Columns might be reordered during optimization, so we better to update sort description. group_by_sort_description = std::move(sort_description); if (memoryBoundMergingWillBeUsed() && should_produce_results_in_order_of_bucket_number) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 5bbf68a5259..1316b7895dc 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -67,7 +67,7 @@ void tryRemoveRedundantSorting(QueryPlan::Node * root); /// Remove redundant distinct steps size_t tryRemoveRedundantDistinct(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); -/// Put some steps under union, so that plan optimisation could be applied to union parts separately. +/// Put some steps under union, so that plan optimization could be applied to union parts separately. /// For example, the plan can be rewritten like: /// - Something - - Expression - Something - /// - Expression - Union - Something - => - Union - Expression - Something - diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 0bf53d1ec4b..2b5d3bb92ca 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -27,10 +27,10 @@ struct QueryPlanOptimizationSettings /// if distinct in order optimization is enabled bool distinct_in_order = false; - /// If read-in-order optimisation is enabled + /// If read-in-order optimization is enabled bool read_in_order = true; - /// If aggregation-in-order optimisation is enabled + /// If aggregation-in-order optimization is enabled bool aggregation_in_order = false; /// If removing redundant sorting is enabled, for example, ORDER BY clauses in subqueries diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 6352edddf71..433422a7c30 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1004,7 +1004,7 @@ void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &) } } -/// This optimisation is obsolete and will be removed. +/// This optimization is obsolete and will be removed. /// optimizeReadInOrder covers it. size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & /*nodes*/) { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 617de8c8530..d3ff5221e1f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1824,7 +1824,7 @@ Pipe ReadFromMergeTree::spreadMarkRanges( chassert(!is_parallel_reading_from_replicas); if (output_each_partition_through_separate_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Optimisation isn't supposed to be used for queries with final"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Optimization isn't supposed to be used for queries with final"); /// Add columns needed to calculate the sorting expression and the sign. for (const auto & column : metadata_for_reading->getColumnsRequiredForSortingKey()) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 35310e14416..4e38e06c6af 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -190,7 +190,7 @@ public: bool isQueryWithFinal() const; bool isQueryWithSampling() const; - /// Returns true if the optimisation is applicable (and applies it then). + /// Returns true if the optimization is applicable (and applies it then). bool requestOutputEachPartitionThroughSeparatePort(); bool willOutputEachPartitionThroughSeparatePort() const { return output_each_partition_through_separate_port; } @@ -255,7 +255,7 @@ private: size_t output_streams_limit = 0; const bool sample_factor_column_queried; - /// Used for aggregation optimisation (see DB::QueryPlanOptimizations::tryAggregateEachPartitionIndependently). + /// Used for aggregation optimization (see DB::QueryPlanOptimizations::tryAggregateEachPartitionIndependently). bool output_each_partition_through_separate_port = false; std::shared_ptr max_block_numbers_to_read; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 2a13b7a02c0..0ce59b18818 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -35,7 +35,7 @@ struct ReplicatedMergeTreeLogEntryData EMPTY, /// Not used. GET_PART, /// Get the part from another replica. ATTACH_PART, /// Attach the part, possibly from our own replica (if found in /detached folder). - /// You may think of it as a GET_PART with some optimisations as they're nearly identical. + /// You may think of it as a GET_PART with some optimizations as they're nearly identical. MERGE_PARTS, /// Merge the parts. DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. From 4b764b8bc9b56bae16263c8220c14c64491133ca Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Nov 2023 14:53:28 +0000 Subject: [PATCH 763/813] Cosmetics --- .../QueryPlan/Optimizations/Optimizations.h | 8 ++------ .../Optimizations/QueryPlanOptimizationSettings.cpp | 11 +++++++++++ .../Optimizations/QueryPlanOptimizationSettings.h | 6 +++--- 3 files changed, 16 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 1316b7895dc..1edde796061 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -85,13 +85,9 @@ inline const auto & getOptimizations() {tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::optimize_plan}, {tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down}, {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::execute_functions_after_sorting}, - {tryReuseStorageOrderingForWindowFunctions, - "reuseStorageOrderingForWindowFunctions", - &QueryPlanOptimizationSettings::optimize_plan}, + {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan}, {tryLiftUpUnion, "liftUpUnion", &QueryPlanOptimizationSettings::optimize_plan}, - {tryAggregatePartitionsIndependently, - "aggregatePartitionsIndependently", - &QueryPlanOptimizationSettings::aggregate_partitions_independently}, + {tryAggregatePartitionsIndependently, "aggregatePartitionsIndependently", &QueryPlanOptimizationSettings::aggregate_partitions_independently}, {tryRemoveRedundantDistinct, "removeRedundantDistinct", &QueryPlanOptimizationSettings::remove_redundant_distinct}, }}; diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 9ab789e1192..4d515cb7d86 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -8,20 +8,31 @@ namespace DB QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const Settings & from) { QueryPlanOptimizationSettings settings; + settings.optimize_plan = from.query_plan_enable_optimizations; settings.max_optimizations_to_apply = from.query_plan_max_optimizations_to_apply; + settings.filter_push_down = from.query_plan_filter_push_down; + settings.execute_functions_after_sorting = from.query_plan_execute_functions_after_sorting; + settings.distinct_in_order = from.optimize_distinct_in_order; + settings.read_in_order = from.optimize_read_in_order && from.query_plan_read_in_order; + settings.aggregation_in_order = from.optimize_aggregation_in_order && from.query_plan_aggregation_in_order; + settings.remove_redundant_sorting = from.query_plan_remove_redundant_sorting; + settings.aggregate_partitions_independently = from.allow_aggregate_partitions_independently; + settings.remove_redundant_distinct = from.query_plan_remove_redundant_distinct; + settings.optimize_projection = from.optimize_use_projections; settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection; settings.force_projection_name = from.force_optimize_projection_name; settings.optimize_use_implicit_projections = settings.optimize_projection && from.optimize_use_implicit_projections; + return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 2b5d3bb92ca..496e3c86f72 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -11,13 +11,13 @@ struct Settings; struct QueryPlanOptimizationSettings { + /// If disabled, no optimization applied. + bool optimize_plan = true; + /// If not zero, throw if too many optimizations were applied to query plan. /// It helps to avoid infinite optimization loop. size_t max_optimizations_to_apply = 0; - /// If disabled, no optimization applied. - bool optimize_plan = true; - /// If filter push down optimization is enabled. bool filter_push_down = true; From d750e3c004d077632c9c8f0206ccac4fba2c7e79 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Nov 2023 19:44:14 +0100 Subject: [PATCH 764/813] fix --- ...eference => 02052_last_granula_adjust_logical_error.reference} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02052_last_granula_adjust_LOGICAL_ERROR.reference => 02052_last_granula_adjust_logical_error.reference} (100%) diff --git a/tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.reference b/tests/queries/0_stateless/02052_last_granula_adjust_logical_error.reference similarity index 100% rename from tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.reference rename to tests/queries/0_stateless/02052_last_granula_adjust_logical_error.reference From 194e676c70db4f109918c0bc503195289f1977ed Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Nov 2023 15:15:17 +0000 Subject: [PATCH 765/813] Put all query plan settings under their own settings --- src/Core/Settings.h | 6 +++++ .../QueryPlan/Optimizations/Optimizations.h | 12 +++++----- .../QueryPlanOptimizationSettings.cpp | 12 ++++++++++ .../QueryPlanOptimizationSettings.h | 23 +++++++++++++++++-- .../QueryPlan/Optimizations/optimizeTree.cpp | 4 ++-- 5 files changed, 47 insertions(+), 10 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index af536cc2846..8c2c24cde8f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -684,8 +684,14 @@ class IColumn; \ M(Bool, query_plan_enable_optimizations, true, "Apply optimizations to query plan", 0) \ M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \ + M(Bool, query_plan_lift_up_array_join, true, "Allow to move array joins up in the query plan", 0) \ + M(Bool, query_plan_push_down_limit, true, "Allow to move LIMITs down in the query plan", 0) \ + M(Bool, query_plan_split_filter, true, "Allow to split filters in the query plan", 0) \ + M(Bool, query_plan_merge_expressions, true, "Allow to merge expressions in the query plan", 0) \ M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \ M(Bool, query_plan_execute_functions_after_sorting, true, "Allow to re-order functions after sorting", 0) \ + M(Bool, query_plan_reuse_storage_ordering_for_window_functions, true, "Allow to use the storage sorting for window functions", 0) \ + M(Bool, query_plan_lift_up_union, true, "Allow to move UNIONs up so that more parts of the query plan can be optimized", 0) \ M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \ M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimization", 0) \ M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimization", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 1edde796061..a30e9b63da4 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -79,14 +79,14 @@ size_t tryAggregatePartitionsIndependently(QueryPlan::Node * node, QueryPlan::No inline const auto & getOptimizations() { static const std::array optimizations = {{ - {tryLiftUpArrayJoin, "liftUpArrayJoin", &QueryPlanOptimizationSettings::optimize_plan}, - {tryPushDownLimit, "pushDownLimit", &QueryPlanOptimizationSettings::optimize_plan}, - {trySplitFilter, "splitFilter", &QueryPlanOptimizationSettings::optimize_plan}, - {tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::optimize_plan}, + {tryLiftUpArrayJoin, "liftUpArrayJoin", &QueryPlanOptimizationSettings::lift_up_array_join}, + {tryPushDownLimit, "pushDownLimit", &QueryPlanOptimizationSettings::push_down_limit}, + {trySplitFilter, "splitFilter", &QueryPlanOptimizationSettings::split_filter}, + {tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::merge_expressions}, {tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down}, {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::execute_functions_after_sorting}, - {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan}, - {tryLiftUpUnion, "liftUpUnion", &QueryPlanOptimizationSettings::optimize_plan}, + {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::reuse_storage_ordering_for_window_functions}, + {tryLiftUpUnion, "liftUpUnion", &QueryPlanOptimizationSettings::lift_up_union}, {tryAggregatePartitionsIndependently, "aggregatePartitionsIndependently", &QueryPlanOptimizationSettings::aggregate_partitions_independently}, {tryRemoveRedundantDistinct, "removeRedundantDistinct", &QueryPlanOptimizationSettings::remove_redundant_distinct}, }}; diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 4d515cb7d86..e188cdb45ce 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -12,10 +12,22 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.optimize_plan = from.query_plan_enable_optimizations; settings.max_optimizations_to_apply = from.query_plan_max_optimizations_to_apply; + settings.lift_up_array_join = from.query_plan_lift_up_array_join; + + settings.push_down_limit = from.query_plan_push_down_limit; + + settings.split_filter = from.query_plan_split_filter; + + settings.merge_expressions = from.query_plan_merge_expressions; + settings.filter_push_down = from.query_plan_filter_push_down; settings.execute_functions_after_sorting = from.query_plan_execute_functions_after_sorting; + settings.reuse_storage_ordering_for_window_functions = from.query_plan_reuse_storage_ordering_for_window_functions; + + settings.lift_up_union = from.query_plan_lift_up_union; + settings.distinct_in_order = from.optimize_distinct_in_order; settings.read_in_order = from.optimize_read_in_order && from.query_plan_read_in_order; diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 496e3c86f72..e1ff2f74701 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -18,11 +18,29 @@ struct QueryPlanOptimizationSettings /// It helps to avoid infinite optimization loop. size_t max_optimizations_to_apply = 0; + /// If moving-up-of-array-join optimization is enabled. + bool lift_up_array_join = true; + + /// If moving-limit-down optimization is enabled. + bool push_down_limit = true; + + /// If split-filter optimization is enabled. + bool split_filter = true; + + /// If merge-expressions optimization is enabled. + bool merge_expressions = true; + /// If filter push down optimization is enabled. bool filter_push_down = true; /// If reorder-functions-after-sorting optimization is enabled. - bool execute_functions_after_sorting; + bool execute_functions_after_sorting = true; + + /// If window-functions-can-use-storage-sorting optimization is enabled. + bool reuse_storage_ordering_for_window_functions = true; + + /// If lifting-unions-up optimization is enabled. + bool lift_up_union = true; /// if distinct in order optimization is enabled bool distinct_in_order = false; @@ -36,9 +54,10 @@ struct QueryPlanOptimizationSettings /// If removing redundant sorting is enabled, for example, ORDER BY clauses in subqueries bool remove_redundant_sorting = true; + /// If aggregate-partitions-independently optimization is enabled. bool aggregate_partitions_independently = false; - /// If removing redundant distinct steps is enabled + /// If remove-redundant-distinct-steps optimization is enabled. bool remove_redundant_distinct = true; /// If reading from projection can be applied diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 0caedff67a5..c8c95e7443f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -41,7 +41,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query std::stack stack; stack.push({.node = &root}); - size_t max_optimizations_to_apply = settings.max_optimizations_to_apply; + const size_t max_optimizations_to_apply = settings.max_optimizations_to_apply; size_t total_applied_optimizations = 0; while (!stack.empty()) @@ -105,7 +105,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { - size_t max_optimizations_to_apply = optimization_settings.max_optimizations_to_apply; + const size_t max_optimizations_to_apply = optimization_settings.max_optimizations_to_apply; size_t num_applied_projection = 0; bool has_reading_from_mt = false; From e392fcab5b36960d4e0ce47fdc0f60ace4abdd36 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Nov 2023 15:42:22 +0000 Subject: [PATCH 766/813] Allow global and individual disabling of settings - 'query_plan_enable_optimizations' is now a global kill switch for all plan-level optimizations - even if it is true, individual optimizations can still be disabled individually via their respective settings --- src/Core/Settings.h | 2 +- .../QueryPlanOptimizationSettings.cpp | 28 +++++++++---------- .../QueryPlanOptimizationSettings.h | 3 +- 3 files changed, 17 insertions(+), 16 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8c2c24cde8f..2af0ffd28e1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -682,7 +682,7 @@ class IColumn; M(Bool, optimize_group_by_constant_keys, true, "Optimize GROUP BY when all keys in block are constant", 0) \ M(Bool, legacy_column_name_of_tuple_literal, false, "List all names of element of large tuple literals in their column names instead of hash. This settings exists only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher.", 0) \ \ - M(Bool, query_plan_enable_optimizations, true, "Apply optimizations to query plan", 0) \ + M(Bool, query_plan_enable_optimizations, true, "Globally enable/disable query optimization at the query plan level", 0) \ M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \ M(Bool, query_plan_lift_up_array_join, true, "Allow to move array joins up in the query plan", 0) \ M(Bool, query_plan_push_down_limit, true, "Allow to move LIMITs down in the query plan", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index e188cdb45ce..55b8cb5ed53 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -12,33 +12,33 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.optimize_plan = from.query_plan_enable_optimizations; settings.max_optimizations_to_apply = from.query_plan_max_optimizations_to_apply; - settings.lift_up_array_join = from.query_plan_lift_up_array_join; + settings.lift_up_array_join = from.query_plan_enable_optimizations && from.query_plan_lift_up_array_join; - settings.push_down_limit = from.query_plan_push_down_limit; + settings.push_down_limit = from.query_plan_enable_optimizations && from.query_plan_push_down_limit; - settings.split_filter = from.query_plan_split_filter; + settings.split_filter = from.query_plan_enable_optimizations && from.query_plan_split_filter; - settings.merge_expressions = from.query_plan_merge_expressions; + settings.merge_expressions = from.query_plan_enable_optimizations && from.query_plan_merge_expressions; - settings.filter_push_down = from.query_plan_filter_push_down; + settings.filter_push_down = from.query_plan_enable_optimizations && from.query_plan_filter_push_down; - settings.execute_functions_after_sorting = from.query_plan_execute_functions_after_sorting; + settings.execute_functions_after_sorting = from.query_plan_enable_optimizations && from.query_plan_execute_functions_after_sorting; - settings.reuse_storage_ordering_for_window_functions = from.query_plan_reuse_storage_ordering_for_window_functions; + settings.reuse_storage_ordering_for_window_functions = from.query_plan_enable_optimizations && from.query_plan_reuse_storage_ordering_for_window_functions; - settings.lift_up_union = from.query_plan_lift_up_union; + settings.lift_up_union = from.query_plan_enable_optimizations && from.query_plan_lift_up_union; - settings.distinct_in_order = from.optimize_distinct_in_order; + settings.distinct_in_order = from.query_plan_enable_optimizations && from.optimize_distinct_in_order; - settings.read_in_order = from.optimize_read_in_order && from.query_plan_read_in_order; + settings.read_in_order = from.query_plan_enable_optimizations && from.optimize_read_in_order && from.query_plan_read_in_order; - settings.aggregation_in_order = from.optimize_aggregation_in_order && from.query_plan_aggregation_in_order; + settings.aggregation_in_order = from.query_plan_enable_optimizations && from.optimize_aggregation_in_order && from.query_plan_aggregation_in_order; - settings.remove_redundant_sorting = from.query_plan_remove_redundant_sorting; + settings.remove_redundant_sorting = from.query_plan_enable_optimizations && from.query_plan_remove_redundant_sorting; - settings.aggregate_partitions_independently = from.allow_aggregate_partitions_independently; + settings.aggregate_partitions_independently = from.query_plan_enable_optimizations && from.allow_aggregate_partitions_independently; - settings.remove_redundant_distinct = from.query_plan_remove_redundant_distinct; + settings.remove_redundant_distinct = from.query_plan_enable_optimizations && from.query_plan_remove_redundant_distinct; settings.optimize_projection = from.optimize_use_projections; settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection; diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index e1ff2f74701..791b8e57475 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -11,7 +11,8 @@ struct Settings; struct QueryPlanOptimizationSettings { - /// If disabled, no optimization applied. + /// Allows to globally disable all plan-level optimizations. + /// Note: Even if '= true', individual optimizations may still be disabled via below settings. bool optimize_plan = true; /// If not zero, throw if too many optimizations were applied to query plan. From 682ffccccc40bd1946b296f2397affb63a703d7d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 23 Nov 2023 19:55:55 +0100 Subject: [PATCH 767/813] Add debugging info for 01600_parts_types_metrics on failures It looks like sometimes counters do not match [1], and it does not looks like a timing issue, since I don't see any new parts according to the server log. [1]: https://s3.amazonaws.com/clickhouse-test-reports/57074/05784b66c969e195c4513d131de95bf4828183ce/stateless_tests__release_.html Signed-off-by: Azat Khuzhin --- .../01600_parts_types_metrics.reference | 3 ++ .../0_stateless/01600_parts_types_metrics.sh | 53 +++++++++++++++++++ 2 files changed, 56 insertions(+) create mode 100644 tests/queries/0_stateless/01600_parts_types_metrics.reference create mode 100755 tests/queries/0_stateless/01600_parts_types_metrics.sh diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.reference b/tests/queries/0_stateless/01600_parts_types_metrics.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/01600_parts_types_metrics.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.sh b/tests/queries/0_stateless/01600_parts_types_metrics.sh new file mode 100755 index 00000000000..d4ffbfed613 --- /dev/null +++ b/tests/queries/0_stateless/01600_parts_types_metrics.sh @@ -0,0 +1,53 @@ +#!/usr/bin/env bash +# Tags: no-s3-storage, no-asan, long, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e +set -o pipefail + +# The query is not atomic - it can compare states between system.parts and system.metrics from different points in time. +# So, there is inherent race condition (especially in fasttest that runs tests in parallel). +# +# But it should get the expected result eventually. +# In case of test failure, this code will do infinite loop and timeout. +verify() +{ + for ((i = 0; i < 100; ++i)); do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted + result=$( $CLICKHOUSE_CLIENT -m --query "SELECT + (SELECT sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics) = + (SELECT countIf(part_type = 'Compact'), countIf(part_type = 'Wide') FROM (SELECT part_type FROM system.parts UNION ALL SELECT part_type FROM system.projection_parts))") + if [ "$result" = "1" ]; then + echo 1 + return + fi + sleep 0.1 + done + + echo "ERROR: metrics does not match:" >&2 + $CLICKHOUSE_CLIENT -nm --query " + -- { echo } + SELECT sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics; + SELECT countIf(part_type = 'Compact'), countIf(part_type = 'Wide') FROM (SELECT part_type FROM system.parts UNION ALL SELECT part_type FROM system.projection_parts); + " +} + +$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE IF EXISTS data_01600" +# Compact - (5..10] +# Wide - >10 +$CLICKHOUSE_CLIENT --query="CREATE TABLE data_01600 (part_type String, key Int) ENGINE = MergeTree PARTITION BY part_type ORDER BY key SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=10, index_granularity = 8192, index_granularity_bytes = '10Mi'" + +# Compact +$CLICKHOUSE_CLIENT --query="INSERT INTO data_01600 SELECT 'Compact', number FROM system.numbers LIMIT 6" +verify + +# Wide +$CLICKHOUSE_CLIENT --query="INSERT INTO data_01600 SELECT 'Wide', number FROM system.numbers LIMIT 11 OFFSET 6" +verify + +# DROP and check +$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE data_01600" +verify From a8a3214185a1f36f0e0e2c36e2afd8c27c1c8a2d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Nov 2023 19:43:40 +0000 Subject: [PATCH 768/813] + documentation --- docs/en/operations/settings/settings.md | 177 ++++++++++++++++++++++++ 1 file changed, 177 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index edc1c9bdfd7..93051177ca9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4805,6 +4805,183 @@ a Tuple( 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. +## query_plan_enable_optimizations {#query_plan_enable_optimizations} + +Toggles query optimization at the query plan level. + +Possible values: + +- 0 - Disable all optimizations at the query plan level +- 1 - Enable optimizations at the query plan level (but individual optimizations may still be disabled via their individual settings) + +Default value: `1`. + +## query_plan_max_optimizations_to_apply + +Limits the total number of optimizations applied to query plan, see setting [query_plan_enable_optimizations](#query_plan_enable_optimizations). +Useful to avoid long optimization times for complex queries. +If the actual number of optimizations exceeds this setting, an exception is thrown. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +Default value: '10000' + +## query_plan_lift_up_array_join + +Toggles a query-plan-level optimization which moves ARRAY JOINs up in the execution plan. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_push_down_limit + +Toggles a query-plan-level optimization which moves LIMITs down in the execution plan. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_split_filter + +Toggles a query-plan-level optimization which splits filters into expressions. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_merge_expressions + +Toggles a query-plan-level optimization which merges consecutive filters. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_filter_push_down + +Toggles a query-plan-level optimization which moves filters down in the execution plan. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_execute_functions_after_sorting + +Toggles a query-plan-level optimization which moves expressions after sorting steps. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_reuse_storage_ordering_for_window_functions + +Toggles a query-plan-level optimization which uses storage sorting when sorting for window functions. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_lift_up_union + +Toggles a query-plan-level optimization which moves larger subtrees of the query plan into union to enable further optimizations. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_distinct_in_order + +Toggles the distinct in-order optimization query-plan-level optimization. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_read_in_order + +Toggles the read in-order optimization query-plan-level optimization. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_aggregation_in_order + +Toggles the aggregation in-order query-plan-level optimization. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `0`. + +## query_plan_remove_redundant_sorting + +Toggles a query-plan-level optimization which removes redundant sorting steps, e.g. in subqueries. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + +## query_plan_remove_redundant_distinct + +Toggles a query-plan-level optimization which removes redundant DISTINCT steps. +Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. + +Possible values: + +- 0 - Disable +- 1 - Enable + +Default value: `1`. + ## dictionary_use_async_executor {#dictionary_use_async_executor} Execute a pipeline for reading dictionary source in several threads. It's supported only by dictionaries with local CLICKHOUSE source. From e9f7690026265b3c1cd4e2251e1babb8aed8073e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Nov 2023 22:27:28 +0000 Subject: [PATCH 769/813] +s --- tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference index 999505f54c9..3603ebe3e0d 100644 --- a/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference +++ b/tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference @@ -18,7 +18,7 @@ 719529 719529 1970-01-02 1970-01-02 1970-01-02 1970-01-02 -- NULL handling \N \N --- ubsan bug +-- ubsan bugs 2299-12-31 2299-12-31 -- Alias From ace830a61823f0a760290ab4172378b643e0d21e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Nov 2023 22:56:40 +0000 Subject: [PATCH 770/813] Fix spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 077e323ad4e..711108863cb 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -261,6 +261,7 @@ FOSDEM FQDN Failover FarmHash +FileLog FilesystemCacheBytes FilesystemCacheElements FilesystemCacheFiles @@ -278,7 +279,6 @@ FilesystemMainPathTotalBytes FilesystemMainPathTotalINodes FilesystemMainPathUsedBytes FilesystemMainPathUsedINodes -FileLog FixedString Flink ForEach @@ -441,6 +441,7 @@ Kolmogorov Kubernetes LDAP LGPL +LIMITs LLDB LLVM's LOCALTIME @@ -571,13 +572,13 @@ NetworkSendPackets NodeJs NuRaft NumHexagons +NumPy NumToString NumToStringClassC NumberOfDatabases NumberOfDetachedByUserParts NumberOfDetachedParts NumberOfTables -NumPy OFNS OLAP OLTP @@ -588,10 +589,10 @@ OSGuestNiceTimeNormalized OSGuestTime OSGuestTimeCPU OSGuestTimeNormalized +OSIOWaitMicroseconds OSIOWaitTime OSIOWaitTimeCPU OSIOWaitTimeNormalized -OSIOWaitMicroseconds OSIdleTime OSIdleTimeCPU OSIdleTimeNormalized @@ -1470,12 +1471,12 @@ fastops fcoverage fibonacci fifo +filelog filesystem filesystemAvailable filesystemCapacity filesystemFree filesystems -filelog finalizeAggregation fips firstLine @@ -2348,6 +2349,7 @@ subtractSeconds subtractWeeks subtractYears subtree +subtrees subtype sudo sumCount From e5284c125f758adfe31fce74cb4eeaa5fc1c8d2a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 24 Nov 2023 10:02:47 +0100 Subject: [PATCH 771/813] Update blob_storage_log.md --- docs/en/operations/system-tables/blob_storage_log.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/blob_storage_log.md b/docs/en/operations/system-tables/blob_storage_log.md index db08b0c583d..2328f7f0346 100644 --- a/docs/en/operations/system-tables/blob_storage_log.md +++ b/docs/en/operations/system-tables/blob_storage_log.md @@ -1,7 +1,7 @@ --- slug: /en/operations/system-tables/blob_storage_log --- -# Blob Storage Operations Log +# blob_storage_log Contains logging entries with information about various blob storage operations such as uploads and deletes. From b3d378abd54403f9df73838ff298fd3dd1991c70 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 Nov 2023 12:50:29 +0100 Subject: [PATCH 772/813] Add test for #5323 --- .../02923_cte_equality_disjunction.reference | 1 + .../0_stateless/02923_cte_equality_disjunction.sql | 12 ++++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/02923_cte_equality_disjunction.reference create mode 100644 tests/queries/0_stateless/02923_cte_equality_disjunction.sql diff --git a/tests/queries/0_stateless/02923_cte_equality_disjunction.reference b/tests/queries/0_stateless/02923_cte_equality_disjunction.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02923_cte_equality_disjunction.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02923_cte_equality_disjunction.sql b/tests/queries/0_stateless/02923_cte_equality_disjunction.sql new file mode 100644 index 00000000000..288bed9e491 --- /dev/null +++ b/tests/queries/0_stateless/02923_cte_equality_disjunction.sql @@ -0,0 +1,12 @@ +--https://github.com/ClickHouse/ClickHouse/issues/5323 +CREATE TABLE test_bug_optimization +( + `path` String +) +ENGINE = MergeTree +ORDER BY path; + +WITH (path = 'test1') OR match(path, 'test2') OR (match(path, 'test3') AND match(path, 'test2')) OR match(path, 'test4') OR (path = 'test5') OR (path = 'test6') AS alias_in_error +SELECT count(1) +FROM test_bug_optimization +WHERE alias_in_error; From 683827c6df992c253c05f87db9e4b27eb31d3402 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 24 Nov 2023 14:01:52 +0100 Subject: [PATCH 773/813] Add comment. --- .../test_replicated_merge_tree_encryption_codec/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_replicated_merge_tree_encryption_codec/test.py b/tests/integration/test_replicated_merge_tree_encryption_codec/test.py index 39eb4ee02eb..c8b63f9502a 100644 --- a/tests/integration/test_replicated_merge_tree_encryption_codec/test.py +++ b/tests/integration/test_replicated_merge_tree_encryption_codec/test.py @@ -96,6 +96,9 @@ def test_different_keys(): node2.query("INSERT INTO tbl VALUES (2, 'str2')") node1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") + # After "SYSTEM SYNC REPLICA" we expect node1 and node2 here both having a part for (1, 'str1') encrypted with "key_a", + # and a part for (2, 'str2') encrypted with "key_b". + # So the command "SELECT * from tbl" must fail on both nodes because each node has only one encryption key. assert "BAD_DECRYPT" in node1.query_and_get_error("SELECT * FROM tbl") assert "BAD_DECRYPT" in node2.query_and_get_error("SELECT * FROM tbl") From d18828c826e84a1ecbde3d9aeb6d3f6436d3a1c8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Nov 2023 14:04:42 +0100 Subject: [PATCH 774/813] Revert "Mark select() as harmful function" --- base/harmful/harmful.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 6526b7e577e..78796ca0c05 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -145,8 +145,6 @@ TRAP(qecvt) TRAP(qfcvt) TRAP(register_printf_function) TRAP(seed48) -TRAP(select) -TRAP(pselect) //TRAP(setenv) TRAP(setfsent) TRAP(setgrent) From 740cc3c053c5281bc5f1a13222472aed9f09fd3d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 24 Nov 2023 12:32:02 +0100 Subject: [PATCH 775/813] Set log_comment to the file name while processing files in client This will be useful for fuzzer, to know which file had been processed to trigger the crash, since right now you need to find unique parts of the query that had not been added by fuzzer to reproduce. But I guess this will be useful not only for fuzzing, but for general introspection as well. Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 8 ++++++++ .../02930_client_file_log_comment.reference | 4 ++++ .../02930_client_file_log_comment.sh | 19 +++++++++++++++++++ 3 files changed, 31 insertions(+) create mode 100644 tests/queries/0_stateless/02930_client_file_log_comment.reference create mode 100755 tests/queries/0_stateless/02930_client_file_log_comment.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b513e623829..9607e989bed 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2566,6 +2566,14 @@ bool ClientBase::processMultiQueryFromFile(const String & file_name) ReadBufferFromFile in(file_name); readStringUntilEOF(queries_from_file, in); + if (!global_context->getSettings().log_comment.changed) + { + Settings settings = global_context->getSettings(); + /// NOTE: cannot use even weakly_canonical() since it fails for /dev/stdin due to resolving of "pipe:[X]" + settings.log_comment = fs::absolute(fs::path(file_name)); + global_context->setSettings(settings); + } + return executeMultiQuery(queries_from_file); } diff --git a/tests/queries/0_stateless/02930_client_file_log_comment.reference b/tests/queries/0_stateless/02930_client_file_log_comment.reference new file mode 100644 index 00000000000..09639302c0f --- /dev/null +++ b/tests/queries/0_stateless/02930_client_file_log_comment.reference @@ -0,0 +1,4 @@ +42 +select 42\n /dev/stdin +4242 +select 4242\n foo diff --git a/tests/queries/0_stateless/02930_client_file_log_comment.sh b/tests/queries/0_stateless/02930_client_file_log_comment.sh new file mode 100755 index 00000000000..c425f28ecbe --- /dev/null +++ b/tests/queries/0_stateless/02930_client_file_log_comment.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --queries-file /dev/stdin <<<'select 42' +$CLICKHOUSE_CLIENT -nm -q " + system flush logs; + select query, log_comment from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and event_date >= yesterday() and query = 'select 42\n' and type != 'QueryStart'; +" + +$CLICKHOUSE_CLIENT --log_comment foo --queries-file /dev/stdin <<<'select 4242' +$CLICKHOUSE_CLIENT -nm -q " + system flush logs; + select query, log_comment from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and event_date >= yesterday() and query = 'select 4242\n' and type != 'QueryStart'; +" From 472027e948c0bbce4b24623a8444fa894ad8a930 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 24 Nov 2023 21:47:00 +0800 Subject: [PATCH 776/813] Fix JOIN plan with normal projection --- .../Optimizations/optimizeUseNormalProjection.cpp | 4 ++-- .../01710_normal_projection_join_plan_fix.reference | 0 .../01710_normal_projection_join_plan_fix.sql | 13 +++++++++++++ 3 files changed, 15 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01710_normal_projection_join_plan_fix.reference create mode 100644 tests/queries/0_stateless/01710_normal_projection_join_plan_fix.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 6880d21facb..e4b3e4f84ab 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -268,7 +268,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) } else { - const auto & main_stream = iter->node->children.front()->step->getOutputStream(); + const auto & main_stream = iter->node->children[iter->next_child - 1]->step->getOutputStream(); const auto * proj_stream = &next_node->step->getOutputStream(); if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header)) @@ -284,7 +284,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) auto & union_node = nodes.emplace_back(); DataStreams input_streams = {main_stream, *proj_stream}; union_node.step = std::make_unique(std::move(input_streams)); - union_node.children = {iter->node->children.front(), next_node}; + union_node.children = {iter->node->children[iter->next_child - 1], next_node}; iter->node->children[iter->next_child - 1] = &union_node; } diff --git a/tests/queries/0_stateless/01710_normal_projection_join_plan_fix.reference b/tests/queries/0_stateless/01710_normal_projection_join_plan_fix.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_normal_projection_join_plan_fix.sql b/tests/queries/0_stateless/01710_normal_projection_join_plan_fix.sql new file mode 100644 index 00000000000..40847a301c2 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_join_plan_fix.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (id UInt32, s String) Engine = MergeTree ORDER BY id; +CREATE TABLE t2 (id1 UInt32, id2 UInt32) Engine = MergeTree ORDER BY id1 SETTINGS index_granularity = 1; +INSERT INTO t2 SELECT number, number from numbers(100); +ALTER TABLE t2 ADD PROJECTION proj (SELECT id2 ORDER BY id2); +INSERT INTO t2 SELECT number, number from numbers(100); + +SELECT s FROM t1 as lhs LEFT JOIN (SELECT * FROM t2 WHERE id2 = 2) as rhs ON lhs.id = rhs.id2; + +DROP TABLE t1; +DROP TABLE t2; From 81da52bdf4e2e8b6d64da1d4ecc2df84f55d3034 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 23 Nov 2023 17:22:39 +0100 Subject: [PATCH 777/813] Fix 02903_rmt_retriable_merge_exception flakiness for replicated database In case of replicated database system stop pulling replication log for rmt2 should be done on all replicas, otherwise some replica may merge the part and all other replicas may fetch it. Also, since SYSTEM STOP PULLING REPLICATION LOG does not waits for the current pull, let's trigger log pull explicitly to provide at least some guarantee that replication log pulling had been stopped, otherwise race is possible [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/57155/f68717ccd0a07a499911c9b0db7537ae8205e41b/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin --- .../02903_rmt_retriable_merge_exception.sh | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh index 074a3a6725e..095239954f4 100755 --- a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh +++ b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh @@ -10,7 +10,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # (i.e. "No active replica has part X or covering part") # does not appears as errors (level=Error), only as info message (level=Information). -$CLICKHOUSE_CLIENT -nm -q " +cluster=default +if [[ $($CLICKHOUSE_CLIENT -q "select count()>0 from system.clusters where cluster = 'test_cluster_database_replicated'") = 1 ]]; then + cluster=test_cluster_database_replicated +fi + +$CLICKHOUSE_CLIENT -nm --distributed_ddl_output_mode=none -q " drop table if exists rmt1; drop table if exists rmt2; @@ -21,7 +26,12 @@ $CLICKHOUSE_CLIENT -nm -q " insert into rmt1 values (2); system sync replica rmt1; - system stop pulling replication log rmt2; + -- SYSTEM STOP PULLING REPLICATION LOG does not waits for the current pull, + -- trigger it explicitly to 'avoid race' (though proper way will be to wait + -- for current pull in the StorageReplicatedMergeTree::getActionLock()) + system sync replica rmt2; + -- NOTE: CLICKHOUSE_DATABASE is required + system stop pulling replication log on cluster $cluster $CLICKHOUSE_DATABASE.rmt2; optimize table rmt1 final settings alter_sync=0, optimize_throw_if_noop=1; " || exit 1 From 106053dee8adb455522833eb73cdeaa7b317b712 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Nov 2023 14:22:39 +0000 Subject: [PATCH 778/813] Add a test. --- .../0_stateless/02915_analyzer_fuzz_6.reference | 0 .../queries/0_stateless/02915_analyzer_fuzz_6.sql | 14 ++++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/02915_analyzer_fuzz_6.reference create mode 100644 tests/queries/0_stateless/02915_analyzer_fuzz_6.sql diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_6.reference b/tests/queries/0_stateless/02915_analyzer_fuzz_6.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql b/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql new file mode 100644 index 00000000000..9e54207c9ee --- /dev/null +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql @@ -0,0 +1,14 @@ +set allow_suspicious_low_cardinality_types=1; +CREATE TABLE t__fuzz_307 (`k1` DateTime, `k2` LowCardinality(Nullable(Float64)), `v` Nullable(UInt32)) ENGINE = + ReplacingMergeTree ORDER BY (k1, k2) settings allow_nullable_key=1; + set allow_experimental_analyzer=1; + insert into t__fuzz_307 select * from generateRandom() limit 10; + SELECT arrayJoin([tuple([(toNullable(NULL), -9223372036854775808, toNullable(3.4028234663852886e38), arrayJoin( +[tuple([(toNullable(NULL), 2147483647, toNullable(0.5), k2)])]), k2)])]) AS row, arrayJoin([(1024, k2)]), -9223372036854775807, 256, tupleElement(row, 1048576, 1024) AS k FROM t__fuzz_307 FINAL ORDER BY (toNullable('655.36'), 2, toNullable +('0.2147483648'), k2) ASC, toNullable('102.3') DESC NULLS FIRST, '10.25' DESC, k ASC NULLS FIRST format Null; + +CREATE TABLE t__fuzz_282 (`k1` DateTime, `k2` LowCardinality(Nullable(Float64)), `v` Nullable(UInt32)) ENGINE = ReplacingMergeTree ORDER BY (k1, k2) SETTINGS allow_nullable_key = 1; +SET optimize_on_insert = 0; +INSERT INTO t__fuzz_282 VALUES (1, 2, 3) (1, 2, 4) (2, 3, 4), (2, 3, 5); + +SELECT arrayJoin([tuple([(toNullable(NULL), -9223372036854775808, toNullable(3.4028234663852886e38), arrayJoin([tuple([(toNullable(NULL), 2147483647, toNullable(0.5), k2)])]), k2)])]) AS row, arrayJoin([(1024, k2)]), -9223372036854775807, 256, tupleElement(row, 1048576, 1024) AS k FROM t__fuzz_282 FINAL ORDER BY (toNullable('655.36'), 2, toNullable('0.2147483648'), k2) ASC, toNullable('102.3') DESC NULLS FIRST, '10.25' DESC, k ASC NULLS FIRST; From 75ceaf6200976eabd219ffa818df9f9186acac3f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 Nov 2023 14:31:39 +0000 Subject: [PATCH 779/813] Docs: Improve math function docs --- .../sql-reference/functions/math-functions.md | 166 +++++++++++++++++- 1 file changed, 162 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 9eab2274210..9edd35ce354 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -6,11 +6,9 @@ sidebar_label: Mathematical # Mathematical Functions -All the functions return a Float64 number. Results are generally as close to the actual result as possible, but in some cases less precise than the machine-representable number. - ## e -Returns e. +Returns e ([Euler's constant](https://en.wikipedia.org/wiki/Euler%27s_constant)) **Syntax** @@ -18,15 +16,22 @@ Returns e. e() ``` +**Returned value** + +Type: [Float64](../../sql-reference/data-types/float.md). + ## pi -Returns π. +Returns π ([Pi](https://en.wikipedia.org/wiki/Pi)). **Syntax** ```sql pi() ``` +**Returned value** + +Type: [Float64](../../sql-reference/data-types/float.md). ## exp @@ -38,6 +43,14 @@ Returns e to the power of the given argument. exp(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## log Returns the natural logarithm of the argument. @@ -50,6 +63,14 @@ log(x) Alias: `ln(x)` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## exp2 Returns 2 to the power of the given argument @@ -60,6 +81,14 @@ Returns 2 to the power of the given argument exp2(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## intExp2 Like `exp` but returns a UInt64. @@ -80,6 +109,14 @@ Returns the binary logarithm of the argument. log2(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## exp10 Returns 10 to the power of the given argument. @@ -90,6 +127,14 @@ Returns 10 to the power of the given argument. exp10(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## intExp10 Like `exp10` but returns a UInt64. @@ -110,6 +155,14 @@ Returns the decimal logarithm of the argument. log10(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## sqrt Returns the square root of the argument. @@ -118,6 +171,14 @@ Returns the square root of the argument. sqrt(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## cbrt Returns the cubic root of the argument. @@ -126,6 +187,14 @@ Returns the cubic root of the argument. cbrt(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## erf If `x` is non-negative, then `erf(x / σ√2)` is the probability that a random variable having a normal distribution with standard deviation `σ` takes the value that is separated from the expected value by more than `x`. @@ -136,6 +205,14 @@ If `x` is non-negative, then `erf(x / σ√2)` is the probability that a random erf(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + **Example** (three sigma rule) @@ -160,6 +237,14 @@ Returns a number close to `1 - erf(x)` without loss of precision for large ‘x erfc(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## lgamma Returns the logarithm of the gamma function. @@ -170,6 +255,14 @@ Returns the logarithm of the gamma function. lgamma(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## tgamma Returns the gamma function. @@ -180,6 +273,14 @@ Returns the gamma function. gamma(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## sin Returns the sine of the argument @@ -190,6 +291,14 @@ Returns the sine of the argument sin(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## cos Returns the cosine of the argument. @@ -200,6 +309,14 @@ Returns the cosine of the argument. cos(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## tan Returns the tangent of the argument. @@ -210,6 +327,14 @@ Returns the tangent of the argument. tan(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## asin Returns the arc sine of the argument. @@ -220,6 +345,14 @@ Returns the arc sine of the argument. asin(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## acos Returns the arc cosine of the argument. @@ -230,6 +363,14 @@ Returns the arc cosine of the argument. acos(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## atan Returns the arc tangent of the argument. @@ -240,6 +381,14 @@ Returns the arc tangent of the argument. atan(x) ``` +**Arguments** + +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) + +**Returned value** + +Type: [Float*](../../sql-reference/data-types/float.md). + ## pow Returns `x` to the power of `y`. @@ -252,6 +401,15 @@ pow(x, y) Alias: `power(x, y)` +**Arguments** + +- `x` - [(U)Int8/16/32/64](../../sql-reference/data-types/int-uint.md) or [Float*](../../sql-reference/data-types/float.md) +- `y` - [(U)Int8/16/32/64](../../sql-reference/data-types/int-uint.md) or [Float*](../../sql-reference/data-types/float.md) + +**Returned value** + +Type: [Float64](../../sql-reference/data-types/float.md). + ## cosh Returns the [hyperbolic cosine](https://in.mathworks.com/help/matlab/ref/cosh.html) of the argument. From ca63881357017a642e4ec37c551ea2823b896e3b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 Nov 2023 14:46:02 +0000 Subject: [PATCH 780/813] Mention that the settings are dev settings --- docs/en/operations/settings/settings.md | 60 +++++++++++++++++++++++++ 1 file changed, 60 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 93051177ca9..663572d91c8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4809,6 +4809,10 @@ If a table has a space-filling curve in its index, e.g. `ORDER BY mortonEncode(x Toggles query optimization at the query plan level. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable all optimizations at the query plan level @@ -4822,6 +4826,10 @@ Limits the total number of optimizations applied to query plan, see setting [que Useful to avoid long optimization times for complex queries. If the actual number of optimizations exceeds this setting, an exception is thrown. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Type: [UInt64](../../sql-reference/data-types/int-uint.md). Default value: '10000' @@ -4831,6 +4839,10 @@ Default value: '10000' Toggles a query-plan-level optimization which moves ARRAY JOINs up in the execution plan. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4843,6 +4855,10 @@ Default value: `1`. Toggles a query-plan-level optimization which moves LIMITs down in the execution plan. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4852,6 +4868,10 @@ Default value: `1`. ## query_plan_split_filter +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Toggles a query-plan-level optimization which splits filters into expressions. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. @@ -4867,6 +4887,10 @@ Default value: `1`. Toggles a query-plan-level optimization which merges consecutive filters. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4879,6 +4903,10 @@ Default value: `1`. Toggles a query-plan-level optimization which moves filters down in the execution plan. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4891,6 +4919,10 @@ Default value: `1`. Toggles a query-plan-level optimization which moves expressions after sorting steps. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4903,6 +4935,10 @@ Default value: `1`. Toggles a query-plan-level optimization which uses storage sorting when sorting for window functions. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4915,6 +4951,10 @@ Default value: `1`. Toggles a query-plan-level optimization which moves larger subtrees of the query plan into union to enable further optimizations. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4927,6 +4967,10 @@ Default value: `1`. Toggles the distinct in-order optimization query-plan-level optimization. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4939,6 +4983,10 @@ Default value: `1`. Toggles the read in-order optimization query-plan-level optimization. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4951,6 +4999,10 @@ Default value: `1`. Toggles the aggregation in-order query-plan-level optimization. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4963,6 +5015,10 @@ Default value: `0`. Toggles a query-plan-level optimization which removes redundant sorting steps, e.g. in subqueries. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable @@ -4975,6 +5031,10 @@ Default value: `1`. Toggles a query-plan-level optimization which removes redundant DISTINCT steps. Only takes effect if setting [query_plan_enable_optimizations](#query_plan_enable_optimizations) is 1. +:::note +This is an expert-level setting which should only be used for debugging by developers. The setting may change in future in backward-incompatible ways or be removed. +::: + Possible values: - 0 - Disable From fc0f690b3ac82a8794b1ed43220a32353bf6341a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Nov 2023 14:54:58 +0000 Subject: [PATCH 781/813] Simplify test. --- .../0_stateless/02915_analyzer_fuzz_6.reference | 2 ++ tests/queries/0_stateless/02915_analyzer_fuzz_6.sql | 11 ++++++++--- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_6.reference b/tests/queries/0_stateless/02915_analyzer_fuzz_6.reference index e69de29bb2d..b5c035d8576 100644 --- a/tests/queries/0_stateless/02915_analyzer_fuzz_6.reference +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_6.reference @@ -0,0 +1,2 @@ +[(0,0)] +[(1,1)] diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql b/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql index 9e54207c9ee..b4eb1b4aff4 100644 --- a/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql @@ -1,14 +1,19 @@ set allow_suspicious_low_cardinality_types=1; +set allow_experimental_analyzer=1; + +create table tab (x LowCardinality(Nullable(Float64))) engine = MergeTree order by x settings allow_nullable_key=1; +insert into tab select number from numbers(2); +SELECT [(arrayJoin([x]), x)] AS row FROM tab; + + CREATE TABLE t__fuzz_307 (`k1` DateTime, `k2` LowCardinality(Nullable(Float64)), `v` Nullable(UInt32)) ENGINE = ReplacingMergeTree ORDER BY (k1, k2) settings allow_nullable_key=1; - set allow_experimental_analyzer=1; insert into t__fuzz_307 select * from generateRandom() limit 10; SELECT arrayJoin([tuple([(toNullable(NULL), -9223372036854775808, toNullable(3.4028234663852886e38), arrayJoin( [tuple([(toNullable(NULL), 2147483647, toNullable(0.5), k2)])]), k2)])]) AS row, arrayJoin([(1024, k2)]), -9223372036854775807, 256, tupleElement(row, 1048576, 1024) AS k FROM t__fuzz_307 FINAL ORDER BY (toNullable('655.36'), 2, toNullable ('0.2147483648'), k2) ASC, toNullable('102.3') DESC NULLS FIRST, '10.25' DESC, k ASC NULLS FIRST format Null; CREATE TABLE t__fuzz_282 (`k1` DateTime, `k2` LowCardinality(Nullable(Float64)), `v` Nullable(UInt32)) ENGINE = ReplacingMergeTree ORDER BY (k1, k2) SETTINGS allow_nullable_key = 1; -SET optimize_on_insert = 0; INSERT INTO t__fuzz_282 VALUES (1, 2, 3) (1, 2, 4) (2, 3, 4), (2, 3, 5); -SELECT arrayJoin([tuple([(toNullable(NULL), -9223372036854775808, toNullable(3.4028234663852886e38), arrayJoin([tuple([(toNullable(NULL), 2147483647, toNullable(0.5), k2)])]), k2)])]) AS row, arrayJoin([(1024, k2)]), -9223372036854775807, 256, tupleElement(row, 1048576, 1024) AS k FROM t__fuzz_282 FINAL ORDER BY (toNullable('655.36'), 2, toNullable('0.2147483648'), k2) ASC, toNullable('102.3') DESC NULLS FIRST, '10.25' DESC, k ASC NULLS FIRST; +SELECT arrayJoin([tuple([(toNullable(NULL), -9223372036854775808, toNullable(3.4028234663852886e38), arrayJoin([tuple([(toNullable(NULL), 2147483647, toNullable(0.5), k2)])]), k2)])]) AS row, arrayJoin([(1024, k2)]), -9223372036854775807, 256, tupleElement(row, 1048576, 1024) AS k FROM t__fuzz_282 FINAL ORDER BY (toNullable('655.36'), 2, toNullable('0.2147483648'), k2) ASC, toNullable('102.3') DESC NULLS FIRST, '10.25' DESC, k ASC NULLS FIRST format Null; From 0bf0e9b0864959b9cbfe202a6892ff7578392039 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 Nov 2023 16:00:29 +0100 Subject: [PATCH 782/813] Add test for #47366 --- .../02923_join_use_nulls_modulo.reference | 0 .../02923_join_use_nulls_modulo.sql | 22 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02923_join_use_nulls_modulo.reference create mode 100644 tests/queries/0_stateless/02923_join_use_nulls_modulo.sql diff --git a/tests/queries/0_stateless/02923_join_use_nulls_modulo.reference b/tests/queries/0_stateless/02923_join_use_nulls_modulo.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02923_join_use_nulls_modulo.sql b/tests/queries/0_stateless/02923_join_use_nulls_modulo.sql new file mode 100644 index 00000000000..4134a42c599 --- /dev/null +++ b/tests/queries/0_stateless/02923_join_use_nulls_modulo.sql @@ -0,0 +1,22 @@ +--https://github.com/ClickHouse/ClickHouse/issues/47366 +SELECT + id % 255, + toTypeName(d.id) +FROM +( + SELECT + toLowCardinality(1048577) AS id, + toLowCardinality(9223372036854775807) AS value + GROUP BY + GROUPING SETS ( + (toLowCardinality(1024)), + (id % 10.0001), + ((id % 2147483646) != -9223372036854775807), + ((id % -1) != 255)) + ) AS a + SEMI LEFT JOIN +( + SELECT toLowCardinality(9223372036854775807) AS id + WHERE (id % 2147483646) != NULL +) AS d USING (id) +SETTINGS join_use_nulls=1; From ed4bd7b4e87ccce7fb8a94793e9ddf2514ab3779 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 24 Nov 2023 15:16:44 +0000 Subject: [PATCH 783/813] do not demangle stack frames from __functional --- src/Common/StackTrace.cpp | 28 ++++++++++++++++++++++------ 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index b323f1e4363..21235914f7c 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -291,9 +291,20 @@ void StackTrace::tryCapture() constexpr std::pair replacements[] = {{"::__1", ""}, {"std::basic_string, std::allocator>", "String"}}; -String collapseNames(String && haystack) +// Demangle @c symbol_name if it's not from __functional header (as such functions don't provide any useful +// information but pollute stack traces). +// Replace parts from @c replacements with shorter aliases +String demangleAndCollapseNames(std::string_view file, const char * const symbol_name) { - // TODO: surely there is a written version already for better in place search&replace + std::string_view file_copy = file; + if (auto trim_pos = file.find_last_of('/'); trim_pos != file.npos) + file_copy.remove_suffix(file.size() - trim_pos); + if (file_copy.ends_with("functional")) + return "?"; + + String haystack = demangle(symbol_name); + + // TODO myrrc surely there is a written version already for better in place search&replace for (auto [needle, to] : replacements) { size_t pos = 0; @@ -354,6 +365,7 @@ toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & s DB::WriteBufferFromOwnString out; out << i << ". "; + String file; if (std::error_code ec; object && std::filesystem::exists(object->name, ec) && !ec) { auto dwarf_it = dwarfs.try_emplace(object->name, object->elf).first; @@ -361,11 +373,14 @@ toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & s DB::Dwarf::LocationInfo location; if (dwarf_it->second.findAddress(uintptr_t(physical_addr), location, mode, inline_frames)) - out << location.file.toString() << ":" << location.line << ": "; + { + file = location.file.toString(); + out << file << ":" << location.line << ": "; + } } if (const auto * const symbol = symbol_index.findSymbol(virtual_addr)) - out << collapseNames(demangle(symbol->name)); + out << demangleAndCollapseNames(file, symbol->name); else out << "?"; @@ -380,13 +395,14 @@ toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & s for (size_t j = 0; j < inline_frames.size(); ++j) { const auto & frame = inline_frames[j]; + const String file_for_inline_frame = frame.location.file.toString(); callback(fmt::format( "{}.{}. inlined from {}:{}: {}", i, j + 1, - frame.location.file.toString(), + file_for_inline_frame, frame.location.line, - collapseNames(demangle(frame.name)))); + demangleAndCollapseNames(file_for_inline_frame, frame.name))); } callback(out.str()); From 22fcf5b99f677e805824187ad75751379341cced Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 Nov 2023 17:39:14 +0100 Subject: [PATCH 784/813] Add test for #51321 --- .../0_stateless/02923_explain_expired_context.reference | 0 tests/queries/0_stateless/02923_explain_expired_context.sql | 3 +++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02923_explain_expired_context.reference create mode 100644 tests/queries/0_stateless/02923_explain_expired_context.sql diff --git a/tests/queries/0_stateless/02923_explain_expired_context.reference b/tests/queries/0_stateless/02923_explain_expired_context.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02923_explain_expired_context.sql b/tests/queries/0_stateless/02923_explain_expired_context.sql new file mode 100644 index 00000000000..fa365004cb0 --- /dev/null +++ b/tests/queries/0_stateless/02923_explain_expired_context.sql @@ -0,0 +1,3 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/51321 +EXPLAIN ESTIMATE SELECT any(toTypeName(s)) FROM (SELECT 'bbbbbbbb', toTypeName(s), CAST('', 'LowCardinality(String)'), NULL, CAST('\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0', 'String') AS s WITH TOTALS) AS t1 FULL OUTER JOIN (SELECT CAST('bbbbb\0\0bbb\0bb\0bb', 'LowCardinality(String)'), CAST(CAST('a', 'String'), 'LowCardinality(String)') AS s GROUP BY CoNnEcTiOn_Id()) AS t2 USING (s) WITH TOTALS; +EXPLAIN ESTIMATE SELECT any(s) FROM (SELECT '' AS s) AS t1 JOIN (SELECT '' AS s GROUP BY connection_id()) AS t2 USING (s); From 0eb20620cd99259ec3fe37f9e201392bb3880f2d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Nov 2023 16:42:59 +0000 Subject: [PATCH 785/813] Fix type for arrayJoin(LC) --- src/Functions/array/arrayJoin.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/array/arrayJoin.cpp b/src/Functions/array/arrayJoin.cpp index cc854dc6807..264c65bc963 100644 --- a/src/Functions/array/arrayJoin.cpp +++ b/src/Functions/array/arrayJoin.cpp @@ -50,6 +50,7 @@ public: } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { From b6bd5ecb199ef8a10e3008a4ea3d96087db8a8c1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 24 Nov 2023 17:46:25 +0100 Subject: [PATCH 786/813] Fix possible crash of fuzzy finder in client Refs: https://github.com/lotabout/tuikit/pull/51 Signed-off-by: Azat Khuzhin --- rust/skim/Cargo.toml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/rust/skim/Cargo.toml b/rust/skim/Cargo.toml index 0381ad81619..35bd0fa018b 100644 --- a/rust/skim/Cargo.toml +++ b/rust/skim/Cargo.toml @@ -23,3 +23,6 @@ debug = true inherits = "release" # We use LTO here as well to slightly decrease binary size lto = true + +[patch.crates-io] +tuikit = { git = "https://github.com/azat-rust/tuikit.git", rev = "e1994c0e03ff02c49cf1471f0cc3cbf185ce0104" } From c3b617bd7baf2a3f61aa883d440f72f39d568b8a Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Fri, 24 Nov 2023 19:00:00 +0100 Subject: [PATCH 787/813] Move class functions upper --- .../RocksDB/StorageEmbeddedRocksDB.cpp | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 7897bd1cba7..524d8e0b614 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -690,17 +690,6 @@ Chunk StorageEmbeddedRocksDB::getBySerializedKeys( return Chunk(std::move(columns), num_rows); } -void registerStorageEmbeddedRocksDB(StorageFactory & factory) -{ - StorageFactory::StorageFeatures features{ - .supports_sort_order = true, - .supports_ttl = true, - .supports_parallel_insert = true, - }; - - factory.registerStorage("EmbeddedRocksDB", create, features); -} - std::optional StorageEmbeddedRocksDB::totalRows(const Settings & settings) const { if (!settings.optimize_trivial_approximate_count_query) @@ -725,4 +714,14 @@ std::optional StorageEmbeddedRocksDB::totalBytes(const Settings & /*sett return estimated_bytes; } +void registerStorageEmbeddedRocksDB(StorageFactory & factory) +{ + StorageFactory::StorageFeatures features{ + .supports_sort_order = true, + .supports_ttl = true, + .supports_parallel_insert = true, + }; + + factory.registerStorage("EmbeddedRocksDB", create, features); +} } From 57445d46cc0bc9cf4f2129fd2f7e545f56da452a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Nov 2023 22:24:53 +0100 Subject: [PATCH 788/813] Slightly improve the generator readbility --- utils/security-generator/generate_security.py | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/utils/security-generator/generate_security.py b/utils/security-generator/generate_security.py index 83180ccce1c..ccf9a82067e 100755 --- a/utils/security-generator/generate_security.py +++ b/utils/security-generator/generate_security.py @@ -52,10 +52,7 @@ def generate_supported_versions() -> str: with open(VERSIONS_FILE, "r", encoding="utf-8") as fd: versions = [line.split(maxsplit=1)[0][1:] for line in fd.readlines()] - # The versions in VERSIONS_FILE are ordered ascending, so the first one is - # the greatest one. We may have supported versions in the previous year - greatest_year = int(versions[0].split(".", maxsplit=1)[0]) - unsupported_year = greatest_year - 2 + supported_year = 0 # set automatically when all supported versions are filled # 3 regular versions regular = [] # type: List[str] max_regular = 3 @@ -82,14 +79,12 @@ def generate_supported_versions() -> str: lts.append(version) to_append = f"| {version} | ✔️ |" if to_append: - if len(regular) == max_regular and len(lts) == max_lts: - # if we reached the max number of supported versions, the rest - # are unsopported, so year.* will be used - unsupported_year = min(greatest_year - 1, year) + if len(regular) == max_regular or len(lts) == max_lts: + supported_year = year table.append(to_append) continue - if year <= unsupported_year: - # The whole year is unsopported + if year < supported_year: + # The whole year is unsupported version = f"{year}.*" if not version in unsupported: unsupported.append(version) From dce47b4af4f247288c09e98209f532efc9e2b95c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 24 Nov 2023 20:14:32 +0100 Subject: [PATCH 789/813] Remove long-living remainings --- docker/server/.dockerignore | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 docker/server/.dockerignore diff --git a/docker/server/.dockerignore b/docker/server/.dockerignore deleted file mode 100644 index d360712c18f..00000000000 --- a/docker/server/.dockerignore +++ /dev/null @@ -1,8 +0,0 @@ -# post / preinstall scripts (not needed, we do it in Dockerfile) -alpine-root/install/* - -# docs (looks useless) -alpine-root/usr/share/doc/* - -# packages, etc. (used by alpine-build.sh) -tgz-packages/* From d0b4917fd33ba5ee096e3d0f9edaf589ac4a891a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 24 Nov 2023 20:28:18 +0100 Subject: [PATCH 790/813] Fix test_distributed_storage_configuration flakiness It fails [1] due to extra reservation on disk2 for some system.*_log tables, sure we can turn them off, but better to fix it explicitly with a separate policy. [1]: https://s3.amazonaws.com/clickhouse-test-reports/0/98cddf5312722e403dcea429639ac13dc6cada33/integration_tests__tsan__[2_6].html Signed-off-by: Azat Khuzhin --- .../configs/config.d/overrides.xml | 4 ++-- .../test_distributed_storage_configuration/test.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_distributed_storage_configuration/configs/config.d/overrides.xml b/tests/integration/test_distributed_storage_configuration/configs/config.d/overrides.xml index 91a22a81a22..e1e2444992a 100644 --- a/tests/integration/test_distributed_storage_configuration/configs/config.d/overrides.xml +++ b/tests/integration/test_distributed_storage_configuration/configs/config.d/overrides.xml @@ -27,14 +27,14 @@ - +
disk1 disk2
-
+
diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index b0e17da37b2..00620668bd9 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -53,7 +53,7 @@ def test_insert(start_cluster): test, foo, key%2, - 'default' + 'jbod_policy' ) """ ) From bccd33932b5fe17ced2dc2f27813da0b1c034afa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Nov 2023 22:59:29 +0300 Subject: [PATCH 791/813] Update Cargo.toml --- rust/skim/Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/rust/skim/Cargo.toml b/rust/skim/Cargo.toml index 35bd0fa018b..22af40c3e33 100644 --- a/rust/skim/Cargo.toml +++ b/rust/skim/Cargo.toml @@ -25,4 +25,5 @@ inherits = "release" lto = true [patch.crates-io] +# Ref: https://github.com/lotabout/tuikit/pull/51 tuikit = { git = "https://github.com/azat-rust/tuikit.git", rev = "e1994c0e03ff02c49cf1471f0cc3cbf185ce0104" } From bad410c6c4bc0222d4da301bd07cd0d89a1676c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Nov 2023 23:10:48 +0300 Subject: [PATCH 792/813] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index fd977b4b420..8aeb06ec27b 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -212,11 +212,11 @@ quit gdb -batch -command script.gdb -p $server_pid & sleep 5 - # gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) + # gdb will send SIGSTOP, spend some time loading debug info, and then send SIGCONT, wait for it (up to send_timeout, 300s) time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: # Check connectivity after we attach gdb, because it might cause the server - # to freeze and the fuzzer will fail. In debug build it can take a lot of time. + # to freeze, and the fuzzer will fail. In debug build, it can take a lot of time. for _ in {1..180} do if clickhouse-client --query "select 1" @@ -226,15 +226,15 @@ quit sleep 1 done kill -0 $server_pid # This checks that it is our server that is started and not some other one - echo 'Server started and responded' + echo 'Server started and responded.' setup_logs_replication # SC2012: Use find instead of ls to better handle non-alphanumeric filenames. They are all alphanumeric. - # SC2046: Quote this to prevent word splitting. Actually I need word splitting. + # SC2046: Quote this to prevent word splitting. Actually, I need word splitting. # shellcheck disable=SC2012,SC2046 timeout -s TERM --preserve-status 30m clickhouse-client \ - --max-memory-usage-for-client=1000000000 \ + --max_memory_usage_in_client=1000000000 \ --receive_timeout=10 \ --receive_data_timeout_ms=10000 \ --stacktrace \ @@ -254,10 +254,10 @@ quit wait "$fuzzer_pid" || fuzzer_exit_code=$? echo "Fuzzer exit code is $fuzzer_exit_code" - # If the server dies, most often the fuzzer returns code 210: connetion + # If the server dies, most often the fuzzer returns Code 210: Connetion # refused, and sometimes also code 32: attempt to read after eof. For - # simplicity, check again whether the server is accepting connections, using - # clickhouse-client. We don't check for existence of server process, because + # simplicity, check again whether the server is accepting connections using + # clickhouse-client. We don't check for the existence of the server process, because # the process is still present while the server is terminating and not # accepting the connections anymore. From 13181db20e07039afb6de9f79b077815df2523a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Nov 2023 23:11:23 +0300 Subject: [PATCH 793/813] Update stress.py --- tests/ci/stress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 0a6b379e714..aec27af6381 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -125,7 +125,7 @@ def call_with_retry(query: str, timeout: int = 30, retry_count: int = 5) -> None def make_query_command(query: str) -> str: return ( f'clickhouse client -q "{query}" --max_untracked_memory=1Gi ' - "--memory_profiler_step=1Gi --max_memory_usage_for_user=0 --max-memory-usage-for-client=1000000000" + "--memory_profiler_step=1Gi --max_memory_usage_for_user=0 --max_memory_usage_in_client=1000000000" ) From 143617e303a04121ff4cf9613974960165b92ec6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 02:54:41 +0100 Subject: [PATCH 794/813] Remove garbage --- contrib/libunwind-cmake/CMakeLists.txt | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 0d872bae5d1..8f3cd8bd07b 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -20,15 +20,7 @@ set(LIBUNWIND_ASM_SOURCES "${LIBUNWIND_SOURCE_DIR}/src/UnwindRegistersRestore.S" "${LIBUNWIND_SOURCE_DIR}/src/UnwindRegistersSave.S") -# CMake doesn't pass the correct architecture for Apple prior to CMake 3.19 [1] -# Workaround these two issues by compiling as C. -# -# [1]: https://gitlab.kitware.com/cmake/cmake/-/issues/20771 -if (APPLE AND CMAKE_VERSION VERSION_LESS 3.19) - set_source_files_properties(${LIBUNWIND_ASM_SOURCES} PROPERTIES LANGUAGE C) -else() - enable_language(ASM) -endif() +enable_language(ASM) set(LIBUNWIND_SOURCES ${LIBUNWIND_CXX_SOURCES} From 5eb3cafb52508406469255244592bbd80b370707 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 02:54:57 +0100 Subject: [PATCH 795/813] libunwind: fix slowness under Musl --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index 30cc1d3fd36..40d8eadf96b 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 30cc1d3fd3655a5cfa0ab112fe320fb9fc0a8344 +Subproject commit 40d8eadf96b127d9b22d53ce7a4fc52aaedea965 From 2222d8cbf20209874a84383426d6e52ea8e65857 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 18:47:21 +0100 Subject: [PATCH 796/813] Update Sentry --- contrib/sentry-native | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sentry-native b/contrib/sentry-native index ae10fb8c224..d1f0883e020 160000 --- a/contrib/sentry-native +++ b/contrib/sentry-native @@ -1 +1 @@ -Subproject commit ae10fb8c224c3f41571446e1ed7fd57b9e5e366b +Subproject commit d1f0883e020f5c1b182c2b36ea26ea9c6fa64e39 From 1e00048cf7aea3f1123d34443968195f6fee21d8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 19:11:24 +0100 Subject: [PATCH 797/813] Update Sentry --- contrib/sentry-native-cmake/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/contrib/sentry-native-cmake/CMakeLists.txt b/contrib/sentry-native-cmake/CMakeLists.txt index 377f955f856..6364e75db28 100644 --- a/contrib/sentry-native-cmake/CMakeLists.txt +++ b/contrib/sentry-native-cmake/CMakeLists.txt @@ -13,6 +13,7 @@ set (SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/sentry-native") set (SRCS ${SRC_DIR}/vendor/mpack.c + ${SRC_DIR}/vendor/stb_sprintf.c ${SRC_DIR}/src/sentry_alloc.c ${SRC_DIR}/src/sentry_backend.c ${SRC_DIR}/src/sentry_core.c @@ -21,6 +22,7 @@ set (SRCS ${SRC_DIR}/src/sentry_json.c ${SRC_DIR}/src/sentry_logger.c ${SRC_DIR}/src/sentry_options.c + ${SRC_DIR}/src/sentry_os.c ${SRC_DIR}/src/sentry_random.c ${SRC_DIR}/src/sentry_ratelimiter.c ${SRC_DIR}/src/sentry_scope.c @@ -29,6 +31,7 @@ set (SRCS ${SRC_DIR}/src/sentry_string.c ${SRC_DIR}/src/sentry_sync.c ${SRC_DIR}/src/sentry_transport.c + ${SRC_DIR}/src/sentry_tracing.c ${SRC_DIR}/src/sentry_utils.c ${SRC_DIR}/src/sentry_uuid.c ${SRC_DIR}/src/sentry_value.c From e58d2ae5d62b2d955081c5d3f55ed4e63ce61969 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 25 Nov 2023 18:36:28 +0000 Subject: [PATCH 798/813] Update version_date.tsv and changelogs after v23.10.5.20-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.10.5.20-stable.md | 28 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 +++ 5 files changed, 34 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.10.5.20-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 63de9f6c462..b174dfde675 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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index d26bb344fef..d4498abda6a 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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 53a36818121..08e95cd535b 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.10.5.20-stable.md b/docs/changelogs/v23.10.5.20-stable.md new file mode 100644 index 00000000000..03e8c47481b --- /dev/null +++ b/docs/changelogs/v23.10.5.20-stable.md @@ -0,0 +1,28 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.10.5.20-stable (e84001e5c61) FIXME as compared to v23.10.4.25-stable (330fd687d41) + +#### Improvement +* Backported in [#56924](https://github.com/ClickHouse/ClickHouse/issues/56924): There was a potential vulnerability in previous ClickHouse versions: if a user has connected and unsuccessfully tried to authenticate with the "interserver secret" method, the server didn't terminate the connection immediately but continued to receive and ignore the leftover packets from the client. While these packets are ignored, they are still parsed, and if they use a compression method with another known vulnerability, it will lead to exploitation of it without authentication. This issue was found with [ClickHouse Bug Bounty Program](https://github.com/ClickHouse/ClickHouse/issues/38986) by https://twitter.com/malacupa. [#56794](https://github.com/ClickHouse/ClickHouse/pull/56794) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Backported in [#57023](https://github.com/ClickHouse/ClickHouse/issues/57023): There was an attempt to have the proper listing in [#44311](https://github.com/ClickHouse/ClickHouse/issues/44311), but the fix itself was in the wrong place, so it's still broken. See an [example](https://github.com/ClickHouse/ClickHouse/actions/runs/6897342568/job/18781001022#step:8:25). [#56989](https://github.com/ClickHouse/ClickHouse/pull/56989) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix ON CLUSTER queries without database on initial node [#56484](https://github.com/ClickHouse/ClickHouse/pull/56484) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix buffer overflow in Gorilla codec [#57107](https://github.com/ClickHouse/ClickHouse/pull/57107) ([Nikolay Degterinsky](https://github.com/evillique)). +* Close interserver connection on any exception before authentication [#57142](https://github.com/ClickHouse/ClickHouse/pull/57142) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix client suggestions for user without grants [#56234](https://github.com/ClickHouse/ClickHouse/pull/56234) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix pygithub [#56778](https://github.com/ClickHouse/ClickHouse/pull/56778) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid dependencies with no fixed versions [#56914](https://github.com/ClickHouse/ClickHouse/pull/56914) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Tiny improvement security [#57171](https://github.com/ClickHouse/ClickHouse/pull/57171) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 014ee5e9a17..fed19917fba 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,12 +1,15 @@ +v23.10.5.20-stable 2023-11-25 v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 v23.10.2.13-stable 2023-11-08 v23.10.1.1976-stable 2023-11-02 +v23.9.6.20-stable 2023-11-25 v23.9.5.29-stable 2023-11-17 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.8.20-lts 2023-11-25 v23.8.7.24-lts 2023-11-17 v23.8.6.16-lts 2023-11-08 v23.8.5.16-lts 2023-10-31 From 2bb12386dd239686edd6faaee3ef8fcfceff2c26 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 25 Nov 2023 18:37:17 +0000 Subject: [PATCH 799/813] Update version_date.tsv and changelogs after v23.9.6.20-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.9.6.20-stable.md | 28 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 5 files changed, 35 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.9.6.20-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 63de9f6c462..b174dfde675 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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index d26bb344fef..d4498abda6a 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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 53a36818121..08e95cd535b 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.9.6.20-stable.md b/docs/changelogs/v23.9.6.20-stable.md new file mode 100644 index 00000000000..b4aed625fea --- /dev/null +++ b/docs/changelogs/v23.9.6.20-stable.md @@ -0,0 +1,28 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.9.6.20-stable (cf7e84bb8cf) FIXME as compared to v23.9.5.29-stable (f8554c1a1ff) + +#### Improvement +* Backported in [#56930](https://github.com/ClickHouse/ClickHouse/issues/56930): There was a potential vulnerability in previous ClickHouse versions: if a user has connected and unsuccessfully tried to authenticate with the "interserver secret" method, the server didn't terminate the connection immediately but continued to receive and ignore the leftover packets from the client. While these packets are ignored, they are still parsed, and if they use a compression method with another known vulnerability, it will lead to exploitation of it without authentication. This issue was found with [ClickHouse Bug Bounty Program](https://github.com/ClickHouse/ClickHouse/issues/38986) by https://twitter.com/malacupa. [#56794](https://github.com/ClickHouse/ClickHouse/pull/56794) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Backported in [#57022](https://github.com/ClickHouse/ClickHouse/issues/57022): There was an attempt to have the proper listing in [#44311](https://github.com/ClickHouse/ClickHouse/issues/44311), but the fix itself was in the wrong place, so it's still broken. See an [example](https://github.com/ClickHouse/ClickHouse/actions/runs/6897342568/job/18781001022#step:8:25). [#56989](https://github.com/ClickHouse/ClickHouse/pull/56989) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix ON CLUSTER queries without database on initial node [#56484](https://github.com/ClickHouse/ClickHouse/pull/56484) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix buffer overflow in Gorilla codec [#57107](https://github.com/ClickHouse/ClickHouse/pull/57107) ([Nikolay Degterinsky](https://github.com/evillique)). +* Close interserver connection on any exception before authentication [#57142](https://github.com/ClickHouse/ClickHouse/pull/57142) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix client suggestions for user without grants [#56234](https://github.com/ClickHouse/ClickHouse/pull/56234) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix pygithub [#56778](https://github.com/ClickHouse/ClickHouse/pull/56778) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid dependencies with no fixed versions [#56914](https://github.com/ClickHouse/ClickHouse/pull/56914) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Tiny improvement security [#57171](https://github.com/ClickHouse/ClickHouse/pull/57171) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 014ee5e9a17..ebe138d597a 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,12 +1,15 @@ +v23.10.5.20-stable 2023-11-25 v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 v23.10.2.13-stable 2023-11-08 v23.10.1.1976-stable 2023-11-02 +v23.9.6.20-stable 2023-11-25 v23.9.5.29-stable 2023-11-17 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.8.20-lts 2023-11-25 v23.8.7.24-lts 2023-11-17 v23.8.6.16-lts 2023-11-08 v23.8.5.16-lts 2023-10-31 @@ -34,6 +37,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.18.15-lts 2023-11-25 v23.3.17.13-lts 2023-11-17 v23.3.16.7-lts 2023-11-08 v23.3.15.29-lts 2023-10-31 From f78612f37eec9d45431cf7d1cf79ff3ebd907dcc Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 25 Nov 2023 18:40:50 +0000 Subject: [PATCH 800/813] Update version_date.tsv and changelogs after v23.3.18.15-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.3.18.15-lts.md | 26 ++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 5 files changed, 33 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.3.18.15-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 63de9f6c462..b174dfde675 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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index d26bb344fef..d4498abda6a 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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 53a36818121..08e95cd535b 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.3.18.15-lts.md b/docs/changelogs/v23.3.18.15-lts.md new file mode 100644 index 00000000000..3bf993a0960 --- /dev/null +++ b/docs/changelogs/v23.3.18.15-lts.md @@ -0,0 +1,26 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.18.15-lts (7228475d77a) FIXME as compared to v23.3.17.13-lts (e867d59020f) + +#### Improvement +* Backported in [#56928](https://github.com/ClickHouse/ClickHouse/issues/56928): There was a potential vulnerability in previous ClickHouse versions: if a user has connected and unsuccessfully tried to authenticate with the "interserver secret" method, the server didn't terminate the connection immediately but continued to receive and ignore the leftover packets from the client. While these packets are ignored, they are still parsed, and if they use a compression method with another known vulnerability, it will lead to exploitation of it without authentication. This issue was found with [ClickHouse Bug Bounty Program](https://github.com/ClickHouse/ClickHouse/issues/38986) by https://twitter.com/malacupa. [#56794](https://github.com/ClickHouse/ClickHouse/pull/56794) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Backported in [#57019](https://github.com/ClickHouse/ClickHouse/issues/57019): There was an attempt to have the proper listing in [#44311](https://github.com/ClickHouse/ClickHouse/issues/44311), but the fix itself was in the wrong place, so it's still broken. See an [example](https://github.com/ClickHouse/ClickHouse/actions/runs/6897342568/job/18781001022#step:8:25). [#56989](https://github.com/ClickHouse/ClickHouse/pull/56989) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix buffer overflow in Gorilla codec [#57107](https://github.com/ClickHouse/ClickHouse/pull/57107) ([Nikolay Degterinsky](https://github.com/evillique)). +* Close interserver connection on any exception before authentication [#57142](https://github.com/ClickHouse/ClickHouse/pull/57142) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix pygithub [#56778](https://github.com/ClickHouse/ClickHouse/pull/56778) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid dependencies with no fixed versions [#56914](https://github.com/ClickHouse/ClickHouse/pull/56914) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Tiny improvement security [#57171](https://github.com/ClickHouse/ClickHouse/pull/57171) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 014ee5e9a17..ebe138d597a 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,12 +1,15 @@ +v23.10.5.20-stable 2023-11-25 v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 v23.10.2.13-stable 2023-11-08 v23.10.1.1976-stable 2023-11-02 +v23.9.6.20-stable 2023-11-25 v23.9.5.29-stable 2023-11-17 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.8.20-lts 2023-11-25 v23.8.7.24-lts 2023-11-17 v23.8.6.16-lts 2023-11-08 v23.8.5.16-lts 2023-10-31 @@ -34,6 +37,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.18.15-lts 2023-11-25 v23.3.17.13-lts 2023-11-17 v23.3.16.7-lts 2023-11-08 v23.3.15.29-lts 2023-10-31 From 2e7c16e13893db29e0d69eadfea89174bd640a03 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 25 Nov 2023 19:47:32 +0000 Subject: [PATCH 801/813] Fix broken links --- .../sql-reference/functions/math-functions.md | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 9edd35ce354..b27668caf0c 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -45,7 +45,7 @@ exp(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -65,7 +65,7 @@ Alias: `ln(x)` **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -83,7 +83,7 @@ exp2(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -111,7 +111,7 @@ log2(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -129,7 +129,7 @@ exp10(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -157,7 +157,7 @@ log10(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -173,7 +173,7 @@ sqrt(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -189,7 +189,7 @@ cbrt(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -207,7 +207,7 @@ erf(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -239,7 +239,7 @@ erfc(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -257,7 +257,7 @@ lgamma(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -275,7 +275,7 @@ gamma(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -293,7 +293,7 @@ sin(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -311,7 +311,7 @@ cos(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -329,7 +329,7 @@ tan(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -347,7 +347,7 @@ asin(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -365,7 +365,7 @@ acos(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** @@ -383,7 +383,7 @@ atan(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-type/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) **Returned value** From ff34726e0c1f3a72a5b70619a88a1a666407b65c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 25 Nov 2023 19:48:29 +0000 Subject: [PATCH 802/813] Update version_date.tsv and changelogs after v23.8.8.20-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.8.8.20-lts.md | 28 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 5 files changed, 35 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.8.8.20-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 63de9f6c462..b174dfde675 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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index d26bb344fef..d4498abda6a 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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 53a36818121..08e95cd535b 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,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="23.10.4.25" +ARG VERSION="23.10.5.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.8.8.20-lts.md b/docs/changelogs/v23.8.8.20-lts.md new file mode 100644 index 00000000000..345cfcccf17 --- /dev/null +++ b/docs/changelogs/v23.8.8.20-lts.md @@ -0,0 +1,28 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.8.8.20-lts (5e012a03bf2) FIXME as compared to v23.8.7.24-lts (812b95e14ba) + +#### Improvement +* Backported in [#56509](https://github.com/ClickHouse/ClickHouse/issues/56509): Allow backup of materialized view with dropped inner table instead of failing the backup. [#56387](https://github.com/ClickHouse/ClickHouse/pull/56387) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#56929](https://github.com/ClickHouse/ClickHouse/issues/56929): There was a potential vulnerability in previous ClickHouse versions: if a user has connected and unsuccessfully tried to authenticate with the "interserver secret" method, the server didn't terminate the connection immediately but continued to receive and ignore the leftover packets from the client. While these packets are ignored, they are still parsed, and if they use a compression method with another known vulnerability, it will lead to exploitation of it without authentication. This issue was found with [ClickHouse Bug Bounty Program](https://github.com/ClickHouse/ClickHouse/issues/38986) by https://twitter.com/malacupa. [#56794](https://github.com/ClickHouse/ClickHouse/pull/56794) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Backported in [#57020](https://github.com/ClickHouse/ClickHouse/issues/57020): There was an attempt to have the proper listing in [#44311](https://github.com/ClickHouse/ClickHouse/issues/44311), but the fix itself was in the wrong place, so it's still broken. See an [example](https://github.com/ClickHouse/ClickHouse/actions/runs/6897342568/job/18781001022#step:8:25). [#56989](https://github.com/ClickHouse/ClickHouse/pull/56989) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix ON CLUSTER queries without database on initial node [#56484](https://github.com/ClickHouse/ClickHouse/pull/56484) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix buffer overflow in Gorilla codec [#57107](https://github.com/ClickHouse/ClickHouse/pull/57107) ([Nikolay Degterinsky](https://github.com/evillique)). +* Close interserver connection on any exception before authentication [#57142](https://github.com/ClickHouse/ClickHouse/pull/57142) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix pygithub [#56778](https://github.com/ClickHouse/ClickHouse/pull/56778) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid dependencies with no fixed versions [#56914](https://github.com/ClickHouse/ClickHouse/pull/56914) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Tiny improvement security [#57171](https://github.com/ClickHouse/ClickHouse/pull/57171) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 014ee5e9a17..ebe138d597a 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,12 +1,15 @@ +v23.10.5.20-stable 2023-11-25 v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 v23.10.2.13-stable 2023-11-08 v23.10.1.1976-stable 2023-11-02 +v23.9.6.20-stable 2023-11-25 v23.9.5.29-stable 2023-11-17 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.8.20-lts 2023-11-25 v23.8.7.24-lts 2023-11-17 v23.8.6.16-lts 2023-11-08 v23.8.5.16-lts 2023-10-31 @@ -34,6 +37,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.18.15-lts 2023-11-25 v23.3.17.13-lts 2023-11-17 v23.3.16.7-lts 2023-11-08 v23.3.15.29-lts 2023-10-31 From ced0bbd932059f81fda6c6dc0a8aa44a8dd8046e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Nov 2023 22:00:56 +0100 Subject: [PATCH 803/813] Change cursor style for overwrite mode (INS) to blinking in client Signed-off-by: Azat Khuzhin --- src/Client/ReplxxLineReader.cpp | 11 +++++++++++ src/Client/ReplxxLineReader.h | 1 + 2 files changed, 12 insertions(+) diff --git a/src/Client/ReplxxLineReader.cpp b/src/Client/ReplxxLineReader.cpp index 681d06ce583..8ebbbf9ce94 100644 --- a/src/Client/ReplxxLineReader.cpp +++ b/src/Client/ReplxxLineReader.cpp @@ -447,6 +447,17 @@ ReplxxLineReader::ReplxxLineReader( uint32_t reverse_search = Replxx::KEY::control('R'); return rx.invoke(Replxx::ACTION::HISTORY_INCREMENTAL_SEARCH, reverse_search); }); + + /// Change cursor style for overwrite mode to blinking (see console_codes(5)) + rx.bind_key(Replxx::KEY::INSERT, [this](char32_t) + { + overwrite_mode = !overwrite_mode; + if (overwrite_mode) + rx.print("%s", "\033[5 q"); + else + rx.print("%s", "\033[0 q"); + return rx.invoke(Replxx::ACTION::TOGGLE_OVERWRITE_MODE, 0); + }); } ReplxxLineReader::~ReplxxLineReader() diff --git a/src/Client/ReplxxLineReader.h b/src/Client/ReplxxLineReader.h index 9cc2853f77a..6ad149e38f2 100644 --- a/src/Client/ReplxxLineReader.h +++ b/src/Client/ReplxxLineReader.h @@ -41,6 +41,7 @@ private: bool bracketed_paste_enabled = false; std::string editor; + bool overwrite_mode = false; }; } From 92e580997c5fbd436f829de62d7073504380986a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Nov 2023 00:57:12 +0300 Subject: [PATCH 804/813] Update IDataType.h --- src/DataTypes/IDataType.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 81524c5a697..bfc6ff3d0ee 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -177,9 +177,9 @@ public: /// Checks that two instances belong to the same type virtual bool equals(const IDataType & rhs) const = 0; - /** Checks that two types is exactly identical. Note that two types may equals but not identical. + /** Checks that two types are exactly identical. Note that two types may be equal but not identical. * For example, `SimpleAggregateFunction(max, String)` and `SimpleAggregateFunction(anyLast, String)` - * This is used when creating replicated table. Column types in different replicas must be identical + * This is used when creating a replicated table. Column types in different replicas must be identical */ virtual bool identical(const IDataType & rhs) const; From e074629749857da0a41908be6b54b252334c7e79 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 25 Nov 2023 22:04:49 +0000 Subject: [PATCH 805/813] Docs: Mention alias 'database' for 'name' in system.databases --- docs/en/operations/system-tables/databases.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/system-tables/databases.md b/docs/en/operations/system-tables/databases.md index f3d3d388c36..e3b0ded96e8 100644 --- a/docs/en/operations/system-tables/databases.md +++ b/docs/en/operations/system-tables/databases.md @@ -14,6 +14,7 @@ Columns: - `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Database UUID. - `comment` ([String](../../sql-reference/data-types/enum.md)) — Database comment. - `engine_full` ([String](../../sql-reference/data-types/enum.md)) — Parameters of the database engine. +- `database` ([String](../../sql-reference/data-types/string.md)) – Alias for `name`. The `name` column from this system table is used for implementing the `SHOW DATABASES` query. From e60941f7c53dda4a4b3fa4bdda82495dae10b515 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Nov 2023 04:30:05 +0300 Subject: [PATCH 806/813] Revert "Update Sentry" --- contrib/sentry-native | 2 +- contrib/sentry-native-cmake/CMakeLists.txt | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/contrib/sentry-native b/contrib/sentry-native index d1f0883e020..ae10fb8c224 160000 --- a/contrib/sentry-native +++ b/contrib/sentry-native @@ -1 +1 @@ -Subproject commit d1f0883e020f5c1b182c2b36ea26ea9c6fa64e39 +Subproject commit ae10fb8c224c3f41571446e1ed7fd57b9e5e366b diff --git a/contrib/sentry-native-cmake/CMakeLists.txt b/contrib/sentry-native-cmake/CMakeLists.txt index 6364e75db28..377f955f856 100644 --- a/contrib/sentry-native-cmake/CMakeLists.txt +++ b/contrib/sentry-native-cmake/CMakeLists.txt @@ -13,7 +13,6 @@ set (SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/sentry-native") set (SRCS ${SRC_DIR}/vendor/mpack.c - ${SRC_DIR}/vendor/stb_sprintf.c ${SRC_DIR}/src/sentry_alloc.c ${SRC_DIR}/src/sentry_backend.c ${SRC_DIR}/src/sentry_core.c @@ -22,7 +21,6 @@ set (SRCS ${SRC_DIR}/src/sentry_json.c ${SRC_DIR}/src/sentry_logger.c ${SRC_DIR}/src/sentry_options.c - ${SRC_DIR}/src/sentry_os.c ${SRC_DIR}/src/sentry_random.c ${SRC_DIR}/src/sentry_ratelimiter.c ${SRC_DIR}/src/sentry_scope.c @@ -31,7 +29,6 @@ set (SRCS ${SRC_DIR}/src/sentry_string.c ${SRC_DIR}/src/sentry_sync.c ${SRC_DIR}/src/sentry_transport.c - ${SRC_DIR}/src/sentry_tracing.c ${SRC_DIR}/src/sentry_utils.c ${SRC_DIR}/src/sentry_uuid.c ${SRC_DIR}/src/sentry_value.c From b92c416ced49dc6b0657ac5502ff8e698f3f44ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Nov 2023 02:34:18 +0100 Subject: [PATCH 807/813] Remove test `01280_ttl_where_group_by` --- .../01280_ttl_where_group_by.reference | 26 ---- .../0_stateless/01280_ttl_where_group_by.sh | 120 ------------------ 2 files changed, 146 deletions(-) delete mode 100644 tests/queries/0_stateless/01280_ttl_where_group_by.reference delete mode 100755 tests/queries/0_stateless/01280_ttl_where_group_by.sh diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.reference b/tests/queries/0_stateless/01280_ttl_where_group_by.reference deleted file mode 100644 index 65e7e5b158f..00000000000 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.reference +++ /dev/null @@ -1,26 +0,0 @@ -ttl_01280_1 -1 1 0 4 -1 2 3 7 -1 3 0 5 -2 1 0 1 -2 1 20 1 -ttl_01280_2 -1 1 [0,2,3] 4 -1 1 [5,4,1] 13 -1 3 [1,0,1,0] 17 -2 1 [3,1,0,3] 8 -3 1 [2,4,5] 8 -ttl_01280_3 -1 1 0 4 -1 1 10 6 -2 1 0 3 -3 1 8 2 -ttl_01280_4 -0 4 -13 9 -ttl_01280_5 -1 2 7 5 -2 3 6 5 -ttl_01280_6 -1 3 5 -2 3 5 diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.sh b/tests/queries/0_stateless/01280_ttl_where_group_by.sh deleted file mode 100755 index e6f83d6edd1..00000000000 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.sh +++ /dev/null @@ -1,120 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-parallel, no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_1" - -function optimize() -{ - for _ in {0..20}; do - $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE $1 FINAL SETTINGS optimize_throw_if_noop=1" 2>/dev/null && break - sleep 0.3 - done -} - -# "SETTINGS max_parts_to_merge_at_once = 1" prevents merges to start before our own OPTIMIZE FINAL - -echo "ttl_01280_1" -$CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_1 (a Int, b Int, x Int, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second delete where x % 10 == 0 and y > 5 SETTINGS max_parts_to_merge_at_once = 1; -insert into ttl_01280_1 values (1, 1, 0, 4, now() + 10); -insert into ttl_01280_1 values (1, 1, 10, 6, now()); -insert into ttl_01280_1 values (1, 2, 3, 7, now()); -insert into ttl_01280_1 values (1, 3, 0, 5, now()); -insert into ttl_01280_1 values (2, 1, 20, 1, now()); -insert into ttl_01280_1 values (2, 1, 0, 1, now()); -insert into ttl_01280_1 values (3, 1, 0, 8, now());" - -sleep 2 -optimize "ttl_01280_1" -$CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_1 ORDER BY a, b, x, y" - -$CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_2" - -echo "ttl_01280_2" -$CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_2 (a Int, b Int, x Array(Int32), y Double, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set x = minForEach(x), y = sum(y), d = max(d) SETTINGS max_parts_to_merge_at_once = 1; -insert into ttl_01280_2 values (1, 1, array(0, 2, 3), 4, now() + 10); -insert into ttl_01280_2 values (1, 1, array(5, 4, 3), 6, now()); -insert into ttl_01280_2 values (1, 1, array(5, 5, 1), 7, now()); -insert into ttl_01280_2 values (1, 3, array(3, 0, 4), 5, now()); -insert into ttl_01280_2 values (1, 3, array(1, 1, 2, 1), 9, now()); -insert into ttl_01280_2 values (1, 3, array(3, 2, 1, 0), 3, now()); -insert into ttl_01280_2 values (2, 1, array(3, 3, 3), 7, now()); -insert into ttl_01280_2 values (2, 1, array(11, 1, 0, 3), 1, now()); -insert into ttl_01280_2 values (3, 1, array(2, 4, 5), 8, now());" - -sleep 2 -optimize "ttl_01280_2" -$CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_2 ORDER BY a, b, x, y" - -$CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_3" - -echo "ttl_01280_3" -$CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_3 (a Int, b Int, x Int64, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set b = min(b), x = argMax(x, d), y = argMax(y, d), d = max(d) SETTINGS max_parts_to_merge_at_once = 1; -insert into ttl_01280_3 values (1, 1, 0, 4, now() + 10); -insert into ttl_01280_3 values (1, 1, 10, 6, now() + 1); -insert into ttl_01280_3 values (1, 2, 3, 7, now()); -insert into ttl_01280_3 values (1, 3, 0, 5, now()); -insert into ttl_01280_3 values (2, 1, 20, 1, now()); -insert into ttl_01280_3 values (2, 1, 0, 3, now() + 1); -insert into ttl_01280_3 values (3, 1, 0, 3, now()); -insert into ttl_01280_3 values (3, 2, 8, 2, now() + 1); -insert into ttl_01280_3 values (3, 5, 5, 8, now());" - -sleep 2 -optimize "ttl_01280_3" -$CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_3 ORDER BY a, b, x, y" - -$CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_4" - -echo "ttl_01280_4" -$CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_4 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), -(a + b)) ttl d + interval 1 second group by toDate(d) set x = sum(x), y = max(y) SETTINGS max_parts_to_merge_at_once = 1; -insert into ttl_01280_4 values (1, 1, 0, 4, now() + 10); -insert into ttl_01280_4 values (10, 2, 3, 3, now()); -insert into ttl_01280_4 values (2, 10, 1, 7, now()); -insert into ttl_01280_4 values (3, 3, 5, 2, now()); -insert into ttl_01280_4 values (1, 5, 4, 9, now())" - -sleep 2 -optimize "ttl_01280_4" -$CLICKHOUSE_CLIENT --query "select x, y from ttl_01280_4 ORDER BY a, b, x, y" - -$CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_5" - -echo "ttl_01280_5" -$CLICKHOUSE_CLIENT -n --query "create table ttl_01280_5 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a set x = sum(x), b = argMax(b, -b) SETTINGS max_parts_to_merge_at_once = 1; -insert into ttl_01280_5 values (1, 2, 3, 5, now()); -insert into ttl_01280_5 values (2, 10, 1, 5, now()); -insert into ttl_01280_5 values (2, 3, 5, 5, now()); -insert into ttl_01280_5 values (1, 5, 4, 5, now());" - -sleep 2 -optimize "ttl_01280_5" -$CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_5 ORDER BY a, b, x, y" - -$CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_6" - -echo "ttl_01280_6" -$CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_6 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a SETTINGS max_parts_to_merge_at_once = 1; -insert into ttl_01280_6 values (1, 2, 3, 5, now()); -insert into ttl_01280_6 values (2, 10, 3, 5, now()); -insert into ttl_01280_6 values (2, 3, 3, 5, now()); -insert into ttl_01280_6 values (1, 5, 3, 5, now())" - -sleep 2 -optimize "ttl_01280_6" -$CLICKHOUSE_CLIENT --query "select a, x, y from ttl_01280_6 ORDER BY a, b, x, y" - -$CLICKHOUSE_CLIENT -q "DROP TABLE ttl_01280_1" -$CLICKHOUSE_CLIENT -q "DROP TABLE ttl_01280_2" -$CLICKHOUSE_CLIENT -q "DROP TABLE ttl_01280_3" -$CLICKHOUSE_CLIENT -q "DROP TABLE ttl_01280_4" -$CLICKHOUSE_CLIENT -q "DROP TABLE ttl_01280_5" -$CLICKHOUSE_CLIENT -q "DROP TABLE ttl_01280_6" From 32da588d5e85264ef678e05b0e3cef4751f14f8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Nov 2023 05:57:54 +0300 Subject: [PATCH 808/813] Revert "Add debugging info for 01600_parts_types_metrics on failures" --- .../01600_parts_types_metrics.reference | 3 -- .../0_stateless/01600_parts_types_metrics.sh | 53 ------------------- 2 files changed, 56 deletions(-) delete mode 100644 tests/queries/0_stateless/01600_parts_types_metrics.reference delete mode 100755 tests/queries/0_stateless/01600_parts_types_metrics.sh diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.reference b/tests/queries/0_stateless/01600_parts_types_metrics.reference deleted file mode 100644 index e8183f05f5d..00000000000 --- a/tests/queries/0_stateless/01600_parts_types_metrics.reference +++ /dev/null @@ -1,3 +0,0 @@ -1 -1 -1 diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.sh b/tests/queries/0_stateless/01600_parts_types_metrics.sh deleted file mode 100755 index d4ffbfed613..00000000000 --- a/tests/queries/0_stateless/01600_parts_types_metrics.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-s3-storage, no-asan, long, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -set -e -set -o pipefail - -# The query is not atomic - it can compare states between system.parts and system.metrics from different points in time. -# So, there is inherent race condition (especially in fasttest that runs tests in parallel). -# -# But it should get the expected result eventually. -# In case of test failure, this code will do infinite loop and timeout. -verify() -{ - for ((i = 0; i < 100; ++i)); do - # NOTE: database = $CLICKHOUSE_DATABASE is unwanted - result=$( $CLICKHOUSE_CLIENT -m --query "SELECT - (SELECT sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics) = - (SELECT countIf(part_type = 'Compact'), countIf(part_type = 'Wide') FROM (SELECT part_type FROM system.parts UNION ALL SELECT part_type FROM system.projection_parts))") - if [ "$result" = "1" ]; then - echo 1 - return - fi - sleep 0.1 - done - - echo "ERROR: metrics does not match:" >&2 - $CLICKHOUSE_CLIENT -nm --query " - -- { echo } - SELECT sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics; - SELECT countIf(part_type = 'Compact'), countIf(part_type = 'Wide') FROM (SELECT part_type FROM system.parts UNION ALL SELECT part_type FROM system.projection_parts); - " -} - -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE IF EXISTS data_01600" -# Compact - (5..10] -# Wide - >10 -$CLICKHOUSE_CLIENT --query="CREATE TABLE data_01600 (part_type String, key Int) ENGINE = MergeTree PARTITION BY part_type ORDER BY key SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=10, index_granularity = 8192, index_granularity_bytes = '10Mi'" - -# Compact -$CLICKHOUSE_CLIENT --query="INSERT INTO data_01600 SELECT 'Compact', number FROM system.numbers LIMIT 6" -verify - -# Wide -$CLICKHOUSE_CLIENT --query="INSERT INTO data_01600 SELECT 'Wide', number FROM system.numbers LIMIT 11 OFFSET 6" -verify - -# DROP and check -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE data_01600" -verify From faa3a43803a6b7ede6784fa701584272afa8dec6 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 26 Nov 2023 19:21:39 +0000 Subject: [PATCH 809/813] Fix docs --- .../table-engines/mergetree-family/mergetree.md | 12 ++++++------ .../functions/comparison-functions.md | 12 ++++++------ .../table-engines/mergetree-family/mergetree.md | 14 +++++++------- .../functions/comparison-functions.md | 12 ++++++------ .../table-engines/mergetree-family/mergetree.md | 12 ++++++------ .../functions/comparison-functions.md | 12 ++++++------ 6 files changed, 37 insertions(+), 37 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index f0bc45b9f53..9cbb48ef847 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -504,8 +504,8 @@ Indexes of type `set` can be utilized by all functions. The other index types ar | Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | inverted | |------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------|----------| -| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | | [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | | [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | | [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | @@ -513,10 +513,10 @@ Indexes of type `set` can be utilized by all functions. The other index types ar | [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ | | [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | | [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | | [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | | [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | | [has](/docs/en/sql-reference/functions/array-functions#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ | diff --git a/docs/en/sql-reference/functions/comparison-functions.md b/docs/en/sql-reference/functions/comparison-functions.md index 297d84eb8a5..abe923adeb3 100644 --- a/docs/en/sql-reference/functions/comparison-functions.md +++ b/docs/en/sql-reference/functions/comparison-functions.md @@ -20,7 +20,7 @@ Strings are compared byte-by-byte. Note that this may lead to unexpected results A string S1 which has another string S2 as prefix is considered longer than S2. -## equals, `=`, `==` operators +## equals, `=`, `==` operators {#equals} **Syntax** @@ -32,7 +32,7 @@ Alias: - `a = b` (operator) - `a == b` (operator) -## notEquals, `!=`, `<>` operators +## notEquals, `!=`, `<>` operators {#notequals} **Syntax** @@ -44,7 +44,7 @@ Alias: - `a != b` (operator) - `a <> b` (operator) -## less, `<` operator +## less, `<` operator {#less} **Syntax** @@ -55,7 +55,7 @@ less(a, b) Alias: - `a < b` (operator) -## greater, `>` operator +## greater, `>` operator {#greater} **Syntax** @@ -66,7 +66,7 @@ greater(a, b) Alias: - `a > b` (operator) -## lessOrEquals, `<=` operator +## lessOrEquals, `<=` operator {#lessorequals} **Syntax** @@ -77,7 +77,7 @@ lessOrEquals(a, b) Alias: - `a <= b` (operator) -## greaterOrEquals, `>=` operator +## greaterOrEquals, `>=` operator {#greaterorequals} **Syntax** diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 00eb830c9ef..7195ee38af6 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -337,7 +337,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 Поддерживаемые типы данных: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`. - Фильтром могут пользоваться функции: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md#hasarr-elem), [hasAny](../../../sql-reference/functions/array-functions.md#hasany), [hasAll](../../../sql-reference/functions/array-functions.md#hasall). + Фильтром могут пользоваться функции: [equals](../../../sql-reference/functions/comparison-functions.md#equals), [notEquals](../../../sql-reference/functions/comparison-functions.md#notequals), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md#hasarr-elem), [hasAny](../../../sql-reference/functions/array-functions.md#hasany), [hasAll](../../../sql-reference/functions/array-functions.md#hasall). **Примеры** @@ -354,8 +354,8 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT | Функция (оператор) / Индекс | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | |------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| -| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](../../../sql-reference/functions/comparison-functions.md#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | | [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | | [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | @@ -363,10 +363,10 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT | [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | | [in](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notIn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (\<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greater (\>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [less (\<)](../../../sql-reference/functions/comparison-functions.md#less) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greater (\>)](../../../sql-reference/functions/comparison-functions.md#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | | [empty](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | | [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | | hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | diff --git a/docs/ru/sql-reference/functions/comparison-functions.md b/docs/ru/sql-reference/functions/comparison-functions.md index f66b42977cc..bb9322d5a82 100644 --- a/docs/ru/sql-reference/functions/comparison-functions.md +++ b/docs/ru/sql-reference/functions/comparison-functions.md @@ -23,14 +23,14 @@ sidebar_label: "Функции сравнения" Замечание. До версии 1.1.54134 сравнение знаковых и беззнаковых целых чисел производилось также, как в C++. То есть, вы могли получить неверный результат в таких случаях: SELECT 9223372036854775807 \> -1. С версии 1.1.54134 поведение изменилось и стало математически корректным. -## equals, оператор a = b и a == b {#function-equals} +## equals, оператор a = b и a == b {#equals} -## notEquals, оператор a != b и a `<>` b {#function-notequals} +## notEquals, оператор a != b и a `<>` b {#notequals} -## less, оператор `<` {#function-less} +## less, оператор `<` {#less} -## greater, оператор `>` {#function-greater} +## greater, оператор `>` {#greater} -## lessOrEquals, оператор `<=` {#function-lessorequals} +## lessOrEquals, оператор `<=` {#lessorequals} -## greaterOrEquals, оператор `>=` {#function-greaterorequals} +## greaterOrEquals, оператор `>=` {#greaterorequals} diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index cec4cb09047..57c49d88732 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -349,8 +349,8 @@ WHERE 子句中的条件可以包含对某列数据进行运算的函数表达 | 函数 (操作符) / 索引 | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | | ------------------------------------------------------------ | ----------- | ------ | ---------- | ---------- | ------------ | -| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](../../../sql-reference/functions/comparison-functions.md#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✗ | ✗ | ✗ | | [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | @@ -358,10 +358,10 @@ WHERE 子句中的条件可以包含对某列数据进行运算的函数表达 | [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | | [in](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notIn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (\<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greater (\>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [less (\<)](../../../sql-reference/functions/comparison-functions.md#less) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greater (\>)](../../../sql-reference/functions/comparison-functions.md#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | | [empty](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | | [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | | hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | diff --git a/docs/zh/sql-reference/functions/comparison-functions.md b/docs/zh/sql-reference/functions/comparison-functions.md index ef3adf427f1..ed33dc40531 100644 --- a/docs/zh/sql-reference/functions/comparison-functions.md +++ b/docs/zh/sql-reference/functions/comparison-functions.md @@ -21,14 +21,14 @@ sidebar_label: 比较函数 字符串按字节进行比较。较短的字符串小于以其开头并且至少包含一个字符的所有字符串。 -## 等于,a=b和a==b 运算符 {#equals-a-b-and-a-b-operator} +## 等于,a=b和a==b 运算符 {#equals} -## 不等于,a!=b和a<>b 运算符 {#notequals-a-operator-b-and-a-b} +## 不等于,a!=b和a<>b 运算符 {#notequals} -## 少, < 运算符 {#less-operator} +## 少, < 运算符 {#less} -## 大于, > 运算符 {#greater-operator} +## 大于, > 运算符 {#greater} -## 小于等于, <= 运算符 {#lessorequals-operator} +## 小于等于, <= 运算符 {#lessorequals} -## 大于等于, >= 运算符 {#greaterorequals-operator} +## 大于等于, >= 运算符 {#greaterorequals} From dd49b46ca50d8b338accdbd36238817ec3aa52dc Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 26 Nov 2023 22:32:11 +0000 Subject: [PATCH 810/813] Fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 5d11185ff76..4077802460a 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 2633 +personal_ws-1.1 en 2643 AArch ACLs ALTERs @@ -261,6 +261,7 @@ FOSDEM FQDN Failover FarmHash +FileLog FilesystemCacheBytes FilesystemCacheElements FilesystemCacheFiles @@ -278,7 +279,6 @@ FilesystemMainPathTotalBytes FilesystemMainPathTotalINodes FilesystemMainPathUsedBytes FilesystemMainPathUsedINodes -FileLog FixedString Flink ForEach @@ -571,13 +571,13 @@ NetworkSendPackets NodeJs NuRaft NumHexagons +NumPy NumToString NumToStringClassC NumberOfDatabases NumberOfDetachedByUserParts NumberOfDetachedParts NumberOfTables -NumPy OFNS OLAP OLTP @@ -588,10 +588,10 @@ OSGuestNiceTimeNormalized OSGuestTime OSGuestTimeCPU OSGuestTimeNormalized +OSIOWaitMicroseconds OSIOWaitTime OSIOWaitTimeCPU OSIOWaitTimeNormalized -OSIOWaitMicroseconds OSIdleTime OSIdleTimeCPU OSIdleTimeNormalized @@ -1470,12 +1470,12 @@ fastops fcoverage fibonacci fifo +filelog filesystem filesystemAvailable filesystemCapacity filesystemFree filesystems -filelog finalizeAggregation fips firstLine From 124fae7549597b7b8b6651b4e79aab4a3de6ba18 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 26 Nov 2023 23:58:34 +0000 Subject: [PATCH 811/813] Fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 4077802460a..49afc562ae8 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 2643 +personal_ws-1.1 en 2646 AArch ACLs ALTERs @@ -1554,6 +1554,7 @@ graphql greatCircleAngle greatCircleDistance greaterOrEquals +greaterorequals greenspace groupArray groupArrayInsertAt @@ -1739,6 +1740,7 @@ lemmatize lemmatized lengthUTF lessOrEquals +lessorequals levenshtein levenshteinDistance lexicographically @@ -1917,6 +1919,7 @@ notEquals notILike notIn notLike +notequals notretry nowInBlock ntile From deea84cac14c14002abe57efadab8a71788a5e10 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 27 Nov 2023 10:11:10 +0100 Subject: [PATCH 812/813] Adjust a test so it's a valid query --- tests/queries/0_stateless/02923_explain_expired_context.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02923_explain_expired_context.sql b/tests/queries/0_stateless/02923_explain_expired_context.sql index fa365004cb0..68277508eb2 100644 --- a/tests/queries/0_stateless/02923_explain_expired_context.sql +++ b/tests/queries/0_stateless/02923_explain_expired_context.sql @@ -1,3 +1,3 @@ -- https://github.com/ClickHouse/ClickHouse/issues/51321 -EXPLAIN ESTIMATE SELECT any(toTypeName(s)) FROM (SELECT 'bbbbbbbb', toTypeName(s), CAST('', 'LowCardinality(String)'), NULL, CAST('\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0', 'String') AS s WITH TOTALS) AS t1 FULL OUTER JOIN (SELECT CAST('bbbbb\0\0bbb\0bb\0bb', 'LowCardinality(String)'), CAST(CAST('a', 'String'), 'LowCardinality(String)') AS s GROUP BY CoNnEcTiOn_Id()) AS t2 USING (s) WITH TOTALS; +EXPLAIN ESTIMATE SELECT any(toTypeName(s)) FROM (SELECT 'bbbbbbbb', toTypeName(s), CAST('', 'LowCardinality(String)'), NULL, CAST('\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0', 'String') AS s) AS t1 FULL OUTER JOIN (SELECT CAST('bbbbb\0\0bbb\0bb\0bb', 'LowCardinality(String)'), CAST(CAST('a', 'String'), 'LowCardinality(String)') AS s GROUP BY CoNnEcTiOn_Id()) AS t2 USING (s) WITH TOTALS; EXPLAIN ESTIMATE SELECT any(s) FROM (SELECT '' AS s) AS t1 JOIN (SELECT '' AS s GROUP BY connection_id()) AS t2 USING (s); From b26767d5bb0e5b0094c7c227bd71c41d0c722608 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Nov 2023 13:43:40 +0100 Subject: [PATCH 813/813] Add some additional groups to CI --- tests/ci/lambda_shared_package/lambda_shared/__init__.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/ci/lambda_shared_package/lambda_shared/__init__.py b/tests/ci/lambda_shared_package/lambda_shared/__init__.py index aa88342fcc3..9e6c5dde298 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/__init__.py +++ b/tests/ci/lambda_shared_package/lambda_shared/__init__.py @@ -19,6 +19,12 @@ RUNNER_TYPE_LABELS = [ "stress-tester", "style-checker", "style-checker-aarch64", + # private runners + "private-style-checker", + "private-builder", + "private-func-tester", + "private-fuzzer-unit-tester", + "private-stress-tester", ]