From e51bee58ecf8f684abfb0f1f9ac1806fb47b5efd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Apr 2022 22:32:45 +0200 Subject: [PATCH 0001/1190] 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 0002/1190] 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 0003/1190] 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 0004/1190] 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 8ad8e32f232106a242ff6e9c4b65c720be3e284c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Feb 2023 05:26:38 +0100 Subject: [PATCH 0005/1190] Tune the concurrency settings --- programs/server/config.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 85cb299e188..3787415c553 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -307,10 +307,10 @@ Query can upscale to desired number of threads during execution if more threads become available. --> 0 - 0 + 2 - 100 + 1000 + 0 + diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py index f758c3629b3..e364ec3fe4d 100644 --- a/tests/integration/test_overcommit_tracker/test.py +++ b/tests/integration/test_overcommit_tracker/test.py @@ -6,6 +6,9 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", + main_configs=[ + "configs/config.d/config.xml" + ], user_configs=[ "configs/users.d/users.xml", ], From 8f310fb1525479d9cffc1a05a7bd64e8211e0922 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 21:44:01 +0100 Subject: [PATCH 0016/1190] Update a test --- tests/queries/0_stateless/00963_achimbab.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00963_achimbab.sql b/tests/queries/0_stateless/00963_achimbab.sql index 758ecf5acf3..60fd1911e85 100644 --- a/tests/queries/0_stateless/00963_achimbab.sql +++ b/tests/queries/0_stateless/00963_achimbab.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel, long + SET output_format_write_statistics = 0; select From e2fc2e31ca7d8dfc9cd078118da1e4f7eb6a1d75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 21:48:21 +0100 Subject: [PATCH 0017/1190] Fix typo --- .../0_stateless/01605_adaptive_granularity_block_borders.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index ca7d0f3c950..340950f16fe 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -5,8 +5,8 @@ SET allow_prefetched_read_pool_for_remote_filesystem=0; DROP TABLE IF EXISTS adaptive_table; ---- If granularity of consequent blocks differs a lot, then adaptive ---- granularity will adjust amout of marks correctly. Data for test empirically +--- If the granularity of consequent blocks differs a lot, then adaptive +--- granularity will adjust the amount of marks correctly. Data for test empirically --- derived, it's quite hard to get good parameters. CREATE TABLE adaptive_table( From 17a0943b30e0a42b395c499f70f7934d920fed8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 17 Mar 2023 21:49:02 +0100 Subject: [PATCH 0018/1190] Update a test --- tests/queries/0_stateless/01091_num_threads.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01091_num_threads.sql b/tests/queries/0_stateless/01091_num_threads.sql index 0d2a66a8c2e..9fc82b470c9 100644 --- a/tests/queries/0_stateless/01091_num_threads.sql +++ b/tests/queries/0_stateless/01091_num_threads.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + set log_queries=1; set log_query_threads=1; set max_threads=0; From 0be8cd5bce1488cb220f4ab64bf7fadd69ca087a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 17 Mar 2023 21:03:25 +0000 Subject: [PATCH 0019/1190] Automatic style fix --- tests/integration/test_overcommit_tracker/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py index e364ec3fe4d..a2ddbbda9fe 100644 --- a/tests/integration/test_overcommit_tracker/test.py +++ b/tests/integration/test_overcommit_tracker/test.py @@ -6,9 +6,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", - main_configs=[ - "configs/config.d/config.xml" - ], + main_configs=["configs/config.d/config.xml"], user_configs=[ "configs/users.d/users.xml", ], From b680b1ac1ed8623e5dc57707a5fec74726f0c9fb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Jun 2023 23:58:49 +0000 Subject: [PATCH 0020/1190] 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 0021/1190] 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 0022/1190] 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 0023/1190] 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 0024/1190] [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 0025/1190] 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 0026/1190] 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 0027/1190] 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 0028/1190] 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 0029/1190] 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 ca07275143ac391d0fcb9d7fe80746e263436c36 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 24 Sep 2023 11:38:08 +0200 Subject: [PATCH 0030/1190] Support compression for keeper protocol --- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 4 + src/Common/ZooKeeper/ZooKeeperArgs.h | 1 + src/Common/ZooKeeper/ZooKeeperConstants.h | 1 + src/Common/ZooKeeper/ZooKeeperIO.h | 1 + src/Common/ZooKeeper/ZooKeeperImpl.cpp | 102 ++++++++++++++++------ src/Common/ZooKeeper/ZooKeeperImpl.h | 11 ++- src/Server/KeeperTCPHandler.cpp | 52 ++++++++--- src/Server/KeeperTCPHandler.h | 8 +- tests/config/config.d/zookeeper.xml | 1 + 9 files changed, 137 insertions(+), 44 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 5d01294e9b0..eea026b0346 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -212,6 +212,10 @@ void ZooKeeperArgs::initFromKeeperSection(const Poco::Util::AbstractConfiguratio .max_sec = config.getUInt(config_name + "." + key + ".max"), }; } + else if (key == "compressed_protocol") + { + compressed_protocol = config.getBool(config_name + "." + key); + } else throw KeeperException(Coordination::Error::ZBADARGUMENTS, "Unknown key {} in config file", key); } diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h index 9c48de02f61..4f68288a1c7 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.h +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -44,6 +44,7 @@ struct ZooKeeperArgs double recv_sleep_probability = 0.0; UInt64 send_sleep_ms = 0; UInt64 recv_sleep_ms = 0; + bool compressed_protocol = false; SessionLifetimeConfiguration fallback_session_lifetime = {}; DB::GetPriorityForLoadBalancing get_priority_load_balancing; diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index a773fbbab74..e2dcadfbbc9 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -45,6 +45,7 @@ enum class OpNum : int32_t OpNum getOpNum(int32_t raw_op_num); static constexpr int32_t ZOOKEEPER_PROTOCOL_VERSION = 0; +static constexpr int32_t ZOOKEEPER_PROTOCOL_VERSION_WITH_COMPRESSION = 1; static constexpr int32_t KEEPER_PROTOCOL_VERSION_CONNECTION_REJECT = 42; static constexpr int32_t CLIENT_HANDSHAKE_LENGTH = 44; static constexpr int32_t CLIENT_HANDSHAKE_LENGTH_WITH_READONLY = 45; diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index 83973c1ae22..b1cfce3f621 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace Coordination diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 5220aba1d27..f2e07b175cf 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -16,6 +16,9 @@ #include #include #include +#include +#include +#include #include "Coordination/KeeperConstants.h" #include "config.h" @@ -274,13 +277,13 @@ using namespace DB; template void ZooKeeper::write(const T & x) { - Coordination::write(x, *out); + Coordination::write(x, *maybe_compressed_out); } template void ZooKeeper::read(T & x) { - Coordination::read(x, *in); + Coordination::read(x, *maybe_compressed_in); } static void removeRootPath(String & path, const String & chroot) @@ -345,7 +348,21 @@ ZooKeeper::ZooKeeper( if (args.enable_fault_injections_during_startup) setupFaultDistributions(); - connect(nodes, args.connection_timeout_ms * 1000); + try + { + use_compression = args.compressed_protocol; + connect(nodes, args.connection_timeout_ms * 1000); + } + catch (...) + { + if (use_compression) + { + use_compression = false; + connect(nodes, args.connection_timeout_ms * 1000); + } + else + throw; + } if (!args.auth_scheme.empty()) sendAuth(args.auth_scheme, args.identity); @@ -422,8 +439,10 @@ void ZooKeeper::connect( socket.setSendTimeout(args.operation_timeout_ms * 1000); socket.setNoDelay(true); - in.emplace(socket); - out.emplace(socket); + in = std::make_shared(socket); + out = std::make_shared(socket); + maybe_compressed_in = in; + maybe_compressed_out = out; try { @@ -444,7 +463,15 @@ void ZooKeeper::connect( e.addMessage("while receiving handshake from ZooKeeper"); throw; } + connected = true; + if (use_compression) + { + maybe_compressed_in = std::make_shared(*in); + maybe_compressed_out = std::make_shared(*out, + CompressionCodecFactory::instance().get( + "ZSTD", {})); + } if (connected_callback.has_value()) (*connected_callback)(i, node); @@ -513,16 +540,19 @@ void ZooKeeper::sendHandshake() std::array passwd {}; write(handshake_length); - write(ZOOKEEPER_PROTOCOL_VERSION); + if (use_compression) + write(ZOOKEEPER_PROTOCOL_VERSION_WITH_COMPRESSION); + else + write(ZOOKEEPER_PROTOCOL_VERSION); write(last_zxid_seen); write(timeout); write(previous_session_id); write(passwd); + maybe_compressed_out->next(); out->next(); } - void ZooKeeper::receiveHandshake() { int32_t handshake_length; @@ -535,18 +565,22 @@ void ZooKeeper::receiveHandshake() throw Exception(Error::ZMARSHALLINGERROR, "Unexpected handshake length received: {}", handshake_length); read(protocol_version_read); - if (protocol_version_read != ZOOKEEPER_PROTOCOL_VERSION) + + /// Special way to tell a client that server is not ready to serve it. + /// It's better for faster failover than just connection drop. + /// Implemented in clickhouse-keeper. + if (protocol_version_read == KEEPER_PROTOCOL_VERSION_CONNECTION_REJECT) + throw Exception::fromMessage(Error::ZCONNECTIONLOSS, + "Keeper server rejected the connection during the handshake. " + "Possibly it's overloaded, doesn't see leader or stale"); + + if (use_compression) { - /// Special way to tell a client that server is not ready to serve it. - /// It's better for faster failover than just connection drop. - /// Implemented in clickhouse-keeper. - if (protocol_version_read == KEEPER_PROTOCOL_VERSION_CONNECTION_REJECT) - throw Exception::fromMessage(Error::ZCONNECTIONLOSS, - "Keeper server rejected the connection during the handshake. " - "Possibly it's overloaded, doesn't see leader or stale"); - else - throw Exception(Error::ZMARSHALLINGERROR, "Unexpected protocol version: {}", protocol_version_read); + if (protocol_version_read != ZOOKEEPER_PROTOCOL_VERSION_WITH_COMPRESSION) + throw Exception(Error::ZMARSHALLINGERROR,"Unexpected protocol version with compression: {}", protocol_version_read); } + else if (protocol_version_read != ZOOKEEPER_PROTOCOL_VERSION) + throw Exception(Error::ZMARSHALLINGERROR, "Unexpected protocol version: {}", protocol_version_read); read(timeout); if (timeout != args.session_timeout_ms) @@ -564,7 +598,9 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) request.scheme = scheme; request.data = data; request.xid = AUTH_XID; - request.write(*out); + request.write(*maybe_compressed_out); + maybe_compressed_out->next(); + out->next(); int32_t length; XID read_xid; @@ -580,10 +616,14 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) if (read_xid != AUTH_XID) throw Exception(Error::ZMARSHALLINGERROR, "Unexpected event received in reply to auth request: {}", read_xid); - int32_t actual_length = static_cast(in->count() - count_before_event); - if (length != actual_length) + if (!use_compression) + { + int32_t actual_length = static_cast(in->count() - count_before_event); + if (length != actual_length) throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length); + } + if (err != Error::ZOK) throw Exception(Error::ZMARSHALLINGERROR, "Error received in reply to auth request. Code: {}. Message: {}", static_cast(err), err); @@ -639,7 +679,10 @@ void ZooKeeper::sendThread() info.request->addRootPath(args.chroot); info.request->probably_sent = true; - info.request->write(*out); + info.request->write(*maybe_compressed_out); + + maybe_compressed_out->next(); + out->next(); logOperationIfNeeded(info.request); @@ -655,7 +698,9 @@ void ZooKeeper::sendThread() ZooKeeperHeartbeatRequest request; request.xid = PING_XID; - request.write(*out); + request.write(*maybe_compressed_out); + maybe_compressed_out->next(); + out->next(); } ProfileEvents::increment(ProfileEvents::ZooKeeperBytesSent, out->count() - prev_bytes_sent); @@ -827,7 +872,7 @@ void ZooKeeper::receiveEvent() } else { - response->readImpl(*in); + response->readImpl(*maybe_compressed_in); response->removeRootPath(args.chroot); } /// Instead of setting the watch in sendEvent, set it in receiveEvent because need to check the response. @@ -860,9 +905,14 @@ void ZooKeeper::receiveEvent() } } - int32_t actual_length = static_cast(in->count() - count_before_event); - if (length != actual_length) - throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length); + if (!use_compression) + { + int32_t actual_length = static_cast(in->count() - count_before_event); + + if (length != actual_length) + throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", + length, actual_length); + } logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_ms); } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 56e199352e9..bde9fcbd79a 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include #include @@ -236,8 +238,13 @@ private: Poco::Net::StreamSocket socket; /// To avoid excessive getpeername(2) calls. Poco::Net::SocketAddress socket_address; - std::optional in; - std::optional out; + + std::shared_ptr in; + std::shared_ptr out; + std::shared_ptr maybe_compressed_in; + std::shared_ptr maybe_compressed_out; + + bool use_compression = false; int64_t session_id = 0; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 58d227a5ae5..15f366df993 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include @@ -242,12 +243,15 @@ KeeperTCPHandler::KeeperTCPHandler( KeeperTCPHandler::registerConnection(this); } -void KeeperTCPHandler::sendHandshake(bool has_leader) +void KeeperTCPHandler::sendHandshake(bool has_leader, bool & use_compression) { Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out); if (has_leader) { - Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); + if (use_compression) + Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION_WITH_COMPRESSION, *out); + else + Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); } else { @@ -269,7 +273,7 @@ void KeeperTCPHandler::run() runImpl(); } -Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length) +Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length, bool & use_compression) { int32_t protocol_version; int64_t last_zxid_seen; @@ -282,9 +286,11 @@ Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length) Coordination::read(protocol_version, *in); - if (protocol_version != Coordination::ZOOKEEPER_PROTOCOL_VERSION) + if (protocol_version != Coordination::ZOOKEEPER_PROTOCOL_VERSION && protocol_version != Coordination::ZOOKEEPER_PROTOCOL_VERSION_WITH_COMPRESSION) throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected protocol version: {}", toString(protocol_version)); + use_compression = (protocol_version == Coordination::ZOOKEEPER_PROTOCOL_VERSION_WITH_COMPRESSION); + Coordination::read(last_zxid_seen, *in); Coordination::read(timeout_ms, *in); @@ -312,6 +318,8 @@ void KeeperTCPHandler::runImpl() in = std::make_shared(socket()); out = std::make_shared(socket()); + bool use_compression = false; + if (in->eof()) { LOG_WARNING(log, "Client has not sent any data."); @@ -343,7 +351,7 @@ void KeeperTCPHandler::runImpl() try { int32_t handshake_length = header; - auto client_timeout = receiveHandshake(handshake_length); + auto client_timeout = receiveHandshake(handshake_length, use_compression); if (client_timeout.totalMilliseconds() == 0) client_timeout = Poco::Timespan(Coordination::DEFAULT_SESSION_TIMEOUT_MS * Poco::Timespan::MILLISECONDS); @@ -367,20 +375,33 @@ void KeeperTCPHandler::runImpl() catch (const Exception & e) { LOG_WARNING(log, "Cannot receive session id {}", e.displayText()); - sendHandshake(false); + sendHandshake(/* has_leader */ false, use_compression); return; } - sendHandshake(true); + sendHandshake(/* has_leader */ true, use_compression); } else { LOG_WARNING(log, "Ignoring user request, because the server is not active yet"); - sendHandshake(false); + sendHandshake(/* has_leader */ false, use_compression); return; } + if (use_compression) + { + maybe_compressed_in = std::make_shared(*in); + maybe_compressed_out = std::make_shared(*out, + CompressionCodecFactory::instance().get("ZSTD", + {})); + } + else + { + maybe_compressed_in = in; + maybe_compressed_out = out; + } + auto response_fd = poll_wrapper->getResponseFD(); auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response) { @@ -467,7 +488,9 @@ void KeeperTCPHandler::runImpl() updateStats(response); packageSent(); - response->write(*out); + response->write(*maybe_compressed_out); + maybe_compressed_out->next(); + out->next(); log_long_operation("Sending response"); if (response->error == Coordination::Error::ZSESSIONEXPIRED) { @@ -525,7 +548,8 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(int32_t command) try { String res = command_ptr->run(); - out->write(res.data(), res.size()); + maybe_compressed_out->write(res.data(),res.size()); + maybe_compressed_out->next(); out->next(); } catch (...) @@ -540,16 +564,16 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(int32_t command) std::pair KeeperTCPHandler::receiveRequest() { int32_t length; - Coordination::read(length, *in); + Coordination::read(length, *maybe_compressed_in); int32_t xid; - Coordination::read(xid, *in); + Coordination::read(xid, *maybe_compressed_in); Coordination::OpNum opnum; - Coordination::read(opnum, *in); + Coordination::read(opnum, *maybe_compressed_in); Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); request->xid = xid; - request->readImpl(*in); + request->readImpl(*maybe_compressed_in); if (!keeper_dispatcher->putRequest(request, session_id)) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id); diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index ffdd50b805a..8bb8637ebcc 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -17,6 +17,8 @@ #include #include #include +#include +#include namespace DB { @@ -80,13 +82,15 @@ private: /// Streams for reading/writing from/to client connection socket. std::shared_ptr in; std::shared_ptr out; + std::shared_ptr maybe_compressed_in; + std::shared_ptr maybe_compressed_out; std::atomic connected{false}; void runImpl(); - void sendHandshake(bool has_leader); - Poco::Timespan receiveHandshake(int32_t handshake_length); + void sendHandshake(bool has_leader, bool & use_compression); + Poco::Timespan receiveHandshake(int32_t handshake_length, bool & use_compression); static bool isHandShake(int32_t handshake_length); bool tryExecuteFourLetterWordCmd(int32_t command); diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index 75b4a00fe67..8d75a0c4473 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -2,6 +2,7 @@ random + true localhost 9181 From 060724b75e91b44a7b6be98668b00ab55a4df3fe Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 24 Sep 2023 12:02:48 +0200 Subject: [PATCH 0031/1190] Added docs --- docs/en/operations/server-configuration-parameters/settings.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5586dbafd88..6e3cf1e28d2 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2406,6 +2406,8 @@ This section contains the following parameters: * nearest_hostname - selects a ZooKeeper node with a hostname that is most similar to the server’s hostname. * first_or_random - selects the first ZooKeeper node, if it's not available then randomly selects one of remaining ZooKeeper nodes. * round_robin - selects the first ZooKeeper node, if reconnection happens selects the next. +- `compressed_protocol` — If set to true, enables compression in keeper protocol. + **Example configuration** From 9c8815386aaeb35241601e502984647be6371f6a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 25 Sep 2023 14:48:09 +0200 Subject: [PATCH 0032/1190] Fixed tryExecuteFourLetterWordCmd --- src/Server/KeeperTCPHandler.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 89b87f9711c..5b7ab44d6f9 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -548,8 +548,7 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(int32_t command) try { String res = command_ptr->run(); - maybe_compressed_out->write(res.data(),res.size()); - maybe_compressed_out->next(); + out->write(res.data(),res.size()); out->next(); } catch (...) From b7cfc4d82d8949a174dc3c08390f65d0531c748a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 25 Sep 2023 15:34:00 +0000 Subject: [PATCH 0033/1190] 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 0034/1190] 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 0035/1190] 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 8d19566c7da606e2e724cde53c9ff2c7a35fdfc0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 19 Sep 2023 13:27:31 +0200 Subject: [PATCH 0036/1190] impl --- src/Storages/StorageMerge.cpp | 398 +++++++++++++++++++++------------- src/Storages/StorageMerge.h | 38 +++- 2 files changed, 277 insertions(+), 159 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 272f35303bd..2fa9e38a816 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,48 +1,54 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include #include #include -#include -#include -#include -#include -#include -#include +#include +#include #include -#include -#include -#include "DataTypes/IDataType.h" -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include #include +#include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include -#include - +#include +#include +#include +#include namespace { @@ -398,6 +404,7 @@ ReadFromMerge::ReadFromMerge( , context(std::move(context_)) , common_processed_stage(processed_stage) { + createChildPlans(); } void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) @@ -408,6 +415,65 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu return; } + QueryPlanResourceHolder resources; + std::vector> pipelines; + + chassert(selected_tables.size() == child_plans.size()); + chassert(selected_tables.size() == table_aliases.size()); + auto table_it = selected_tables.begin(); + for (size_t i = 0; i < selected_tables.size(); ++i, ++table_it) + { + auto & plan = child_plans.at(i); + const auto & table = *table_it; + + const auto storage = std::get<1>(table); + const auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); + const auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, context); + + auto modified_query_info = getModifiedQueryInfo(query_info, context, table, nested_storage_snaphsot); + + auto source_pipeline = createSources( + plan, nested_storage_snaphsot, modified_query_info, common_processed_stage, common_header, table_aliases.at(i), table, context); + + if (source_pipeline && source_pipeline->initialized()) + { + resources.storage_holders.push_back(std::get<1>(table)); + resources.table_locks.push_back(std::get<2>(table)); + + pipelines.emplace_back(std::move(source_pipeline)); + } + } + + if (pipelines.empty()) + { + pipeline.init(Pipe(std::make_shared(output_stream->header))); + return; + } + + pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines)); + + if (!query_info.input_order_info) + { + size_t tables_count = selected_tables.size(); + Float64 num_streams_multiplier = std::min( + static_cast(tables_count), + std::max(1UL, static_cast(context->getSettingsRef().max_streams_multiplier_for_merge_tables))); + size_t num_streams = static_cast(requested_num_streams * num_streams_multiplier); + + // It's possible to have many tables read from merge, resize(num_streams) might open too many files at the same time. + // Using narrowPipe instead. But in case of reading in order of primary key, we cannot do it, + // because narrowPipe doesn't preserve order. + pipeline.narrow(num_streams); + } + + pipeline.addResources(std::move(resources)); +} + +void ReadFromMerge::createChildPlans() +{ + if (selected_tables.empty()) + return; + size_t tables_count = selected_tables.size(); Float64 num_streams_multiplier = std::min(static_cast(tables_count), std::max(1UL, static_cast(context->getSettingsRef().max_streams_multiplier_for_merge_tables))); @@ -438,11 +504,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; - for (const auto & table : selected_tables) { size_t current_need_streams = tables_count >= num_streams ? 1 : (num_streams / tables_count); @@ -460,7 +521,7 @@ 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"); - Aliases aliases; + auto & aliases = table_aliases.emplace_back(); auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, context); @@ -479,6 +540,8 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu ASTPtr required_columns_expr_list = std::make_shared(); ASTPtr column_expr; + auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); + for (const auto & column : column_names) { const auto column_default = storage_columns.getDefault(column); @@ -515,42 +578,18 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu } } - auto source_pipeline = createSources( + child_plans.emplace_back(createPlanForTable( 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, context, - current_streams); - - if (source_pipeline && source_pipeline->initialized()) - { - resources.storage_holders.push_back(std::get<1>(table)); - resources.table_locks.push_back(std::get<2>(table)); - - pipelines.emplace_back(std::move(source_pipeline)); - } + current_streams)); } - if (pipelines.empty()) - { - pipeline.init(Pipe(std::make_shared(output_stream->header))); - return; - } - - pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines)); - - if (!query_info.input_order_info) - // It's possible to have many tables read from merge, resize(num_streams) might open too many files at the same time. - // Using narrowPipe instead. But in case of reading in order of primary key, we cannot do it, - // because narrowPipe doesn't preserve order. - pipeline.narrow(num_streams); - - pipeline.addResources(std::move(resources)); + applyFilters(); } SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & query_info, @@ -616,23 +655,119 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer return modified_query_info; } +template +bool recursivelyApplyToReadingSteps(QueryPlan::Node * node, const Func & func) +{ + bool ok = true; + for (auto * child : node->children) + ok &= recursivelyApplyToReadingSteps(child, func); + + if (!ok) + return false; + + if (auto * read_from_merge_tree = typeid_cast(node->step.get())) + ok &= func(*read_from_merge_tree); + + return ok; +} + QueryPipelineBuilderPtr ReadFromMerge::createSources( + QueryPlan & plan, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & modified_query_info, const QueryProcessingStage::Enum & processed_stage, - const UInt64 max_block_size, const Block & header, const Aliases & aliases, const StorageWithLockAndName & storage_with_lock, + ContextMutablePtr modified_context, + bool concat_streams) const +{ + if (!plan.isInitialized()) + return std::make_unique(); + + QueryPipelineBuilderPtr builder; + + const auto & [database_name, storage, _, table_name] = storage_with_lock; + bool allow_experimental_analyzer = modified_context->getSettingsRef().allow_experimental_analyzer; + auto storage_stage + = storage->getQueryProcessingStage(modified_context, QueryProcessingStage::Complete, storage_snapshot, modified_query_info); + + builder = plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(modified_context), BuildQueryPipelineSettings::fromContext(modified_context)); + + if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) + { + /** Materialization is needed, since from distributed storage the constants come materialized. + * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, + * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. + */ + builder->addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); + } + + if (builder->initialized()) + { + if (concat_streams && builder->getNumStreams() > 1) + { + // It's possible to have many tables read from merge, resize(1) might open too many files at the same time. + // Using concat instead. + builder->addTransform(std::make_shared(builder->getHeader(), builder->getNumStreams())); + } + + /// Add virtual columns if we don't already have them. + + Block pipe_header = builder->getHeader(); + + if (has_database_virtual_column && !pipe_header.has("_database")) + { + ColumnWithTypeAndName column; + column.name = "_database"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(database_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto adding_column_actions = std::make_shared( + std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + + builder->addSimpleTransform([&](const Block & stream_header) + { return std::make_shared(stream_header, adding_column_actions); }); + } + + if (has_table_virtual_column && !pipe_header.has("_table")) + { + ColumnWithTypeAndName column; + column.name = "_table"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(table_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto adding_column_actions = std::make_shared( + std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); + + builder->addSimpleTransform([&](const Block & stream_header) + { return std::make_shared(stream_header, adding_column_actions); }); + } + + /// Subordinary tables could have different but convertible types, like numeric types of different width. + /// We must return streams with structure equals to structure of Merge table. + convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, *builder, processed_stage); + } + + return builder; +} + +QueryPlan ReadFromMerge::createPlanForTable( + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & modified_query_info, + const QueryProcessingStage::Enum & processed_stage, + UInt64 max_block_size, + const StorageWithLockAndName & storage_with_lock, Names real_column_names, ContextMutablePtr modified_context, - size_t streams_num, - bool concat_streams) + size_t streams_num) { const auto & [database_name, storage, _, table_name] = storage_with_lock; auto & modified_select = modified_query_info.query->as(); - 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. @@ -647,14 +782,14 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( storage_snapshot, modified_query_info); + QueryPlan plan; + 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. if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); - QueryPlan & plan = child_plans.emplace_back(); - StorageView * view = dynamic_cast(storage.get()); if (!view || allow_experimental_analyzer) { @@ -688,16 +823,16 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (!plan.isInitialized()) return {}; - if (auto * read_from_merge_tree = typeid_cast(plan.getRootNode()->step.get())) + auto apply_filters = [this](ReadFromMergeTree & read_from_merge_tree) { 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]); - } + read_from_merge_tree.addFilter(filter_dags[i], filter_nodes.nodes[i]); - builder = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(modified_context), - BuildQueryPipelineSettings::fromContext(modified_context)); + read_from_merge_tree.applyFilters(); + return true; + }; + recursivelyApplyToReadingSteps(plan.getRootNode(), apply_filters); } else if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { @@ -705,14 +840,11 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( modified_context->setSetting("max_threads", streams_num); modified_context->setSetting("max_streams_to_max_threads_ratio", 1); - QueryPlan & plan = child_plans.emplace_back(); - if (allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree, modified_context, SelectQueryOptions(processed_stage).ignoreProjections()); - builder = std::make_unique(interpreter.buildQueryPipeline()); plan = std::move(interpreter.getPlanner()).extractQueryPlan(); } else @@ -722,71 +854,11 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( InterpreterSelectQuery interpreter{modified_query_info.query, modified_context, SelectQueryOptions(processed_stage).ignoreProjections()}; - builder = std::make_unique(interpreter.buildQueryPipeline(plan)); + interpreter.buildQueryPlan(plan); } - - /** Materialization is needed, since from distributed storage the constants come materialized. - * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, - * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. - */ - builder->addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); } - if (builder->initialized()) - { - if (concat_streams && builder->getNumStreams() > 1) - { - // It's possible to have many tables read from merge, resize(1) might open too many files at the same time. - // Using concat instead. - builder->addTransform(std::make_shared(builder->getHeader(), builder->getNumStreams())); - } - - /// Add virtual columns if we don't already have them. - - Block pipe_header = builder->getHeader(); - - if (has_database_virtual_column && !pipe_header.has("_database")) - { - ColumnWithTypeAndName column; - column.name = "_database"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(database_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), - ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - - builder->addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, adding_column_actions); - }); - } - - if (has_table_virtual_column && !pipe_header.has("_table")) - { - ColumnWithTypeAndName column; - column.name = "_table"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(table_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), - ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - - builder->addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, adding_column_actions); - }); - } - - /// Subordinary tables could have different but convertible types, like numeric types of different width. - /// We must return streams with structure equals to structure of Merge table. - convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, *builder, processed_stage); - } - - return builder; + return plan; } StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( @@ -1014,10 +1086,42 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) if (order_info_->direction != 1 && InterpreterSelectQuery::isQueryWithFinal(query_info)) return false; + auto request_read_in_order = [order_info = order_info_](ReadFromMergeTree & read_from_merge_tree) + { + return read_from_merge_tree.requestReadingInOrder( + order_info->used_prefix_of_sorting_key_size, order_info->direction, order_info->limit); + }; + + bool ok = true; + for (const auto & plan : child_plans) + if (plan.isInitialized()) + ok &= recursivelyApplyToReadingSteps(plan.getRootNode(), request_read_in_order); + + if (!ok) + return false; + order_info = order_info_; + query_info.input_order_info = order_info; return true; } +void ReadFromMerge::applyFilters() +{ + auto apply_filters = [this](ReadFromMergeTree & read_from_merge_tree) + { + 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]); + + read_from_merge_tree.applyFilters(); + return true; + }; + + for (const auto & plan : child_plans) + if (plan.isInitialized()) + recursivelyApplyToReadingSteps(plan.getRootNode(), apply_filters); +} + IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const { ColumnSizeByName column_sizes; diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index babf0dd92e8..cb20d37f125 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -1,9 +1,9 @@ #pragma once -#include -#include -#include #include +#include +#include +#include namespace DB @@ -146,6 +146,8 @@ public: /// Returns `false` if requested reading cannot be performed. bool requestReadingInOrder(InputOrderInfoPtr order_info_); + void applyFilters() override; + private: const size_t required_max_block_size; const size_t requested_num_streams; @@ -177,23 +179,35 @@ private: using Aliases = std::vector; - static SelectQueryInfo getModifiedQueryInfo(const SelectQueryInfo & query_info, - const ContextPtr & modified_context, - const StorageWithLockAndName & storage_with_lock_and_name, - const StorageSnapshotPtr & storage_snapshot); + std::vector table_aliases; - QueryPipelineBuilderPtr createSources( + void createChildPlans(); + + QueryPlan createPlanForTable( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, UInt64 max_block_size, - const Block & header, - const Aliases & aliases, const StorageWithLockAndName & storage_with_lock, Names real_column_names, ContextMutablePtr modified_context, - size_t streams_num, - bool concat_streams = false); + size_t streams_num); + + QueryPipelineBuilderPtr createSources( + QueryPlan & plan, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & modified_query_info, + const QueryProcessingStage::Enum & processed_stage, + const Block & header, + const Aliases & aliases, + const StorageWithLockAndName & storage_with_lock, + ContextMutablePtr modified_context, + bool concat_streams = false) const; + + static SelectQueryInfo getModifiedQueryInfo(const SelectQueryInfo & query_info, + const ContextPtr & modified_context, + const StorageWithLockAndName & storage_with_lock_and_name, + const StorageSnapshotPtr & storage_snapshot); static void convertingSourceStream( const Block & header, From 719ccc0c1cb0f3116922142bbd1de1e09e6aa849 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 21 Sep 2023 23:55:11 +0200 Subject: [PATCH 0037/1190] add test --- .../02875_merge_engine_set_index.reference | 1 + .../02875_merge_engine_set_index.sh | 65 +++++++++++++++++++ 2 files changed, 66 insertions(+) create mode 100644 tests/queries/0_stateless/02875_merge_engine_set_index.reference create mode 100755 tests/queries/0_stateless/02875_merge_engine_set_index.sh diff --git a/tests/queries/0_stateless/02875_merge_engine_set_index.reference b/tests/queries/0_stateless/02875_merge_engine_set_index.reference new file mode 100644 index 00000000000..00750edc07d --- /dev/null +++ b/tests/queries/0_stateless/02875_merge_engine_set_index.reference @@ -0,0 +1 @@ +3 diff --git a/tests/queries/0_stateless/02875_merge_engine_set_index.sh b/tests/queries/0_stateless/02875_merge_engine_set_index.sh new file mode 100755 index 00000000000..c4e28e3ba12 --- /dev/null +++ b/tests/queries/0_stateless/02875_merge_engine_set_index.sh @@ -0,0 +1,65 @@ +#!/usr/bin/env bash +# Tags: long, no-tsan + +# shellcheck disable=SC2154 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -nq " + CREATE TABLE t1 + ( + a UInt32, + b UInt32 + ) + ENGINE = MergeTree + ORDER BY (a, b); + + INSERT INTO t1 SELECT number, number FROM numbers_mt(1e6); + + CREATE TABLE t2 + ( + a UInt32, + b UInt32 + ) + ENGINE = MergeTree + ORDER BY (a, b); + + INSERT INTO t2 VALUES (1, 1) (2, 2) (3, 3); + + CREATE TABLE t + ( + a UInt32, + b UInt32 + ) + ENGINE = Merge(currentDatabase(), 't*');" + +query_id="${CLICKHOUSE_DATABASE}_merge_engine_set_index_$RANDOM$RANDOM" +$CLICKHOUSE_CLIENT --query_id="$query_id" --multiquery -q " +SELECT + a, + b +FROM t +WHERE (a, b) IN ( + SELECT DISTINCT + a, + b + FROM t2 +) +GROUP BY + a, + b +ORDER BY + a ASC, + b DESC +FORMAT Null;" + +$CLICKHOUSE_CLIENT -nq " +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['SelectedMarks'] +FROM system.query_log +WHERE (query_id = '$query_id') AND (type = 'QueryFinish'); +" From f7d70d1c3297cb8061ea26ee564979f416ca389d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Sep 2023 12:10:44 +0200 Subject: [PATCH 0038/1190] fix test --- .../0_stateless/02875_merge_engine_set_index.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02875_merge_engine_set_index.sh b/tests/queries/0_stateless/02875_merge_engine_set_index.sh index c4e28e3ba12..57b5db374c1 100755 --- a/tests/queries/0_stateless/02875_merge_engine_set_index.sh +++ b/tests/queries/0_stateless/02875_merge_engine_set_index.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: long, no-tsan # shellcheck disable=SC2154 @@ -15,7 +14,8 @@ $CLICKHOUSE_CLIENT -nq " b UInt32 ) ENGINE = MergeTree - ORDER BY (a, b); + ORDER BY (a, b) + SETTINGS index_granularity = 8192; INSERT INTO t1 SELECT number, number FROM numbers_mt(1e6); @@ -25,7 +25,8 @@ $CLICKHOUSE_CLIENT -nq " b UInt32 ) ENGINE = MergeTree - ORDER BY (a, b); + ORDER BY (a, b) + SETTINGS index_granularity = 8192; INSERT INTO t2 VALUES (1, 1) (2, 2) (3, 3); @@ -61,5 +62,4 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectedMarks'] FROM system.query_log -WHERE (query_id = '$query_id') AND (type = 'QueryFinish'); -" +WHERE event_date >= yesterday() AND current_database = currentDatabase() AND (query_id = '$query_id') AND (type = 'QueryFinish');" From 694792fa5d5faebcb136202a3d12ce4210834267 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 23 Sep 2023 18:10:29 +0200 Subject: [PATCH 0039/1190] fix for analyzer --- src/Storages/StorageMerge.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 2fa9e38a816..8977833d3dc 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -845,7 +845,9 @@ QueryPlan ReadFromMerge::createPlanForTable( InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree, modified_context, SelectQueryOptions(processed_stage).ignoreProjections()); - plan = std::move(interpreter.getPlanner()).extractQueryPlan(); + auto & planner = interpreter.getPlanner(); + planner.buildQueryPlanIfNeeded(); + plan = std::move(planner).extractQueryPlan(); } else { From 595027b1c6497f0b4792818e6c1f5dc5c8f0c28c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 26 Sep 2023 00:20:40 +0200 Subject: [PATCH 0040/1190] better --- src/Storages/StorageMerge.cpp | 33 +++++++++++++++------------------ src/Storages/StorageMerge.h | 3 +++ 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 8977833d3dc..7d666cc4937 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -588,8 +589,6 @@ void ReadFromMerge::createChildPlans() context, current_streams)); } - - applyFilters(); } SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & query_info, @@ -655,13 +654,15 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & quer return modified_query_info; } -template -bool recursivelyApplyToReadingSteps(QueryPlan::Node * node, const Func & func) +bool recursivelyApplyToReadingSteps(QueryPlan::Node * node, const std::function & func) { bool ok = true; for (auto * child : node->children) ok &= recursivelyApplyToReadingSteps(child, func); + // This code is mainly meant to be used to call `requestReadingInOrder` on child steps. + // In this case it is ok if one child will read in order and other will not (though I don't know when it is possible), + // the only important part is to acknowledge this at the parent and don't rely on any particular ordering of input data. if (!ok) return false; @@ -823,16 +824,7 @@ QueryPlan ReadFromMerge::createPlanForTable( if (!plan.isInitialized()) return {}; - auto apply_filters = [this](ReadFromMergeTree & read_from_merge_tree) - { - 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]); - - read_from_merge_tree.applyFilters(); - return true; - }; - recursivelyApplyToReadingSteps(plan.getRootNode(), apply_filters); + applyFilters(plan); } else if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { @@ -1088,10 +1080,10 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) if (order_info_->direction != 1 && InterpreterSelectQuery::isQueryWithFinal(query_info)) return false; - auto request_read_in_order = [order_info = order_info_](ReadFromMergeTree & read_from_merge_tree) + auto request_read_in_order = [order_info_](ReadFromMergeTree & read_from_merge_tree) { return read_from_merge_tree.requestReadingInOrder( - order_info->used_prefix_of_sorting_key_size, order_info->direction, order_info->limit); + order_info_->used_prefix_of_sorting_key_size, order_info_->direction, order_info_->limit); }; bool ok = true; @@ -1107,7 +1099,7 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) return true; } -void ReadFromMerge::applyFilters() +void ReadFromMerge::applyFilters(const QueryPlan & plan) const { auto apply_filters = [this](ReadFromMergeTree & read_from_merge_tree) { @@ -1119,9 +1111,14 @@ void ReadFromMerge::applyFilters() return true; }; + recursivelyApplyToReadingSteps(plan.getRootNode(), apply_filters); +} + +void ReadFromMerge::applyFilters() +{ for (const auto & plan : child_plans) if (plan.isInitialized()) - recursivelyApplyToReadingSteps(plan.getRootNode(), apply_filters); + applyFilters(plan); } IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index cb20d37f125..80a5fa335f7 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -183,6 +184,8 @@ private: void createChildPlans(); + void applyFilters(const QueryPlan & plan) const; + QueryPlan createPlanForTable( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, From 15fe392fed647ece1d18fea8d2a6b8197c1a9ae6 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 27 Sep 2023 10:13:23 +0000 Subject: [PATCH 0041/1190] 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 0042/1190] 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 0043/1190] 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 05f82d48df39960dc84fea8c63e076c0ce68fcc3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 1 Oct 2023 13:39:40 +0200 Subject: [PATCH 0044/1190] Use 2 more thread pools to avoid hanging in case BACKUP/RESTORE ON CLUSTER ASYNC. Create thread pools lazily. --- src/Backups/BackupsWorker.cpp | 224 +++++++++++++++++++++++----------- src/Backups/BackupsWorker.h | 16 +-- 2 files changed, 165 insertions(+), 75 deletions(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index da814dcbc08..bb0d293ccd8 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -218,42 +218,145 @@ namespace } -BackupsWorker::BackupsWorker( - ContextPtr global_context, - size_t num_backup_threads, - size_t num_restore_threads, - bool allow_concurrent_backups_, - bool allow_concurrent_restores_) - : backups_thread_pool(std::make_unique( - CurrentMetrics::BackupsThreads, - CurrentMetrics::BackupsThreadsActive, - num_backup_threads, - /* max_free_threads = */ 0, - num_backup_threads)) - , restores_thread_pool(std::make_unique( - CurrentMetrics::RestoreThreads, - CurrentMetrics::RestoreThreadsActive, - num_restore_threads, - /* max_free_threads = */ 0, - num_restore_threads)) - , backup_async_executor_pool(std::make_unique( - CurrentMetrics::BackupsThreads, - CurrentMetrics::BackupsThreadsActive, - num_backup_threads, - num_backup_threads, - num_backup_threads)) - , restore_async_executor_pool(std::make_unique( - CurrentMetrics::RestoreThreads, - CurrentMetrics::RestoreThreadsActive, - num_restore_threads, - num_restore_threads, - num_restore_threads)) - , log(&Poco::Logger::get("BackupsWorker")) +/// We have to use multiple thread pools because +/// 1) there should be separate thread pools for BACKUP and RESTORE; +/// 2) a task from a thread pool can't wait another task from the same thread pool. (Because if it schedules and waits +/// while the thread pool is still occupied with the waiting task then a scheduled task can be never executed). +enum class BackupsWorker::ThreadPoolId +{ + /// "BACKUP ON CLUSTER ASYNC" waits in background while "BACKUP ASYNC" is finished on the nodes of the cluster, then finalizes the backup. + BACKUP_ASYNC_ON_CLUSTER, + + /// "BACKUP ASYNC" waits in background while all file infos are built and then it copies the backup's files. + BACKUP_ASYNC, + + /// Making a list of files to copy and copying of those files is always sequential, so those operations can share one thread pool. + BACKUP_MAKE_LIST_FILES_TO_COPY, + BACKUP_COPY_FILES = BACKUP_MAKE_LIST_FILES_TO_COPY, + + /// "RESTORE ON CLUSTER ASYNC" waits in background while "BACKUP ASYNC" is finished on the nodes of the cluster, then finalizes the backup. + RESTORE_ASYNC_ON_CLUSTER, + + /// "RESTORE ASYNC" waits in background while the data of all tables are restored. + RESTORE_ASYNC, + + /// Restores the data of tables. + RESTORE_TABLES_DATA, + + MAX, +}; + + +/// Keeps thread pools for BackupsWorker. +class BackupsWorker::ThreadPools +{ +public: + ThreadPools(size_t num_backup_threads_, size_t num_restore_threads_) + : num_backup_threads(num_backup_threads_), num_restore_threads(num_restore_threads_) + { + } + + /// Returns a thread pool, creates it if it's not created yet. + ThreadPool & getThreadPool(ThreadPoolId thread_pool_id) + { + std::lock_guard lock{mutex}; + auto & thread_pool = thread_pools[static_cast(thread_pool_id)]; + if (!thread_pool) + { + CurrentMetrics::Metric metric_threads; + CurrentMetrics::Metric metric_active_threads; + size_t max_threads = 0; + + /// What to do with a new job if a corresponding thread pool is already running `max_threads` jobs: + /// `use_queue == true` - put into the thread pool's queue, + /// `use_queue == false` - schedule() should wait until some of the jobs finish. + bool use_queue = false; + + switch (thread_pool_id) + { + case ThreadPoolId::BACKUP_ASYNC: + case ThreadPoolId::BACKUP_ASYNC_ON_CLUSTER: + case ThreadPoolId::BACKUP_COPY_FILES: + { + metric_threads = CurrentMetrics::BackupsThreads; + metric_active_threads = CurrentMetrics::BackupsThreadsActive; + max_threads = num_backup_threads; + /// We don't use the thread pool's queue for copying files because otherwise that queue could be memory-wasting. + use_queue = (thread_pool_id != ThreadPoolId::BACKUP_COPY_FILES); + break; + } + + case ThreadPoolId::RESTORE_ASYNC: + case ThreadPoolId::RESTORE_ASYNC_ON_CLUSTER: + case ThreadPoolId::RESTORE_TABLES_DATA: + { + metric_threads = CurrentMetrics::RestoreThreads; + metric_active_threads = CurrentMetrics::RestoreThreadsActive; + max_threads = num_restore_threads; + use_queue = (thread_pool_id != ThreadPoolId::RESTORE_TABLES_DATA); + break; + } + + default: + UNREACHABLE(); + } + + /// We set max_free_threads = 0 because we don't want to keep any threads if there is no RESTORE query running right now. + size_t max_free_threads = 0; + size_t queue_size = use_queue ? 0 : max_threads; + thread_pool = std::make_unique(metric_threads, metric_active_threads, max_threads, max_free_threads, queue_size); + } + return *thread_pool; + } + + /// Waits for all threads to finish. + void wait() + { + auto wait_sequence = { + ThreadPoolId::RESTORE_ASYNC_ON_CLUSTER, + ThreadPoolId::RESTORE_ASYNC, + ThreadPoolId::RESTORE_TABLES_DATA, + ThreadPoolId::BACKUP_ASYNC_ON_CLUSTER, + ThreadPoolId::BACKUP_ASYNC, + ThreadPoolId::BACKUP_COPY_FILES, + }; + + for (auto thread_pool_id : wait_sequence) + { + ThreadPool * thread_pool; + { + std::lock_guard lock{mutex}; + thread_pool = thread_pools[static_cast(thread_pool_id)].get(); + } + if (thread_pool) + thread_pool->wait(); + } + } + +private: + const size_t num_backup_threads; + const size_t num_restore_threads; + std::unique_ptr thread_pools[static_cast(ThreadPoolId::MAX)] TSA_GUARDED_BY(mutex); + std::mutex mutex; +}; + + +BackupsWorker::BackupsWorker(ContextPtr global_context, size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_) + : thread_pools(std::make_unique(num_backup_threads, num_restore_threads)) , allow_concurrent_backups(allow_concurrent_backups_) , allow_concurrent_restores(allow_concurrent_restores_) + , log(&Poco::Logger::get("BackupsWorker")) { backup_log = global_context->getBackupLog(); - /// We set max_free_threads = 0 because we don't want to keep any threads if there is no BACKUP or RESTORE query running right now. +} + + +BackupsWorker::~BackupsWorker() = default; + + +ThreadPool & BackupsWorker::getThreadPool(ThreadPoolId thread_pool_id) +{ + return thread_pools->getThreadPool(thread_pool_id); } @@ -313,16 +416,9 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context if (backup_settings.async) { - backup_async_executor_pool->scheduleOrThrowOnError( - [this, - backup_query, - backup_id, - backup_name_for_logging, - backup_info, - backup_settings, - backup_coordination, - context_in_use, - mutable_context] + auto & thread_pool = getThreadPool(on_cluster ? ThreadPoolId::BACKUP_ASYNC_ON_CLUSTER : ThreadPoolId::BACKUP_ASYNC); + thread_pool.scheduleOrThrowOnError( + [this, backup_query, backup_id, backup_name_for_logging, backup_info, backup_settings, backup_coordination, context_in_use, mutable_context] { doBackup( backup_query, @@ -515,7 +611,7 @@ void BackupsWorker::buildFileInfosForBackupEntries(const BackupPtr & backup, con LOG_TRACE(log, "{}", Stage::BUILDING_FILE_INFOS); backup_coordination->setStage(Stage::BUILDING_FILE_INFOS, ""); backup_coordination->waitForStage(Stage::BUILDING_FILE_INFOS); - backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), read_settings, *backups_thread_pool)); + backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), read_settings, getThreadPool(ThreadPoolId::BACKUP_MAKE_LIST_FILES_TO_COPY))); } @@ -541,6 +637,7 @@ void BackupsWorker::writeBackupEntries(BackupMutablePtr backup, BackupEntries && std::exception_ptr exception; bool always_single_threaded = !backup->supportsWritingInMultipleThreads(); + auto & thread_pool = getThreadPool(ThreadPoolId::BACKUP_COPY_FILES); auto thread_group = CurrentThread::getGroup(); for (size_t i = 0; i != backup_entries.size(); ++i) @@ -608,7 +705,7 @@ void BackupsWorker::writeBackupEntries(BackupMutablePtr backup, BackupEntries && continue; } - backups_thread_pool->scheduleOrThrowOnError([job] { job(true); }); + thread_pool.scheduleOrThrowOnError([job] { job(true); }); } { @@ -666,25 +763,19 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt if (restore_settings.async) { - restore_async_executor_pool->scheduleOrThrowOnError( - [this, - restore_query, - restore_id, - backup_name_for_logging, - backup_info, - restore_settings, - restore_coordination, - context_in_use] - { - doRestore( - restore_query, - restore_id, - backup_name_for_logging, - backup_info, - restore_settings, - restore_coordination, - context_in_use, - /* called_async= */ true); + auto & thread_pool = getThreadPool(on_cluster ? ThreadPoolId::RESTORE_ASYNC_ON_CLUSTER : ThreadPoolId::RESTORE_ASYNC); + thread_pool.scheduleOrThrowOnError( + [this, restore_query, restore_id, backup_name_for_logging, backup_info, restore_settings, restore_coordination, context_in_use] + { + doRestore( + restore_query, + restore_id, + backup_name_for_logging, + backup_info, + restore_settings, + restore_coordination, + context_in_use, + /* called_async= */ true); }); } else @@ -818,7 +909,7 @@ void BackupsWorker::doRestore( } /// Execute the data restoring tasks. - restoreTablesData(restore_id, backup, std::move(data_restore_tasks), *restores_thread_pool); + restoreTablesData(restore_id, backup, std::move(data_restore_tasks), getThreadPool(ThreadPoolId::RESTORE_TABLES_DATA)); /// We have restored everything, we need to tell other hosts (they could be waiting for it). restore_coordination->setStage(Stage::COMPLETED, ""); @@ -1049,10 +1140,7 @@ void BackupsWorker::shutdown() if (has_active_backups_and_restores) LOG_INFO(log, "Waiting for {} backups and {} restores to be finished", num_active_backups, num_active_restores); - backups_thread_pool->wait(); - restores_thread_pool->wait(); - backup_async_executor_pool->wait(); - restore_async_executor_pool->wait(); + thread_pools->wait(); if (has_active_backups_and_restores) LOG_INFO(log, "All backup and restore tasks have finished"); diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index bf7f2e8f1e4..b0a76eb0fa8 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -33,6 +33,7 @@ class BackupsWorker { public: BackupsWorker(ContextPtr global_context, size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_); + ~BackupsWorker(); /// Waits until all tasks have been completed. void shutdown(); @@ -88,11 +89,15 @@ private: void setNumFilesAndSize(const BackupOperationID & id, size_t num_files, UInt64 total_size, size_t num_entries, UInt64 uncompressed_size, UInt64 compressed_size, size_t num_read_files, UInt64 num_read_bytes); - std::unique_ptr backups_thread_pool; - std::unique_ptr restores_thread_pool; + enum class ThreadPoolId; + ThreadPool & getThreadPool(ThreadPoolId thread_pool_id); - std::unique_ptr backup_async_executor_pool; - std::unique_ptr restore_async_executor_pool; + class ThreadPools; + std::unique_ptr thread_pools; + + const bool allow_concurrent_backups; + const bool allow_concurrent_restores; + Poco::Logger * log; std::unordered_map infos; std::shared_ptr backup_log; @@ -100,9 +105,6 @@ private: std::atomic num_active_backups = 0; std::atomic num_active_restores = 0; mutable std::mutex infos_mutex; - Poco::Logger * log; - const bool allow_concurrent_backups; - const bool allow_concurrent_restores; }; } From 8966ae4c0e5afabeeef9ba8328323f5546f2bc7d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 1 Oct 2023 13:49:01 +0200 Subject: [PATCH 0045/1190] Schedule threads while making a backup faster. --- src/Backups/BackupFileInfo.cpp | 13 +++++-------- src/Backups/BackupsWorker.cpp | 12 +++++------- 2 files changed, 10 insertions(+), 15 deletions(-) diff --git a/src/Backups/BackupFileInfo.cpp b/src/Backups/BackupFileInfo.cpp index f595c02ddc5..63427de328e 100644 --- a/src/Backups/BackupFileInfo.cpp +++ b/src/Backups/BackupFileInfo.cpp @@ -215,14 +215,13 @@ BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entr ++num_active_jobs; } - auto job = [&mutex, &num_active_jobs, &event, &exception, &infos, &backup_entries, &read_settings, &base_backup, &thread_group, i, log](bool async) + auto job = [&mutex, &num_active_jobs, &event, &exception, &infos, &backup_entries, &read_settings, &base_backup, &thread_group, i, log]() { SCOPE_EXIT_SAFE({ std::lock_guard lock{mutex}; if (!--num_active_jobs) event.notify_all(); - if (async) - CurrentThread::detachFromGroupIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); }); try @@ -230,11 +229,10 @@ BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entr const auto & name = backup_entries[i].first; const auto & entry = backup_entries[i].second; - if (async && thread_group) + if (thread_group) CurrentThread::attachToGroup(thread_group); - if (async) - setThreadName("BackupWorker"); + setThreadName("BackupWorker"); { std::lock_guard lock{mutex}; @@ -252,8 +250,7 @@ BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entr } }; - if (!thread_pool.trySchedule([job] { job(true); })) - job(false); + thread_pool.scheduleOrThrowOnError(job); } { diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index bb0d293ccd8..9598b09bd8c 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -954,23 +954,21 @@ void BackupsWorker::restoreTablesData(const OperationID & restore_id, BackupPtr ++num_active_jobs; } - auto job = [&](bool async) + auto job = [&]() { SCOPE_EXIT_SAFE( std::lock_guard lock{mutex}; if (!--num_active_jobs) event.notify_all(); - if (async) - CurrentThread::detachFromGroupIfNotDetached(); + CurrentThread::detachFromGroupIfNotDetached(); ); try { - if (async && thread_group) + if (thread_group) CurrentThread::attachToGroup(thread_group); - if (async) - setThreadName("RestoreWorker"); + setThreadName("RestoreWorker"); { std::lock_guard lock{mutex}; @@ -997,7 +995,7 @@ void BackupsWorker::restoreTablesData(const OperationID & restore_id, BackupPtr } }; - thread_pool.scheduleOrThrowOnError([job] { job(true); }); + thread_pool.scheduleOrThrowOnError(job); } { From 020d76a383b3a489a3868c3a7c1d770cfaba5549 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 24 May 2023 21:59:46 +0000 Subject: [PATCH 0046/1190] 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 0047/1190] 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 0048/1190] 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 0049/1190] 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 0050/1190] 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 0051/1190] 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 0052/1190] 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 0053/1190] 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 0054/1190] 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 0055/1190] 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 0056/1190] 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 0057/1190] 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 0058/1190] 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 0059/1190] 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 0060/1190] 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 0061/1190] 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 0062/1190] 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 0063/1190] 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 0064/1190] 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 0065/1190] 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 0066/1190] 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 0067/1190] 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 0068/1190] 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 0069/1190] 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 0070/1190] 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 0071/1190] 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 0072/1190] 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 4cb4a46adc5d99ed95095bc9b55da4fc32935412 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 12 Oct 2023 17:03:06 +0200 Subject: [PATCH 0073/1190] Added clickhouse-parser to fix clang-tidy --- src/Common/ZooKeeper/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index 3f7e87ff4a7..0c9e0b79827 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -10,6 +10,7 @@ target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io + clickhouse_parsers common PRIVATE string_utils @@ -20,6 +21,7 @@ add_library(clickhouse_common_zookeeper_no_log ${clickhouse_common_zookeeper_hea target_link_libraries (clickhouse_common_zookeeper_no_log PUBLIC clickhouse_common_io + clickhouse_parsers common PRIVATE string_utils From 16bdd31cee2a0d2326dd9f627dd85c2696cc0156 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 13 Oct 2023 10:38:18 +0200 Subject: [PATCH 0074/1190] Included to use compression lib in utils & zookeeper tests --- src/Common/ZooKeeper/examples/CMakeLists.txt | 6 +++--- utils/zookeeper-cli/CMakeLists.txt | 2 +- utils/zookeeper-dump-tree/CMakeLists.txt | 2 +- utils/zookeeper-remove-by-list/CMakeLists.txt | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index e8932fd3088..45b40b13873 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -1,8 +1,8 @@ clickhouse_add_executable(zkutil_test_commands zkutil_test_commands.cpp) -target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log) +target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers) clickhouse_add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) -target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log string_utils) +target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log string_utils clickhouse_parsers) clickhouse_add_executable(zkutil_test_async zkutil_test_async.cpp) -target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log) +target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers) diff --git a/utils/zookeeper-cli/CMakeLists.txt b/utils/zookeeper-cli/CMakeLists.txt index be8cf81320c..54f1a32ba37 100644 --- a/utils/zookeeper-cli/CMakeLists.txt +++ b/utils/zookeeper-cli/CMakeLists.txt @@ -1,4 +1,4 @@ clickhouse_add_executable(clickhouse-zookeeper-cli zookeeper-cli.cpp ${ClickHouse_SOURCE_DIR}/src/Client/LineReader.cpp) -target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper_no_log) +target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers) diff --git a/utils/zookeeper-dump-tree/CMakeLists.txt b/utils/zookeeper-dump-tree/CMakeLists.txt index 182cb65f194..e43c21b2843 100644 --- a/utils/zookeeper-dump-tree/CMakeLists.txt +++ b/utils/zookeeper-dump-tree/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-dump-tree main.cpp ${SRCS}) -target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io boost::program_options) +target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers clickhouse_common_io boost::program_options) diff --git a/utils/zookeeper-remove-by-list/CMakeLists.txt b/utils/zookeeper-remove-by-list/CMakeLists.txt index 01965413d29..034704e8a95 100644 --- a/utils/zookeeper-remove-by-list/CMakeLists.txt +++ b/utils/zookeeper-remove-by-list/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-remove-by-list main.cpp ${SRCS}) -target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper_no_log boost::program_options) +target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper_no_log boost::program_options clickhouse_parsers) From e8271685a9cb04d0dcbc581e0b99d956e1402b29 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 13 Oct 2023 13:20:02 +0200 Subject: [PATCH 0075/1190] Added basic test --- .../test_keeper_compression/__init__.py | 0 .../configs/keeper.xml | 29 +++++++++++++++++++ .../configs/keeper_with_compression.xml | 26 +++++++++++++++++ .../configs/keeper_without_compression.xml | 26 +++++++++++++++++ .../test_with_compression.py | 26 +++++++++++++++++ .../test_without_compression.py | 26 +++++++++++++++++ 6 files changed, 133 insertions(+) create mode 100644 tests/integration/test_keeper_compression/__init__.py create mode 100644 tests/integration/test_keeper_compression/configs/keeper.xml create mode 100644 tests/integration/test_keeper_compression/configs/keeper_with_compression.xml create mode 100644 tests/integration/test_keeper_compression/configs/keeper_without_compression.xml create mode 100644 tests/integration/test_keeper_compression/test_with_compression.py create mode 100644 tests/integration/test_keeper_compression/test_without_compression.py diff --git a/tests/integration/test_keeper_compression/__init__.py b/tests/integration/test_keeper_compression/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_compression/configs/keeper.xml b/tests/integration/test_keeper_compression/configs/keeper.xml new file mode 100644 index 00000000000..322938c3dc8 --- /dev/null +++ b/tests/integration/test_keeper_compression/configs/keeper.xml @@ -0,0 +1,29 @@ + + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + 5000 + trace + 10000 + + + + true + node1 + 1 + 2888 + 1 + + + + + + + /clickhouse/access + + + diff --git a/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml b/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml new file mode 100644 index 00000000000..adabd76a0a0 --- /dev/null +++ b/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml @@ -0,0 +1,26 @@ + + + true + + node1 + 9181 + + + + + + + + + + + + + + + + + + + + diff --git a/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml b/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml new file mode 100644 index 00000000000..628d0cc8d17 --- /dev/null +++ b/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml @@ -0,0 +1,26 @@ + + + false + + node1 + 9181 + + + + + + + + + + + + + + + + + + + + diff --git a/tests/integration/test_keeper_compression/test_with_compression.py b/tests/integration/test_keeper_compression/test_with_compression.py new file mode 100644 index 00000000000..7c9939c268c --- /dev/null +++ b/tests/integration/test_keeper_compression/test_with_compression.py @@ -0,0 +1,26 @@ +#!/usr/bin/env python3 + +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", main_configs=["configs/keeper.xml", "configs/keeper_with_compression.xml"], stay_alive=True +) + + +# test that server is able to start +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_select(started_cluster): + assert node1.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_keeper_compression/test_without_compression.py b/tests/integration/test_keeper_compression/test_without_compression.py new file mode 100644 index 00000000000..6a98514c352 --- /dev/null +++ b/tests/integration/test_keeper_compression/test_without_compression.py @@ -0,0 +1,26 @@ +#!/usr/bin/env python3 + +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", main_configs=["configs/keeper.xml", "configs/keeper_without_compression.xml"], stay_alive=True +) + + +# test that server is able to start +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_select(started_cluster): + assert node1.query("SELECT 1") == "1\n" From a0491065e5a4d31ad200d2eaef0750b07575f301 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 13 Oct 2023 11:40:14 +0000 Subject: [PATCH 0076/1190] Automatic style fix --- .../test_keeper_compression/test_with_compression.py | 4 +++- .../test_keeper_compression/test_without_compression.py | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_compression/test_with_compression.py b/tests/integration/test_keeper_compression/test_with_compression.py index 7c9939c268c..1c0697ebcbb 100644 --- a/tests/integration/test_keeper_compression/test_with_compression.py +++ b/tests/integration/test_keeper_compression/test_with_compression.py @@ -7,7 +7,9 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/keeper.xml", "configs/keeper_with_compression.xml"], stay_alive=True + "node1", + main_configs=["configs/keeper.xml", "configs/keeper_with_compression.xml"], + stay_alive=True, ) diff --git a/tests/integration/test_keeper_compression/test_without_compression.py b/tests/integration/test_keeper_compression/test_without_compression.py index 6a98514c352..eb7936cfd42 100644 --- a/tests/integration/test_keeper_compression/test_without_compression.py +++ b/tests/integration/test_keeper_compression/test_without_compression.py @@ -7,7 +7,9 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/keeper.xml", "configs/keeper_without_compression.xml"], stay_alive=True + "node1", + main_configs=["configs/keeper.xml", "configs/keeper_without_compression.xml"], + stay_alive=True, ) From c15b85074fb06a3245e8fcb656f8521d9acf8f9b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 13 Oct 2023 20:23:46 +0200 Subject: [PATCH 0077/1190] Updated to include compression in clickhouse_common_io --- src/Common/ZooKeeper/CMakeLists.txt | 2 -- src/Common/ZooKeeper/examples/CMakeLists.txt | 6 +++--- .../configs/keeper_with_compression.xml | 17 ----------------- .../configs/keeper_without_compression.xml | 17 ----------------- utils/zookeeper-cli/CMakeLists.txt | 2 +- utils/zookeeper-dump-tree/CMakeLists.txt | 2 +- utils/zookeeper-remove-by-list/CMakeLists.txt | 2 +- 7 files changed, 6 insertions(+), 42 deletions(-) diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index 0c9e0b79827..3f7e87ff4a7 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -10,7 +10,6 @@ target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io - clickhouse_parsers common PRIVATE string_utils @@ -21,7 +20,6 @@ add_library(clickhouse_common_zookeeper_no_log ${clickhouse_common_zookeeper_hea target_link_libraries (clickhouse_common_zookeeper_no_log PUBLIC clickhouse_common_io - clickhouse_parsers common PRIVATE string_utils diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index 45b40b13873..e8932fd3088 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -1,8 +1,8 @@ clickhouse_add_executable(zkutil_test_commands zkutil_test_commands.cpp) -target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers) +target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log) clickhouse_add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) -target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log string_utils clickhouse_parsers) +target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log string_utils) clickhouse_add_executable(zkutil_test_async zkutil_test_async.cpp) -target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers) +target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log) diff --git a/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml b/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml index adabd76a0a0..c84475cf481 100644 --- a/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml +++ b/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml @@ -6,21 +6,4 @@ 9181 - - - - - - - - - - - - - - - - - diff --git a/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml b/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml index 628d0cc8d17..341b36333d0 100644 --- a/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml +++ b/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml @@ -6,21 +6,4 @@ 9181 - - - - - - - - - - - - - - - - - diff --git a/utils/zookeeper-cli/CMakeLists.txt b/utils/zookeeper-cli/CMakeLists.txt index 54f1a32ba37..be8cf81320c 100644 --- a/utils/zookeeper-cli/CMakeLists.txt +++ b/utils/zookeeper-cli/CMakeLists.txt @@ -1,4 +1,4 @@ clickhouse_add_executable(clickhouse-zookeeper-cli zookeeper-cli.cpp ${ClickHouse_SOURCE_DIR}/src/Client/LineReader.cpp) -target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers) +target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper_no_log) diff --git a/utils/zookeeper-dump-tree/CMakeLists.txt b/utils/zookeeper-dump-tree/CMakeLists.txt index e43c21b2843..182cb65f194 100644 --- a/utils/zookeeper-dump-tree/CMakeLists.txt +++ b/utils/zookeeper-dump-tree/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-dump-tree main.cpp ${SRCS}) -target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers clickhouse_common_io boost::program_options) +target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io boost::program_options) diff --git a/utils/zookeeper-remove-by-list/CMakeLists.txt b/utils/zookeeper-remove-by-list/CMakeLists.txt index 034704e8a95..01965413d29 100644 --- a/utils/zookeeper-remove-by-list/CMakeLists.txt +++ b/utils/zookeeper-remove-by-list/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-remove-by-list main.cpp ${SRCS}) -target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper_no_log boost::program_options clickhouse_parsers) +target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper_no_log boost::program_options) From 4d20516c9b65efd3880fd5eb2258025be2c45b88 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 14 Oct 2023 13:44:22 +0200 Subject: [PATCH 0078/1190] Updated src/CMakeLists.txt --- src/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1c93bc5d35e..b7335690b47 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -83,6 +83,7 @@ add_subdirectory (Formats) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io Common/SSH) +add_headers_and_sources(clickhouse_common_io Compression) add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/Archives) add_headers_and_sources(clickhouse_common_io IO/Resource) @@ -292,6 +293,7 @@ target_link_libraries (clickhouse_common_io common ch_contrib::double_conversion ch_contrib::dragonbox_to_chars + ch_contrib::libdivide ) # Use X86 AVX2/AVX512 instructions to accelerate filter operations From 0c68f1b4539396a0746b48b57647b959d108ad09 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 14 Oct 2023 18:26:57 +0200 Subject: [PATCH 0079/1190] Removed unwanted includes --- src/Common/ZooKeeper/ZooKeeperIO.h | 1 - src/Common/ZooKeeper/ZooKeeperImpl.h | 2 -- 2 files changed, 3 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index b1cfce3f621..83973c1ae22 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -8,7 +8,6 @@ #include #include #include -#include namespace Coordination diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index bde9fcbd79a..ee3a31110d3 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -15,8 +15,6 @@ #include #include #include -#include -#include #include #include From 646dce47bf68cea63e5d2b63b6b5426727b7b7e5 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 14 Oct 2023 21:58:31 +0200 Subject: [PATCH 0080/1190] Included Parsers directory inside common_io --- src/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 704dadcfd6b..83eccbcf097 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -84,6 +84,7 @@ add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io Common/SSH) add_headers_and_sources(clickhouse_common_io Compression) +add_headers_and_sources(clickhouse_common_io Parsers) add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/Archives) add_headers_and_sources(clickhouse_common_io IO/Resource) From 19163497b7a0c0e7e5597b6e1d6d2c04633dae89 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 15 Oct 2023 10:15:03 +0200 Subject: [PATCH 0081/1190] Added clickhouse-parsers to clickhouse_common_zookeeper --- src/Common/ZooKeeper/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index 3f7e87ff4a7..0c9e0b79827 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -10,6 +10,7 @@ target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io + clickhouse_parsers common PRIVATE string_utils @@ -20,6 +21,7 @@ add_library(clickhouse_common_zookeeper_no_log ${clickhouse_common_zookeeper_hea target_link_libraries (clickhouse_common_zookeeper_no_log PUBLIC clickhouse_common_io + clickhouse_parsers common PRIVATE string_utils From 43cec3d8a3486e0d9eb06f6a105dee50fdee5c06 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 15 Oct 2023 11:35:01 +0200 Subject: [PATCH 0082/1190] Added clickhouse_parsers to other builds with compression --- src/CMakeLists.txt | 3 ++- src/Common/Config/CMakeLists.txt | 2 ++ src/Common/ZooKeeper/examples/CMakeLists.txt | 6 +++--- utils/zookeeper-cli/CMakeLists.txt | 2 +- utils/zookeeper-dump-tree/CMakeLists.txt | 2 +- utils/zookeeper-remove-by-list/CMakeLists.txt | 2 +- 6 files changed, 10 insertions(+), 7 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 83eccbcf097..80a8c3b6be5 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -285,8 +285,9 @@ set_source_files_properties( Common/SymbolIndex.cpp PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE}") -target_link_libraries (clickhouse_common_io +target_link_libraries (clickhouse_common_iog PRIVATE + clickhouse_parsers string_utils widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index ec7bdd10196..e70a9944ce8 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -21,6 +21,8 @@ add_library(clickhouse_common_config_no_zookeeper_log ${SRCS}) target_link_libraries(clickhouse_common_config_no_zookeeper_log PUBLIC clickhouse_common_zookeeper_no_log + clickhouse_common_io + clickhouse_parsers common Poco::XML PRIVATE diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index e8932fd3088..7a950634ac9 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -1,8 +1,8 @@ clickhouse_add_executable(zkutil_test_commands zkutil_test_commands.cpp) -target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log) +target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers) clickhouse_add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) -target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log string_utils) +target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers string_utils) clickhouse_add_executable(zkutil_test_async zkutil_test_async.cpp) -target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log) +target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers) diff --git a/utils/zookeeper-cli/CMakeLists.txt b/utils/zookeeper-cli/CMakeLists.txt index be8cf81320c..7c592a62c84 100644 --- a/utils/zookeeper-cli/CMakeLists.txt +++ b/utils/zookeeper-cli/CMakeLists.txt @@ -1,4 +1,4 @@ clickhouse_add_executable(clickhouse-zookeeper-cli zookeeper-cli.cpp ${ClickHouse_SOURCE_DIR}/src/Client/LineReader.cpp) -target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper_no_log) +target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers) diff --git a/utils/zookeeper-dump-tree/CMakeLists.txt b/utils/zookeeper-dump-tree/CMakeLists.txt index 182cb65f194..027713aa26d 100644 --- a/utils/zookeeper-dump-tree/CMakeLists.txt +++ b/utils/zookeeper-dump-tree/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-dump-tree main.cpp ${SRCS}) -target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io boost::program_options) +target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers boost::program_options) diff --git a/utils/zookeeper-remove-by-list/CMakeLists.txt b/utils/zookeeper-remove-by-list/CMakeLists.txt index 01965413d29..4ee186f374a 100644 --- a/utils/zookeeper-remove-by-list/CMakeLists.txt +++ b/utils/zookeeper-remove-by-list/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-remove-by-list main.cpp ${SRCS}) -target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper_no_log boost::program_options) +target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers boost::program_options) From 25fb2b73e92633aa82fcffb27ab853b05b4a39c7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 15 Oct 2023 16:37:12 +0200 Subject: [PATCH 0083/1190] Fixed typo --- src/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 80a8c3b6be5..de4094851e1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -285,7 +285,7 @@ set_source_files_properties( Common/SymbolIndex.cpp PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE}") -target_link_libraries (clickhouse_common_iog +target_link_libraries (clickhouse_common_io PRIVATE clickhouse_parsers string_utils From 7b426e7cae053889fc33cee7ecda5af042fa4856 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 15 Oct 2023 21:56:34 +0200 Subject: [PATCH 0084/1190] Removed unwanted links --- src/Common/ZooKeeper/examples/CMakeLists.txt | 6 +++--- utils/zookeeper-cli/CMakeLists.txt | 2 +- utils/zookeeper-dump-tree/CMakeLists.txt | 2 +- utils/zookeeper-remove-by-list/CMakeLists.txt | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index 7a950634ac9..0a64683e4cb 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -1,8 +1,8 @@ clickhouse_add_executable(zkutil_test_commands zkutil_test_commands.cpp) -target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers) +target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log) clickhouse_add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) -target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers string_utils) +target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log string_utils) clickhouse_add_executable(zkutil_test_async zkutil_test_async.cpp) -target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers) +target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers) diff --git a/utils/zookeeper-cli/CMakeLists.txt b/utils/zookeeper-cli/CMakeLists.txt index 7c592a62c84..be8cf81320c 100644 --- a/utils/zookeeper-cli/CMakeLists.txt +++ b/utils/zookeeper-cli/CMakeLists.txt @@ -1,4 +1,4 @@ clickhouse_add_executable(clickhouse-zookeeper-cli zookeeper-cli.cpp ${ClickHouse_SOURCE_DIR}/src/Client/LineReader.cpp) -target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers) +target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper_no_log) diff --git a/utils/zookeeper-dump-tree/CMakeLists.txt b/utils/zookeeper-dump-tree/CMakeLists.txt index 027713aa26d..381daccc0f7 100644 --- a/utils/zookeeper-dump-tree/CMakeLists.txt +++ b/utils/zookeeper-dump-tree/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-dump-tree main.cpp ${SRCS}) -target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers boost::program_options) +target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log boost::program_options) diff --git a/utils/zookeeper-remove-by-list/CMakeLists.txt b/utils/zookeeper-remove-by-list/CMakeLists.txt index 4ee186f374a..01965413d29 100644 --- a/utils/zookeeper-remove-by-list/CMakeLists.txt +++ b/utils/zookeeper-remove-by-list/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-remove-by-list main.cpp ${SRCS}) -target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io clickhouse_parsers boost::program_options) +target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper_no_log boost::program_options) From 05d5f636ae20d30eac35ec03831f9d6fffb09685 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 16 Oct 2023 08:18:57 +0200 Subject: [PATCH 0085/1190] Remove Parsers directory from common_io --- src/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index de4094851e1..53d75f8446e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -84,7 +84,6 @@ add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io Common/SSH) add_headers_and_sources(clickhouse_common_io Compression) -add_headers_and_sources(clickhouse_common_io Parsers) add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/Archives) add_headers_and_sources(clickhouse_common_io IO/Resource) From 09072097ec33231cd0df95dc91cfd1317f662da2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 14:32:47 +0200 Subject: [PATCH 0086/1190] 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 0087/1190] 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 0088/1190] 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 0089/1190] 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 dc0a5b6f569970bc11cb44534c484422d1f75b7b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 17 Oct 2023 10:16:07 +0200 Subject: [PATCH 0090/1190] Fixed build issue with clang-tidy --- src/CMakeLists.txt | 32 ++++++++++++++++++-- src/Common/Config/CMakeLists.txt | 4 +-- src/Common/ZooKeeper/CMakeLists.txt | 6 ++-- src/Common/ZooKeeper/examples/CMakeLists.txt | 4 +-- 4 files changed, 36 insertions(+), 10 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 53d75f8446e..e2af29a7f4a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -83,13 +83,24 @@ add_subdirectory (Formats) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io Common/SSH) -add_headers_and_sources(clickhouse_common_io Compression) add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/Archives) add_headers_and_sources(clickhouse_common_io IO/Resource) add_headers_and_sources(clickhouse_common_io IO/S3) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) +#set(CLICKHOUSE_KEEPER_STANDALONE_SOURCES + + + +add_headers_and_sources(clickhouse_compression Compression) +add_headers_and_sources(clickhouse_compression Parsers) +add_headers_and_sources(clickhouse_compression Core) +add_headers_and_sources(clickhouse_compression Server) +add_headers_and_sources(clickhouse_compression Common/SSH) +add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhouse_compression_sources}) + + add_headers_and_sources(dbms Disks/IO) add_headers_and_sources(dbms Disks/ObjectStorages) if (TARGET ch_contrib::sqlite) @@ -286,7 +297,6 @@ set_source_files_properties( target_link_libraries (clickhouse_common_io PRIVATE - clickhouse_parsers string_utils widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} @@ -297,6 +307,24 @@ target_link_libraries (clickhouse_common_io ch_contrib::libdivide ) + +target_link_libraries (clickhouse_compression + PUBLIC + string_utils + pcg_random + clickhouse_parsers + PRIVATE + ch_contrib::lz4 + ch_contrib::llvm + clickhouse_grpc_protos + ch_contrib::nuraft + ch_contrib::ldap + ch_contrib::lber + ch_contrib::roaring + ch_contrib::ssh + ch_contrib::krb5 +) + # Use X86 AVX2/AVX512 instructions to accelerate filter operations set_source_files_properties( Columns/ColumnFixedString.cpp diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index e70a9944ce8..7202a72e6ff 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -11,6 +11,7 @@ add_library(clickhouse_common_config ${SRCS}) target_link_libraries(clickhouse_common_config PUBLIC clickhouse_common_zookeeper + clickhouse_compression common Poco::XML PRIVATE @@ -21,8 +22,7 @@ add_library(clickhouse_common_config_no_zookeeper_log ${SRCS}) target_link_libraries(clickhouse_common_config_no_zookeeper_log PUBLIC clickhouse_common_zookeeper_no_log - clickhouse_common_io - clickhouse_parsers + clickhouse_compression common Poco::XML PRIVATE diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index 0c9e0b79827..2d026521e72 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -9,8 +9,7 @@ add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} $ target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) target_link_libraries (clickhouse_common_zookeeper PUBLIC - clickhouse_common_io - clickhouse_parsers + clickhouse_compression common PRIVATE string_utils @@ -20,8 +19,7 @@ target_link_libraries (clickhouse_common_zookeeper add_library(clickhouse_common_zookeeper_no_log ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) target_link_libraries (clickhouse_common_zookeeper_no_log PUBLIC - clickhouse_common_io - clickhouse_parsers + clickhouse_compression common PRIVATE string_utils diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index 0a64683e4cb..a99fbe55dd8 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -2,7 +2,7 @@ clickhouse_add_executable(zkutil_test_commands zkutil_test_commands.cpp) target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper_no_log) clickhouse_add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) -target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log string_utils) +target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper_no_log clickhouse_compression string_utils) clickhouse_add_executable(zkutil_test_async zkutil_test_async.cpp) -target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log clickhouse_parsers) +target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log) From 7515853ad4b3e910f20df99038d706ef77ab2819 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 11:43:51 +0200 Subject: [PATCH 0091/1190] 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 e90f3f0323afafa7647fec01f6b920cc04ec419c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 17 Oct 2023 12:30:11 +0200 Subject: [PATCH 0092/1190] Fix build --- src/CMakeLists.txt | 27 +++++++++++------------- src/Common/Config/CMakeLists.txt | 2 -- src/Common/ZooKeeper/CMakeLists.txt | 2 ++ utils/zookeeper-dump-tree/CMakeLists.txt | 2 +- 4 files changed, 15 insertions(+), 18 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index e2af29a7f4a..5a77946a154 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -89,9 +89,6 @@ add_headers_and_sources(clickhouse_common_io IO/Resource) add_headers_and_sources(clickhouse_common_io IO/S3) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) -#set(CLICKHOUSE_KEEPER_STANDALONE_SOURCES - - add_headers_and_sources(clickhouse_compression Compression) add_headers_and_sources(clickhouse_compression Parsers) @@ -310,19 +307,19 @@ target_link_libraries (clickhouse_common_io target_link_libraries (clickhouse_compression PUBLIC - string_utils - pcg_random - clickhouse_parsers + string_utils + pcg_random + clickhouse_parsers + clickhouse_grpc_protos PRIVATE - ch_contrib::lz4 - ch_contrib::llvm - clickhouse_grpc_protos - ch_contrib::nuraft - ch_contrib::ldap - ch_contrib::lber - ch_contrib::roaring - ch_contrib::ssh - ch_contrib::krb5 + ch_contrib::lz4 + ch_contrib::llvm + ch_contrib::nuraft + ch_contrib::ldap + ch_contrib::lber + ch_contrib::roaring + ch_contrib::ssh + ch_contrib::krb5 ) # Use X86 AVX2/AVX512 instructions to accelerate filter operations diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index 7202a72e6ff..ec7bdd10196 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -11,7 +11,6 @@ add_library(clickhouse_common_config ${SRCS}) target_link_libraries(clickhouse_common_config PUBLIC clickhouse_common_zookeeper - clickhouse_compression common Poco::XML PRIVATE @@ -22,7 +21,6 @@ add_library(clickhouse_common_config_no_zookeeper_log ${SRCS}) target_link_libraries(clickhouse_common_config_no_zookeeper_log PUBLIC clickhouse_common_zookeeper_no_log - clickhouse_compression common Poco::XML PRIVATE diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index 2d026521e72..aa06375bd6a 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -9,6 +9,7 @@ add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} $ target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) target_link_libraries (clickhouse_common_zookeeper PUBLIC + clickhouse_common_io clickhouse_compression common PRIVATE @@ -19,6 +20,7 @@ target_link_libraries (clickhouse_common_zookeeper add_library(clickhouse_common_zookeeper_no_log ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) target_link_libraries (clickhouse_common_zookeeper_no_log PUBLIC + clickhouse_common_io clickhouse_compression common PRIVATE diff --git a/utils/zookeeper-dump-tree/CMakeLists.txt b/utils/zookeeper-dump-tree/CMakeLists.txt index 381daccc0f7..182cb65f194 100644 --- a/utils/zookeeper-dump-tree/CMakeLists.txt +++ b/utils/zookeeper-dump-tree/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (zookeeper-dump-tree main.cpp ${SRCS}) -target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log boost::program_options) +target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_io boost::program_options) From d837aa675f5ec56434aa7f58332fc4b922b1b9ba Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 13:14:28 +0200 Subject: [PATCH 0093/1190] 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 d5b70e641699a64e4c7d5c02d8952fc546081053 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 17 Oct 2023 16:10:45 +0200 Subject: [PATCH 0094/1190] Fix include on needed target --- src/CMakeLists.txt | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 5a77946a154..a0368012a6b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -278,6 +278,7 @@ target_include_directories (clickhouse_common_io PUBLIC "${ClickHouse_SOURCE_DIR if (TARGET ch_contrib::llvm) dbms_target_link_libraries (PUBLIC ch_contrib::llvm) + target_link_libraries (clickhouse_compression PUBLIC ch_contrib::llvm) endif () if (TARGET ch_contrib::gwp_asan) @@ -313,13 +314,7 @@ target_link_libraries (clickhouse_compression clickhouse_grpc_protos PRIVATE ch_contrib::lz4 - ch_contrib::llvm - ch_contrib::nuraft - ch_contrib::ldap - ch_contrib::lber ch_contrib::roaring - ch_contrib::ssh - ch_contrib::krb5 ) # Use X86 AVX2/AVX512 instructions to accelerate filter operations @@ -367,6 +362,7 @@ if (TARGET ch_contrib::crc32-vpmsum) if (TARGET ch_contrib::ssh) target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::ssh) + target_link_libraries(clickhouse_compression PUBLIC ch_contrib::ssh) endif() dbms_target_link_libraries(PUBLIC ch_contrib::abseil_swiss_tables) @@ -390,10 +386,12 @@ endif() if (TARGET ch_contrib::krb5) dbms_target_link_libraries(PRIVATE ch_contrib::krb5) + target_link_libraries (clickhouse_compression PRIVATE ch_contrib::krb5) endif() if (TARGET ch_contrib::nuraft) dbms_target_link_libraries(PUBLIC ch_contrib::nuraft) + target_link_libraries (clickhouse_compression PUBLIC ch_contrib::nuraft) endif() dbms_target_link_libraries ( @@ -463,6 +461,7 @@ endif () if (TARGET ch_contrib::ldap) dbms_target_link_libraries (PRIVATE ch_contrib::ldap ch_contrib::lber) + target_link_libraries (clickhouse_compression PRIVATE ch_contrib::ldap ch_contrib::lber) endif () dbms_target_link_libraries (PUBLIC ch_contrib::sparsehash) From 5d8b1cea910b3beb531e1e6122d2596f7197eae8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 19:19:19 +0200 Subject: [PATCH 0095/1190] 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 0096/1190] 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 0097/1190] 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 0098/1190] 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 0099/1190] 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 52571f62381edc736ba96a473e86b57277eb9130 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 18 Oct 2023 22:26:40 +0200 Subject: [PATCH 0100/1190] Updated to include clickhouse_grpc_protos only on needed targets --- src/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index a0368012a6b..1056a59ac45 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -311,7 +311,6 @@ target_link_libraries (clickhouse_compression string_utils pcg_random clickhouse_parsers - clickhouse_grpc_protos PRIVATE ch_contrib::lz4 ch_contrib::roaring @@ -471,6 +470,7 @@ endif () if (TARGET clickhouse_grpc_protos) dbms_target_link_libraries (PUBLIC clickhouse_grpc_protos) + target_link_libraries (clickhouse_compression PUBLIC clickhouse_grpc_protos) endif() if (TARGET ch_contrib::hdfs) From 89272e0925c91ed659b51741c58ddc364e149792 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 19 Oct 2023 11:23:56 +0200 Subject: [PATCH 0101/1190] 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 c39ca0b549828552b28cc3faecd7db8efb227765 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 19 Oct 2023 11:59:22 +0200 Subject: [PATCH 0102/1190] Fixed clang-tidy and updated to randomize compression for stress test --- docker/test/stateless/stress_tests.lib | 18 ++++++++++-------- src/CMakeLists.txt | 5 +++-- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 11945b68f70..d52ca94334e 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -53,23 +53,25 @@ 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_config_boolean_value { + function randomize_keeper_config_boolean_value { value=$(($RANDOM % 2)) - sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ + sudo cat /etc/clickhouse-server/config.d/$2.xml \ | sed "s|<$1>[01]|<$1>$value|" \ - > /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 + > /etc/clickhouse-server/config.d/$2.xml.tmp + sudo mv /etc/clickhouse-server/config.d/$2.xml.tmp /etc/clickhouse-server/config.d/$2.xml } # Randomize all Keeper feature flags - randomize_config_boolean_value filtered_list - randomize_config_boolean_value multi_read - randomize_config_boolean_value check_not_exists - randomize_config_boolean_value create_if_not_exists + 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 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 compressed_protocol zookeeper + # for clickhouse-server (via service) echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment # for clickhouse-client diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1056a59ac45..34b6a0cc249 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -93,7 +93,9 @@ list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_dele add_headers_and_sources(clickhouse_compression Compression) add_headers_and_sources(clickhouse_compression Parsers) add_headers_and_sources(clickhouse_compression Core) -add_headers_and_sources(clickhouse_compression Server) +#Included these specific files to avoid linking grpc +add_glob(clickhouse_compression_headers Server/ServerType.h) +add_glob(clickhouse_compression_sources Server/ServerType.cpp) add_headers_and_sources(clickhouse_compression Common/SSH) add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhouse_compression_sources}) @@ -470,7 +472,6 @@ endif () if (TARGET clickhouse_grpc_protos) dbms_target_link_libraries (PUBLIC clickhouse_grpc_protos) - target_link_libraries (clickhouse_compression PUBLIC clickhouse_grpc_protos) endif() if (TARGET ch_contrib::hdfs) From 7a096904ed001be79b88d2ef50d0c7c460a933bc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Oct 2023 15:40:32 +0200 Subject: [PATCH 0103/1190] 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 0104/1190] 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 0105/1190] 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 0106/1190] 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 0107/1190] 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 ede91f42e62b0985b5221e574c0d4e44191b069b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 23 Oct 2023 16:42:08 +0200 Subject: [PATCH 0108/1190] Addressed review comments --- .../operations/server-configuration-parameters/settings.md | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 6 ++---- src/Server/KeeperTCPHandler.cpp | 1 - 3 files changed, 3 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 0280985853d..5eaf2e1b836 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2407,7 +2407,7 @@ This section contains the following parameters: * hostname_levenshtein_distance - just like nearest_hostname, but it compares hostname in a levenshtein distance manner. * first_or_random - selects the first ZooKeeper node, if it's not available then randomly selects one of remaining ZooKeeper nodes. * round_robin - selects the first ZooKeeper node, if reconnection happens selects the next. -- `compressed_protocol` — If set to true, enables compression in keeper protocol. +- `compressed_protocol` — If set to true, enables compression in Keeper protocol. **Example configuration** diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 9e31178473e..70d22a0597b 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -355,6 +355,8 @@ ZooKeeper::ZooKeeper( } catch (...) { + /// If we get exception & compression is enabled, then its possible that keeper does not support compression, + /// try without compression if (use_compression) { use_compression = false; @@ -549,7 +551,6 @@ void ZooKeeper::sendHandshake() write(passwd); maybe_compressed_out->next(); - out->next(); } void ZooKeeper::receiveHandshake() @@ -599,7 +600,6 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) request.xid = AUTH_XID; request.write(*maybe_compressed_out); maybe_compressed_out->next(); - out->next(); int32_t length; XID read_xid; @@ -681,7 +681,6 @@ void ZooKeeper::sendThread() info.request->write(*maybe_compressed_out); maybe_compressed_out->next(); - out->next(); logOperationIfNeeded(info.request); @@ -699,7 +698,6 @@ void ZooKeeper::sendThread() request.xid = PING_XID; request.write(*maybe_compressed_out); maybe_compressed_out->next(); - out->next(); } ProfileEvents::increment(ProfileEvents::ZooKeeperBytesSent, out->count() - prev_bytes_sent); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 5b7ab44d6f9..84109453452 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -490,7 +490,6 @@ void KeeperTCPHandler::runImpl() response->write(*maybe_compressed_out); maybe_compressed_out->next(); - out->next(); log_long_operation("Sending response"); if (response->error == Coordination::Error::ZSESSIONEXPIRED) { From 0d27150948a54a9bf8513622111a4feb52476bbb Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 24 Oct 2023 13:09:57 +0000 Subject: [PATCH 0109/1190] 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 0110/1190] 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 0111/1190] 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 0112/1190] 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 0113/1190] 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 0114/1190] 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 0115/1190] 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 f6cc0d66520602059699d4b56a52c365ba951e53 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 26 Oct 2023 17:35:16 +0200 Subject: [PATCH 0116/1190] Fixed style --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 70d22a0597b..df1bf6786b8 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -355,7 +355,7 @@ ZooKeeper::ZooKeeper( } catch (...) { - /// If we get exception & compression is enabled, then its possible that keeper does not support compression, + /// If we get exception & compression is enabled, then its possible that keeper does not support compression, /// try without compression if (use_compression) { @@ -474,7 +474,7 @@ void ZooKeeper::connect( CompressionCodecFactory::instance().get( "ZSTD", {})); } - + original_index = static_cast(node.original_index); if (i != 0) From 960927a323c752aed7e59e97a3b5ca4de283f984 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 27 Oct 2023 14:53:54 +0200 Subject: [PATCH 0117/1190] Updated name to use_compression --- .../settings.md | 2 +- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 4 +- src/Common/ZooKeeper/ZooKeeperArgs.h | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Daemon/GitHash.generated.cpp | 8 +++ .../StorageSystemBuildOptions.generated.cpp | 72 +++++++++++++++++++ tests/config/config.d/zookeeper.xml | 2 +- .../configs/keeper_with_compression.xml | 2 +- .../configs/keeper_without_compression.xml | 2 +- 9 files changed, 88 insertions(+), 8 deletions(-) create mode 100644 src/Daemon/GitHash.generated.cpp create mode 100644 src/Storages/System/StorageSystemBuildOptions.generated.cpp diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 7cfe62bb337..9d8717f781a 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2423,7 +2423,7 @@ This section contains the following parameters: * hostname_levenshtein_distance - just like nearest_hostname, but it compares hostname in a levenshtein distance manner. * first_or_random - selects the first ZooKeeper node, if it's not available then randomly selects one of remaining ZooKeeper nodes. * round_robin - selects the first ZooKeeper node, if reconnection happens selects the next. -- `compressed_protocol` — If set to true, enables compression in Keeper protocol. +- `use_compression` — If set to true, enables compression in Keeper protocol. **Example configuration** diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index f4f6662221a..539ecdd0204 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -214,9 +214,9 @@ void ZooKeeperArgs::initFromKeeperSection(const Poco::Util::AbstractConfiguratio .max_sec = config.getUInt(config_name + "." + key + ".max"), }; } - else if (key == "compressed_protocol") + else if (key == "use_compression") { - compressed_protocol = config.getBool(config_name + "." + key); + use_compression = config.getBool(config_name + "." + key); } else throw KeeperException(Coordination::Error::ZBADARGUMENTS, "Unknown key {} in config file", key); diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h index 4f68288a1c7..ff44a1b191b 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.h +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -44,7 +44,7 @@ struct ZooKeeperArgs double recv_sleep_probability = 0.0; UInt64 send_sleep_ms = 0; UInt64 recv_sleep_ms = 0; - bool compressed_protocol = false; + bool use_compression = false; SessionLifetimeConfiguration fallback_session_lifetime = {}; DB::GetPriorityForLoadBalancing get_priority_load_balancing; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index df1bf6786b8..5e1201d6ba2 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -350,7 +350,7 @@ ZooKeeper::ZooKeeper( try { - use_compression = args.compressed_protocol; + use_compression = args.use_compression; connect(nodes, args.connection_timeout_ms * 1000); } catch (...) diff --git a/src/Daemon/GitHash.generated.cpp b/src/Daemon/GitHash.generated.cpp new file mode 100644 index 00000000000..9954416402a --- /dev/null +++ b/src/Daemon/GitHash.generated.cpp @@ -0,0 +1,8 @@ +/// This file was autogenerated by CMake + +#include + +String getGitHash() +{ + return "f6cc0d66520602059699d4b56a52c365ba951e53"; +} diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp b/src/Storages/System/StorageSystemBuildOptions.generated.cpp new file mode 100644 index 00000000000..0405bab4e02 --- /dev/null +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp @@ -0,0 +1,72 @@ +/// This file was autogenerated by CMake + +const char * auto_config_build[] +{ + "VERSION_FULL", "ClickHouse 23.10.1.1", + "VERSION_DESCRIBE", "v23.10.1.1-testing", + "VERSION_INTEGER", "23010001", + "SYSTEM", "Darwin", + "VERSION_GITHASH", "8f9a227de1f530cdbda52c145d41a6b0f1d29961", + "VERSION_REVISION", "54479", + "BUILD_TYPE", "Debug", + "SYSTEM_PROCESSOR", "arm64", + "CMAKE_VERSION", "3.26.4", + "C_COMPILER", "/Library/Developer/CommandLineTools/usr/bin/cc", + "C_COMPILER_VERSION", "13.1.6.13160021", + "CXX_COMPILER", "/Library/Developer/CommandLineTools/usr/bin/c++", + "CXX_COMPILER_VERSION", "13.1.6.13160021", + "C_FLAGS", " -fdiagnostics-color=always -Xclang -fuse-ctor-homing -gdwarf-aranges -pipe -march=armv8.2-a+simd+crypto+dotprod+ssbs+rcpc -fasynchronous-unwind-tables -falign-functions=32 -fdiagnostics-absolute-paths -Weverything -Wpedantic -Wno-zero-length-array -Wno-c++98-compat-pedantic -Wno-c++98-compat -Wno-c++20-compat -Wno-sign-conversion -Wno-implicit-int-conversion -Wno-implicit-int-float-conversion -Wno-ctad-maybe-unsupported -Wno-disabled-macro-expansion -Wno-documentation-unknown-command -Wno-double-promotion -Wno-exit-time-destructors -Wno-float-equal -Wno-global-constructors -Wno-missing-prototypes -Wno-missing-variable-declarations -Wno-padded -Wno-switch-enum -Wno-undefined-func-template -Wno-unused-template -Wno-vla -Wno-weak-template-vtables -Wno-weak-vtables -Wno-thread-safety-negative -g -O0 -g -gdwarf-4 ", + "CXX_FLAGS", " -fdiagnostics-color=always -Xclang -fuse-ctor-homing -fsized-deallocation -gdwarf-aranges -pipe -march=armv8.2-a+simd+crypto+dotprod+ssbs+rcpc -fasynchronous-unwind-tables -falign-functions=32 -stdlib=libc++ -fdiagnostics-absolute-paths -fstrict-vtable-pointers -Wall -Wextra -Weverything -Wpedantic -Wno-zero-length-array -Wno-c++98-compat-pedantic -Wno-c++98-compat -Wno-c++20-compat -Wno-sign-conversion -Wno-implicit-int-conversion -Wno-implicit-int-float-conversion -Wno-ctad-maybe-unsupported -Wno-disabled-macro-expansion -Wno-documentation-unknown-command -Wno-double-promotion -Wno-exit-time-destructors -Wno-float-equal -Wno-global-constructors -Wno-missing-prototypes -Wno-missing-variable-declarations -Wno-padded -Wno-switch-enum -Wno-undefined-func-template -Wno-unused-template -Wno-vla -Wno-weak-template-vtables -Wno-weak-vtables -Wno-thread-safety-negative -g -O0 -g -gdwarf-4 -D_LIBCPP_DEBUG=0", + "LINK_FLAGS", " --ld-path=/usr/bin/ld -Wl,-U,_inside_main ", + "BUILD_COMPILE_DEFINITIONS", "", + "USE_EMBEDDED_COMPILER", "", + "USE_GLIBC_COMPATIBILITY", "", + "USE_JEMALLOC", "ON", + "USE_ICU", "", + "USE_H3", "1", + "USE_MYSQL", "", + "USE_RDKAFKA", "1", + "USE_CAPNP", "1", + "USE_BASE64", "1", + "USE_HDFS", "", + "USE_SNAPPY", "1", + "USE_PARQUET", "1", + "USE_PROTOBUF", "1", + "USE_BROTLI", "1", + "USE_SSL", "1", + "OPENSSL_VERSION", "1.1.1g", + "OPENSSL_IS_BORING_SSL", "1", + "USE_VECTORSCAN", "ON", + "USE_SIMDJSON", "1", + "USE_ODBC", "", + "USE_GRPC", "1", + "USE_LDAP", "1", + "TZDATA_VERSION", "2023c", + "USE_KRB5", "1", + "USE_FILELOG", "", + "USE_BZIP2", "1", + "USE_AMQPCPP", "1", + "USE_ROCKSDB", "1", + "USE_NURAFT", "1", + "USE_NLP", "1", + "USE_LIBURING", "", + "USE_SQLITE", "1", + "USE_LIBPQXX", "1", + "USE_AZURE_BLOB_STORAGE", "", + "USE_AWS_S3", "1", + "USE_CASSANDRA", "1", + "USE_YAML_CPP", "1", + "USE_SENTRY", "", + "USE_DATASKETCHES", "1", + "USE_AVRO", "1", + "USE_ARROW", "1", + "USE_ORC", "1", + "USE_MSGPACK", "1", + "USE_QPL", "", + "GIT_HASH", "f6cc0d66520602059699d4b56a52c365ba951e53", + "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(49507_Support_compression_for_keeper_protocol)IRjaNsZIL9Yh7FQ4", + "GIT_DATE", "2023-10-26 15:35:16 +0000", + "GIT_COMMIT_SUBJECT", R"Gi17KJMlbGCjErEN(Fixed style)Gi17KJMlbGCjErEN", + + nullptr, nullptr +}; diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index 583f23d449f..a54149e6617 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -2,7 +2,7 @@ random - true + true 127.0.0.1 9181 diff --git a/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml b/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml index c84475cf481..566eb93af36 100644 --- a/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml +++ b/tests/integration/test_keeper_compression/configs/keeper_with_compression.xml @@ -1,6 +1,6 @@ - true + true node1 9181 diff --git a/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml b/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml index 341b36333d0..e328dd43b83 100644 --- a/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml +++ b/tests/integration/test_keeper_compression/configs/keeper_without_compression.xml @@ -1,6 +1,6 @@ - false + false node1 9181 From 13c90c0b869bb6c9ba448496922647aacf55c0e4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 27 Oct 2023 14:56:25 +0200 Subject: [PATCH 0118/1190] Removed unwanted file and added chanegs to stress_test --- docker/test/stateless/stress_tests.lib | 2 +- .../StorageSystemBuildOptions.generated.cpp | 72 ------------------- 2 files changed, 1 insertion(+), 73 deletions(-) delete mode 100644 src/Storages/System/StorageSystemBuildOptions.generated.cpp diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index e3964b7895a..eaa251c0086 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -72,7 +72,7 @@ function configure() 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 compressed_protocol zookeeper + randomize_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 diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp b/src/Storages/System/StorageSystemBuildOptions.generated.cpp deleted file mode 100644 index 0405bab4e02..00000000000 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp +++ /dev/null @@ -1,72 +0,0 @@ -/// This file was autogenerated by CMake - -const char * auto_config_build[] -{ - "VERSION_FULL", "ClickHouse 23.10.1.1", - "VERSION_DESCRIBE", "v23.10.1.1-testing", - "VERSION_INTEGER", "23010001", - "SYSTEM", "Darwin", - "VERSION_GITHASH", "8f9a227de1f530cdbda52c145d41a6b0f1d29961", - "VERSION_REVISION", "54479", - "BUILD_TYPE", "Debug", - "SYSTEM_PROCESSOR", "arm64", - "CMAKE_VERSION", "3.26.4", - "C_COMPILER", "/Library/Developer/CommandLineTools/usr/bin/cc", - "C_COMPILER_VERSION", "13.1.6.13160021", - "CXX_COMPILER", "/Library/Developer/CommandLineTools/usr/bin/c++", - "CXX_COMPILER_VERSION", "13.1.6.13160021", - "C_FLAGS", " -fdiagnostics-color=always -Xclang -fuse-ctor-homing -gdwarf-aranges -pipe -march=armv8.2-a+simd+crypto+dotprod+ssbs+rcpc -fasynchronous-unwind-tables -falign-functions=32 -fdiagnostics-absolute-paths -Weverything -Wpedantic -Wno-zero-length-array -Wno-c++98-compat-pedantic -Wno-c++98-compat -Wno-c++20-compat -Wno-sign-conversion -Wno-implicit-int-conversion -Wno-implicit-int-float-conversion -Wno-ctad-maybe-unsupported -Wno-disabled-macro-expansion -Wno-documentation-unknown-command -Wno-double-promotion -Wno-exit-time-destructors -Wno-float-equal -Wno-global-constructors -Wno-missing-prototypes -Wno-missing-variable-declarations -Wno-padded -Wno-switch-enum -Wno-undefined-func-template -Wno-unused-template -Wno-vla -Wno-weak-template-vtables -Wno-weak-vtables -Wno-thread-safety-negative -g -O0 -g -gdwarf-4 ", - "CXX_FLAGS", " -fdiagnostics-color=always -Xclang -fuse-ctor-homing -fsized-deallocation -gdwarf-aranges -pipe -march=armv8.2-a+simd+crypto+dotprod+ssbs+rcpc -fasynchronous-unwind-tables -falign-functions=32 -stdlib=libc++ -fdiagnostics-absolute-paths -fstrict-vtable-pointers -Wall -Wextra -Weverything -Wpedantic -Wno-zero-length-array -Wno-c++98-compat-pedantic -Wno-c++98-compat -Wno-c++20-compat -Wno-sign-conversion -Wno-implicit-int-conversion -Wno-implicit-int-float-conversion -Wno-ctad-maybe-unsupported -Wno-disabled-macro-expansion -Wno-documentation-unknown-command -Wno-double-promotion -Wno-exit-time-destructors -Wno-float-equal -Wno-global-constructors -Wno-missing-prototypes -Wno-missing-variable-declarations -Wno-padded -Wno-switch-enum -Wno-undefined-func-template -Wno-unused-template -Wno-vla -Wno-weak-template-vtables -Wno-weak-vtables -Wno-thread-safety-negative -g -O0 -g -gdwarf-4 -D_LIBCPP_DEBUG=0", - "LINK_FLAGS", " --ld-path=/usr/bin/ld -Wl,-U,_inside_main ", - "BUILD_COMPILE_DEFINITIONS", "", - "USE_EMBEDDED_COMPILER", "", - "USE_GLIBC_COMPATIBILITY", "", - "USE_JEMALLOC", "ON", - "USE_ICU", "", - "USE_H3", "1", - "USE_MYSQL", "", - "USE_RDKAFKA", "1", - "USE_CAPNP", "1", - "USE_BASE64", "1", - "USE_HDFS", "", - "USE_SNAPPY", "1", - "USE_PARQUET", "1", - "USE_PROTOBUF", "1", - "USE_BROTLI", "1", - "USE_SSL", "1", - "OPENSSL_VERSION", "1.1.1g", - "OPENSSL_IS_BORING_SSL", "1", - "USE_VECTORSCAN", "ON", - "USE_SIMDJSON", "1", - "USE_ODBC", "", - "USE_GRPC", "1", - "USE_LDAP", "1", - "TZDATA_VERSION", "2023c", - "USE_KRB5", "1", - "USE_FILELOG", "", - "USE_BZIP2", "1", - "USE_AMQPCPP", "1", - "USE_ROCKSDB", "1", - "USE_NURAFT", "1", - "USE_NLP", "1", - "USE_LIBURING", "", - "USE_SQLITE", "1", - "USE_LIBPQXX", "1", - "USE_AZURE_BLOB_STORAGE", "", - "USE_AWS_S3", "1", - "USE_CASSANDRA", "1", - "USE_YAML_CPP", "1", - "USE_SENTRY", "", - "USE_DATASKETCHES", "1", - "USE_AVRO", "1", - "USE_ARROW", "1", - "USE_ORC", "1", - "USE_MSGPACK", "1", - "USE_QPL", "", - "GIT_HASH", "f6cc0d66520602059699d4b56a52c365ba951e53", - "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(49507_Support_compression_for_keeper_protocol)IRjaNsZIL9Yh7FQ4", - "GIT_DATE", "2023-10-26 15:35:16 +0000", - "GIT_COMMIT_SUBJECT", R"Gi17KJMlbGCjErEN(Fixed style)Gi17KJMlbGCjErEN", - - nullptr, nullptr -}; From 940d099e84d92eaaacaa96682c5a94b26f7a782c Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 27 Oct 2023 16:50:34 -0700 Subject: [PATCH 0119/1190] 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 0b301f73c11935c1bf25b073e1f3809a955dd76a Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Sat, 28 Oct 2023 18:28:16 +0000 Subject: [PATCH 0120/1190] initial add keeper az info in /keeper/availabilty-zone Signed-off-by: Jianfei Hu --- src/Coordination/KeeperContext.cpp | 17 +++++++++++++++++ src/Coordination/KeeperContext.h | 5 ++++- src/Coordination/KeeperStorage.cpp | 8 ++++++++ src/IO/S3/Credentials.cpp | 1 + 4 files changed, 30 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 25bfb6c6384..bc891a7a1f5 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -9,6 +9,13 @@ #include #include +#if USE_AWS_S3 + +#include +#include +#include +#endif + namespace DB { @@ -30,6 +37,11 @@ KeeperContext::KeeperContext(bool standalone_keeper_) /// for older clients, the default is equivalent to WITH_MULTI_READ version system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); + + #if USE_AWS_S3 + auto metadata_client = S3::InitEC2MetadataClient(Aws::Client::ClientConfiguration{}); + running_availability_zone = metadata_client->getCurrentAvailabilityZone(); + #endif } void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_) @@ -204,6 +216,11 @@ void KeeperContext::setStateFileDisk(DiskPtr disk) state_file_storage = std::move(disk); } +std::string KeeperContext::getRunningAvailabilityZone() const +{ + return running_availability_zone; +} + const std::unordered_map & KeeperContext::getSystemNodesWithData() const { return system_nodes_with_data; diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index ba1a81b4423..ae55f0cd844 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -3,7 +3,6 @@ #include #include #include - #include #include @@ -54,6 +53,8 @@ public: constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; } + std::string getRunningAvailabilityZone() const; + private: /// local disk defined using path or disk name using Storage = std::variant; @@ -89,6 +90,8 @@ private: KeeperFeatureFlags feature_flags; KeeperDispatcher * dispatcher{nullptr}; + + std::string running_availability_zone; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 5534f4c3699..39bc0b91137 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1097,6 +1097,14 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); + /// AZ node information is a special case not persisted in the storage, so we handle it first. + if (request.path == "/keeper/availbility-zone") + { + response.data = storage.keeper_context->getRunningAvailabilityZone(); + response.error = Coordination::Error::ZOK; + return response_ptr; + } + if constexpr (!local) { if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 1eea167e1b9..ccb1ff6f55b 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -704,6 +704,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( aws_client_configuration.retryStrategy = std::make_shared(1, 1000); + // Code that we want to copy. auto ec2_metadata_client = InitEC2MetadataClient(aws_client_configuration); auto config_loader = std::make_shared(ec2_metadata_client, !credentials_configuration.use_insecure_imds_request); From 70e3dd808cc3f087504892d18a9e61eb6f948151 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 02:07:24 +0100 Subject: [PATCH 0121/1190] 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 dec8efc39ea26e5417a4094b3be6d1e82ee48bcd Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Sun, 29 Oct 2023 02:04:38 +0000 Subject: [PATCH 0122/1190] make az as attribute of keepersnapshotmangers3. empty config not working. https://pastila.nl/?00048e4f/d56af90d13f2d871e30dfb1db9660266#yff6d8HVQm0blQUUiZg8SQ== Signed-off-by: Jianfei Hu --- src/Coordination/KeeperContext.cpp | 18 ++--- src/Coordination/KeeperContext.h | 2 +- src/Coordination/KeeperDispatcher.cpp | 4 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 2 + src/Coordination/KeeperSnapshotManagerS3.h | 6 ++ src/IO/S3/Credentials.cpp | 75 +++++++++++--------- src/IO/S3/Credentials.h | 3 + 7 files changed, 61 insertions(+), 49 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index bc891a7a1f5..4ce8b2c732a 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -9,13 +9,6 @@ #include #include -#if USE_AWS_S3 - -#include -#include -#include -#endif - namespace DB { @@ -37,16 +30,15 @@ KeeperContext::KeeperContext(bool standalone_keeper_) /// for older clients, the default is equivalent to WITH_MULTI_READ version system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); - - #if USE_AWS_S3 - auto metadata_client = S3::InitEC2MetadataClient(Aws::Client::ClientConfiguration{}); - running_availability_zone = metadata_client->getCurrentAvailabilityZone(); - #endif } -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_) +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, std::string availability_zone) { dispatcher = dispatcher_; + + running_availability_zone = availability_zone; + LOG_INFO(&Poco::Logger::get("KeeperContext"), "Initialize the KeeperContext with availability zone: '{}'. ", running_availability_zone); + 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 ae55f0cd844..a7ef7d9e937 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_); + void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, std::string availability_zone); Phase getServerState() const; void setServerState(Phase server_state_); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index d93bdb21db0..5807ab8d188 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -11,6 +11,7 @@ #include #include #include +#include "IO/S3/Credentials.h" #include #include @@ -368,9 +369,8 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); snapshot_s3.startup(config, macros); - keeper_context = std::make_shared(standalone_keeper); - keeper_context->initialize(config, this); + keeper_context->initialize(config, this, snapshot_s3.getAvaibilityZone()); server = std::make_unique( configuration_and_settings, diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 302e05c8418..783a8ed8c20 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -98,6 +98,8 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo client_configuration.endpointOverride = new_uri.endpoint; + avaibility_zone = S3::determineAvailabilityZone(client_configuration); + auto client = S3::ClientFactory::instance().create( client_configuration, new_uri.is_virtual_hosted_style, diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index e17cf5a1cfb..25684d4d054 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -9,6 +9,7 @@ #include #if USE_AWS_S3 +#include #include #include @@ -29,6 +30,8 @@ public: void updateS3Configuration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void uploadSnapshot(const SnapshotFileInfo & file_info, bool async_upload = true); + std::string getAvaibilityZone() const { return avaibility_zone; } + /// 'macros' are used to substitute macros in endpoint of disks void startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void shutdown(); @@ -50,6 +53,7 @@ private: UUID uuid; std::shared_ptr getSnapshotS3Client() const; + std::string avaibility_zone; void uploadSnapshotImpl(const SnapshotFileInfo & snapshot_file_info); @@ -65,6 +69,8 @@ public: void updateS3Configuration(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} void uploadSnapshot(const SnapshotFileInfo &, [[maybe_unused]] bool async_upload = true) {} + std::string getAvaibilityZone() const { return ""; } + void startup(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} void shutdown() {} diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index ccb1ff6f55b..15be25a82d7 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -584,6 +584,40 @@ Aws::String SSOCredentialsProvider::loadAccessTokenFile(const Aws::String & sso_ } } +Aws::Client::ClientConfiguration getAwsClientConfig(const DB::S3::PocoHTTPClientConfiguration & configuration) +{ + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( + configuration.region, + configuration.remote_host_filter, + configuration.s3_max_redirects, + configuration.s3_retry_attempts, + configuration.enable_s3_requests_logging, + configuration.for_disk_s3, + configuration.get_request_throttler, + configuration.put_request_throttler, + Aws::Http::SchemeMapper::ToString(Aws::Http::Scheme::HTTP)); + + /// See MakeDefaultHttpResourceClientConfiguration(). + /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside + /// of contrib/aws/aws-cpp-sdk-core/source/internal/AWSHttpResourceClient.cpp + aws_client_configuration.maxConnections = 2; + + /// Explicitly set the proxy settings to empty/zero to avoid relying on defaults that could potentially change + /// in the future. + aws_client_configuration.proxyHost = ""; + aws_client_configuration.proxyUserName = ""; + aws_client_configuration.proxyPassword = ""; + aws_client_configuration.proxyPort = 0; + + /// EC2MetadataService throttles by delaying the response so the service client should set a large read timeout. + /// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds. + aws_client_configuration.connectTimeoutMs = 1000; + aws_client_configuration.requestTimeoutMs = 1000; + + aws_client_configuration.retryStrategy = std::make_shared(1, 1000); + return aws_client_configuration; +} + S3CredentialsProviderChain::S3CredentialsProviderChain( const DB::S3::PocoHTTPClientConfiguration & configuration, const Aws::Auth::AWSCredentials & credentials, @@ -674,38 +708,8 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( } else if (Aws::Utils::StringUtils::ToLower(ec2_metadata_disabled.c_str()) != "true") { - DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( - configuration.region, - configuration.remote_host_filter, - configuration.s3_max_redirects, - configuration.s3_retry_attempts, - configuration.enable_s3_requests_logging, - configuration.for_disk_s3, - configuration.get_request_throttler, - configuration.put_request_throttler, - Aws::Http::SchemeMapper::ToString(Aws::Http::Scheme::HTTP)); - - /// See MakeDefaultHttpResourceClientConfiguration(). - /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside - /// of contrib/aws/aws-cpp-sdk-core/source/internal/AWSHttpResourceClient.cpp - aws_client_configuration.maxConnections = 2; - - /// Explicitly set the proxy settings to empty/zero to avoid relying on defaults that could potentially change - /// in the future. - aws_client_configuration.proxyHost = ""; - aws_client_configuration.proxyUserName = ""; - aws_client_configuration.proxyPassword = ""; - aws_client_configuration.proxyPort = 0; - - /// EC2MetadataService throttles by delaying the response so the service client should set a large read timeout. - /// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds. - aws_client_configuration.connectTimeoutMs = 1000; - aws_client_configuration.requestTimeoutMs = 1000; - - aws_client_configuration.retryStrategy = std::make_shared(1, 1000); - - // Code that we want to copy. - auto ec2_metadata_client = InitEC2MetadataClient(aws_client_configuration); + auto aws_client_config = getAwsClientConfig(configuration); + auto ec2_metadata_client = InitEC2MetadataClient(aws_client_config); auto config_loader = std::make_shared(ec2_metadata_client, !credentials_configuration.use_insecure_imds_request); AddProvider(std::make_shared(config_loader)); @@ -718,8 +722,13 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( AddProvider(std::make_shared()); } +std::string determineAvailabilityZone(const DB::S3::PocoHTTPClientConfiguration & configuration) +{ + auto aws_client_configuration = getAwsClientConfig(configuration); + auto ec2_metadata_client = InitEC2MetadataClient(aws_client_configuration); + return ec2_metadata_client->getCurrentAvailabilityZone(); } } - +} #endif diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 0243e8e4986..c81d06f9fd4 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -166,6 +166,9 @@ struct CredentialsConfiguration bool no_sign_request = false; }; + +std::string determineAvailabilityZone(const DB::S3::PocoHTTPClientConfiguration & configuration); + class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain { public: From 3142921bb4dcb8b7169f7d32a05110c9a5baa351 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 11:15:11 +0100 Subject: [PATCH 0123/1190] 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 0124/1190] 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 0125/1190] 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 0126/1190] 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 0127/1190] 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 0128/1190] 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 0129/1190] 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 0130/1190] 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 0131/1190] 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 0132/1190] 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 0133/1190] 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 0134/1190] 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 0135/1190] 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 0136/1190] 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 0137/1190] 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 0138/1190] 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 0139/1190] 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 0140/1190] 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 0141/1190] 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 0142/1190] 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 0143/1190] 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 0144/1190] 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 43cd030d8d5a7a2be883f13b34cf78b106fd6966 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 30 Oct 2023 20:01:02 +0100 Subject: [PATCH 0145/1190] Removed generated file --- src/Daemon/GitHash.generated.cpp | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 src/Daemon/GitHash.generated.cpp diff --git a/src/Daemon/GitHash.generated.cpp b/src/Daemon/GitHash.generated.cpp deleted file mode 100644 index 9954416402a..00000000000 --- a/src/Daemon/GitHash.generated.cpp +++ /dev/null @@ -1,8 +0,0 @@ -/// This file was autogenerated by CMake - -#include - -String getGitHash() -{ - return "f6cc0d66520602059699d4b56a52c365ba951e53"; -} From f53fdbeeadf7a2be2fa962e0ee0be91628348b93 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Oct 2023 19:11:03 +0000 Subject: [PATCH 0146/1190] 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 6d72873bdb1f16165f12f7c281d5ee8d407e1bf9 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 31 Oct 2023 02:34:50 +0000 Subject: [PATCH 0147/1190] use poco better works. Signed-off-by: Jianfei Hu --- src/Coordination/KeeperDispatcher.cpp | 2 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 3 +- src/IO/S3/Client.cpp | 4 + src/IO/S3/Credentials.cpp | 93 +++++++++++++++----- src/IO/S3/Credentials.h | 2 +- 5 files changed, 78 insertions(+), 26 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 5807ab8d188..e62410e9907 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -370,7 +370,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); keeper_context = std::make_shared(standalone_keeper); - keeper_context->initialize(config, this, snapshot_s3.getAvaibilityZone()); + keeper_context->initialize(config, this, DB::S3::AWSEC2MetadataClient::getCurrentAvailabilityZone()); server = std::make_unique( configuration_and_settings, diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 783a8ed8c20..0204c9dd019 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -98,7 +98,8 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo client_configuration.endpointOverride = new_uri.endpoint; - avaibility_zone = S3::determineAvailabilityZone(client_configuration); + // avaibility_zone = S3::AWSEC2MetadataClient::getCurrentAvailabilityZone(); + avaibility_zone = "321"; auto client = S3::ClientFactory::instance().create( client_configuration, diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 44ab01ba959..ea5af8b5fdc 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -1,4 +1,5 @@ #include +#include #if USE_AWS_S3 @@ -797,6 +798,9 @@ ClientFactory::ClientFactory() Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared(false)); Aws::Http::SetHttpClientFactory(std::make_shared()); + LOG_INFO(&Poco::Logger::get("Application"), "Factory is set"); + // Aws::Http::SetHttpClientFactory(const std::shared_ptr &factory); + // LOG_INFO(&Poco::Logger::get("JianfeiDebug"), "Factory is set {} ", Aws::Http::GetHttpClientFactory() == nullptr); } ClientFactory::~ClientFactory() diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 15be25a82d7..4c63b53b986 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -11,6 +11,7 @@ # include # include +# include # include # include @@ -22,6 +23,12 @@ # include # include +#include +#include +#include +#include +#include + namespace DB { @@ -151,28 +158,73 @@ Aws::String AWSEC2MetadataClient::getDefaultCredentialsSecurely() const return GetResourceWithAWSWebServiceResult(credentials_request).GetPayload(); } -Aws::String AWSEC2MetadataClient::getCurrentAvailabilityZone() const +Aws::String usePocoBetter() { - 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); + Poco::URI uri("http://169.254.169.254/latest/meta-data/placement/availability-zone"); + + Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort()); + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri.getPath()); + Poco::Net::HTTPResponse response; - 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)); + try { + session.sendRequest(request); + std::istream& rs = session.receiveResponse(response); - profile_request->SetHeaderValue(EC2_IMDS_TOKEN_HEADER, token); - profile_request->SetUserAgent(user_agent_string); + if (response.getStatus() == Poco::Net::HTTPResponse::HTTP_OK) { + std::string response_data; + Poco::StreamCopier::copyToString(rs, response_data); + std::cout << "Response Data: " << response_data << std::endl; + return response_data; + } else { + std::cerr << "HTTP Request failed with status code: " << response.getStatus() << std::endl; + } + } catch (Poco::Exception& ex) { + std::cerr << "Poco Exception: " << ex.displayText() << std::endl; + } + return ""; +} - 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()); +Aws::String AWSEC2MetadataClient::getCurrentAvailabilityZone() +{ + return usePocoBetter(); - return Aws::Utils::StringUtils::Trim(result.GetPayload().c_str()); + // String user_agent_string = awsComputeUserAgentString(); + // String endpoint = "http://169.254.169.254"; + // const String url = endpoint + EC2_AVAILABILITY_ZONE_RESOURCE; + + // // profile_request->SetUserAgent(user_agent_string); + + // // instance() singleton doing some registry work, need to use the same method. + // DB::RemoteHostFilter remote_host_filter; + // Aws::Client::ClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( + // "some-region", + // remote_host_filter, + // /* s3_max_redirects = */ 100, + // /* s3_retry_attempts = */ 0, + // /* enable_s3_requests_logging = */ true, + // /* for_disk_s3 = */ false, + // /* get_request_throttler = */ {}, + // /* put_request_throttler = */ {} + // ); + + // NOTE: now need to figure out the same approach for factory class. + // auto factory = std::make_shared(); + // Aws::Http::SetHttpClientFactory(factory); + // LOG_INFO(&Poco::Logger::get("Application"), "Trying to create"); + // auto client = Aws::Http::CreateHttpClient(client_configuration); + + /// NOTE: check here, even before it's okay! or could test whether it's okay or not. if not, reproduce the successful keeper snapshot case. + /// And see what's the delta. + // std::shared_ptr profile_request( + // Aws::Http::CreateHttpRequest(url, Aws::Http::HttpMethod::HTTP_GET, Aws::Utils::Stream::DefaultResponseStreamFactoryMethod)); + + // const auto result = client->MakeRequest(profile_request); + // if (result->GetResponseCode() != Aws::Http::HttpResponseCode::OK) + // return "failed"; + // result->GetResponseBody(); + // Aws::IStreamBufIterator eos; + // return Aws::String(Aws::IStreamBufIterator(result->GetResponseBody()), eos); + // return "123"; } std::pair AWSEC2MetadataClient::getEC2MetadataToken(const std::string & user_agent_string) const @@ -722,12 +774,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( AddProvider(std::make_shared()); } -std::string determineAvailabilityZone(const DB::S3::PocoHTTPClientConfiguration & configuration) -{ - auto aws_client_configuration = getAwsClientConfig(configuration); - auto ec2_metadata_client = InitEC2MetadataClient(aws_client_configuration); - return ec2_metadata_client->getCurrentAvailabilityZone(); -} + } } diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index c81d06f9fd4..3a669baa7aa 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -50,7 +50,7 @@ public: virtual Aws::String getCurrentRegion() const; - virtual Aws::String getCurrentAvailabilityZone() const; + static Aws::String getCurrentAvailabilityZone(); private: std::pair getEC2MetadataToken(const std::string & user_agent_string) const; 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 0148/1190] 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 0149/1190] 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 573054d75542ed1c6809975077666e013557313e Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 31 Oct 2023 04:43:44 +0000 Subject: [PATCH 0150/1190] add new code and rework. Signed-off-by: Jianfei Hu --- src/Common/ErrorCodes.cpp | 1 + src/Coordination/KeeperConstants.h | 1 + src/Coordination/KeeperDispatcher.cpp | 9 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 3 - src/Coordination/KeeperSnapshotManagerS3.h | 3 - src/Coordination/KeeperStorage.cpp | 5 +- src/IO/S3/Credentials.cpp | 153 ++++++++++--------- src/IO/S3/Credentials.h | 9 +- 8 files changed, 99 insertions(+), 85 deletions(-) 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/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index e62410e9907..66a25ab8231 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -370,7 +370,14 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); keeper_context = std::make_shared(standalone_keeper); - keeper_context->initialize(config, this, DB::S3::AWSEC2MetadataClient::getCurrentAvailabilityZone()); + String availability_zone; + try{ + availability_zone = DB::S3::getRunningAvailabilityZone(); + } catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + keeper_context->initialize(config, this, availability_zone); server = std::make_unique( configuration_and_settings, diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 0204c9dd019..302e05c8418 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -98,9 +98,6 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo client_configuration.endpointOverride = new_uri.endpoint; - // avaibility_zone = S3::AWSEC2MetadataClient::getCurrentAvailabilityZone(); - avaibility_zone = "321"; - auto client = S3::ClientFactory::instance().create( client_configuration, new_uri.is_virtual_hosted_style, diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index 25684d4d054..97126e1ddb8 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -30,8 +30,6 @@ public: void updateS3Configuration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void uploadSnapshot(const SnapshotFileInfo & file_info, bool async_upload = true); - std::string getAvaibilityZone() const { return avaibility_zone; } - /// 'macros' are used to substitute macros in endpoint of disks void startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void shutdown(); @@ -53,7 +51,6 @@ private: UUID uuid; std::shared_ptr getSnapshotS3Client() const; - std::string avaibility_zone; void uploadSnapshotImpl(const SnapshotFileInfo & snapshot_file_info); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 39bc0b91137..4fb2b6b1834 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)) @@ -1098,7 +1099,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); /// AZ node information is a special case not persisted in the storage, so we handle it first. - if (request.path == "/keeper/availbility-zone") + if (request.path == Coordination::keeper_availability_zone_path) { response.data = storage.keeper_context->getRunningAvailabilityZone(); response.error = Coordination::Error::ZOK; diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 4c63b53b986..a5ec210f284 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,4 +1,8 @@ +#include #include +#include +#include + #if USE_AWS_S3 @@ -23,6 +27,11 @@ # include # include +/// TODO: these changes do not need to be under AWS_S3. consider to remove or make it separate file libraries. +#include +#include + + #include #include #include @@ -36,6 +45,8 @@ namespace DB namespace ErrorCodes { extern const int AWS_ERROR; + extern const int GCP_ERROR; + extern const int UNSUPPORTED_METHOD; } namespace S3 @@ -158,75 +169,6 @@ Aws::String AWSEC2MetadataClient::getDefaultCredentialsSecurely() const return GetResourceWithAWSWebServiceResult(credentials_request).GetPayload(); } -Aws::String usePocoBetter() -{ - Poco::URI uri("http://169.254.169.254/latest/meta-data/placement/availability-zone"); - - Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort()); - Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, uri.getPath()); - Poco::Net::HTTPResponse response; - - try { - session.sendRequest(request); - std::istream& rs = session.receiveResponse(response); - - if (response.getStatus() == Poco::Net::HTTPResponse::HTTP_OK) { - std::string response_data; - Poco::StreamCopier::copyToString(rs, response_data); - std::cout << "Response Data: " << response_data << std::endl; - return response_data; - } else { - std::cerr << "HTTP Request failed with status code: " << response.getStatus() << std::endl; - } - } catch (Poco::Exception& ex) { - std::cerr << "Poco Exception: " << ex.displayText() << std::endl; - } - return ""; -} - -Aws::String AWSEC2MetadataClient::getCurrentAvailabilityZone() -{ - return usePocoBetter(); - - // String user_agent_string = awsComputeUserAgentString(); - // String endpoint = "http://169.254.169.254"; - // const String url = endpoint + EC2_AVAILABILITY_ZONE_RESOURCE; - - // // profile_request->SetUserAgent(user_agent_string); - - // // instance() singleton doing some registry work, need to use the same method. - // DB::RemoteHostFilter remote_host_filter; - // Aws::Client::ClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - // "some-region", - // remote_host_filter, - // /* s3_max_redirects = */ 100, - // /* s3_retry_attempts = */ 0, - // /* enable_s3_requests_logging = */ true, - // /* for_disk_s3 = */ false, - // /* get_request_throttler = */ {}, - // /* put_request_throttler = */ {} - // ); - - // NOTE: now need to figure out the same approach for factory class. - // auto factory = std::make_shared(); - // Aws::Http::SetHttpClientFactory(factory); - // LOG_INFO(&Poco::Logger::get("Application"), "Trying to create"); - // auto client = Aws::Http::CreateHttpClient(client_configuration); - - /// NOTE: check here, even before it's okay! or could test whether it's okay or not. if not, reproduce the successful keeper snapshot case. - /// And see what's the delta. - // std::shared_ptr profile_request( - // Aws::Http::CreateHttpRequest(url, Aws::Http::HttpMethod::HTTP_GET, Aws::Utils::Stream::DefaultResponseStreamFactoryMethod)); - - // const auto result = client->MakeRequest(profile_request); - // if (result->GetResponseCode() != Aws::Http::HttpResponseCode::OK) - // return "failed"; - // result->GetResponseBody(); - // Aws::IStreamBufIterator eos; - // return Aws::String(Aws::IStreamBufIterator(result->GetResponseBody()), eos); - // return "123"; -} - std::pair AWSEC2MetadataClient::getEC2MetadataToken(const std::string & user_agent_string) const { std::lock_guard locker(token_mutex); @@ -251,10 +193,10 @@ Aws::String AWSEC2MetadataClient::getCurrentRegion() const return Aws::Region::AWS_GLOBAL; } -std::shared_ptr InitEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration) +Aws::String awsMetadataEndpoint() { - 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"); @@ -285,8 +227,73 @@ 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 = awsMetadataEndpoint(); + return std::make_shared(client_configuration, endpoint.c_str()); +} + +std::variant AWSEC2MetadataClient::getAvailabilityZoneOrException() +{ + Poco::URI uri(awsMetadataEndpoint() + 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; +} + +std::variant getGCPAvailabilityZoneOrException() +{ + Poco::URI uri("http://169.254.169.254/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("/")); + 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 getRunningAvailabilityZone() +{ + LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); + + auto aws_az_or_exception = AWSEC2MetadataClient::getAvailabilityZoneOrException(); + if (const auto * aws_az = std::get_if(&aws_az_or_exception)) + return *aws_az; + + auto gcp_zone = getGCPAvailabilityZoneOrException(); + if (const auto * gcp_az = std::get_if(&gcp_zone)) + return *gcp_az; + + /// TODO(incfly): Add Azure support. + + throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Failed to find the availability zone, tried both AWS and GCP"); } AWSEC2InstanceProfileConfigLoader::AWSEC2InstanceProfileConfigLoader(const std::shared_ptr & client_, bool use_secure_pull_) diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 3a669baa7aa..6de34c836f5 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -1,5 +1,6 @@ #pragma once +#include "base/types.h" #include "config.h" #if USE_AWS_S3 @@ -18,6 +19,9 @@ namespace DB::S3 inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; +/// 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 +54,11 @@ public: virtual Aws::String getCurrentRegion() const; - static Aws::String getCurrentAvailabilityZone(); + friend String getRunningAvailabilityZone(); private: std::pair getEC2MetadataToken(const std::string & user_agent_string) const; + static std::variant getAvailabilityZoneOrException(); const Aws::String endpoint; mutable std::recursive_mutex token_mutex; @@ -167,8 +172,6 @@ struct CredentialsConfiguration }; -std::string determineAvailabilityZone(const DB::S3::PocoHTTPClientConfiguration & configuration); - class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain { public: From e231d063fb922c13552ee1e3990313f2fce18d16 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 31 Oct 2023 05:06:08 +0000 Subject: [PATCH 0151/1190] fix the exception handlig. Signed-off-by: Jianfei Hu --- src/IO/S3/Credentials.cpp | 32 ++++++++++++++++++-------------- src/IO/S3/Credentials.h | 2 +- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index a5ec210f284..7d1e118bcfc 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -27,7 +27,6 @@ # include # include -/// TODO: these changes do not need to be under AWS_S3. consider to remove or make it separate file libraries. #include #include @@ -236,7 +235,7 @@ std::shared_ptr InitEC2MetadataClient(const Aws::Client::C return std::make_shared(client_configuration, endpoint.c_str()); } -std::variant AWSEC2MetadataClient::getAvailabilityZoneOrException() +String AWSEC2MetadataClient::getAvailabilityZoneOrException() { Poco::URI uri(awsMetadataEndpoint() + EC2_AVAILABILITY_ZONE_RESOURCE); Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort()); @@ -255,7 +254,7 @@ std::variant AWSEC2MetadataClient::getAvailabilityZo return response_data; } -std::variant getGCPAvailabilityZoneOrException() +String getGCPAvailabilityZoneOrException() { Poco::URI uri("http://169.254.169.254/computeMetadata/v1/instance/zone"); Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort()); @@ -283,17 +282,22 @@ String getRunningAvailabilityZone() { LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); - auto aws_az_or_exception = AWSEC2MetadataClient::getAvailabilityZoneOrException(); - if (const auto * aws_az = std::get_if(&aws_az_or_exception)) - return *aws_az; - - auto gcp_zone = getGCPAvailabilityZoneOrException(); - if (const auto * gcp_az = std::get_if(&gcp_zone)) - return *gcp_az; - - /// TODO(incfly): Add Azure support. - - throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Failed to find the availability zone, tried both AWS and GCP"); + 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, GCP. AWS Error {}\n GCP Error {}", aws_ex.displayText(), gcp_ex.displayText()); + } + } } AWSEC2InstanceProfileConfigLoader::AWSEC2InstanceProfileConfigLoader(const std::shared_ptr & client_, bool use_secure_pull_) diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 6de34c836f5..9576718ea72 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -58,7 +58,7 @@ public: private: std::pair getEC2MetadataToken(const std::string & user_agent_string) const; - static std::variant getAvailabilityZoneOrException(); + static String getAvailabilityZoneOrException(); const Aws::String endpoint; mutable std::recursive_mutex token_mutex; 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 0152/1190] 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 0153/1190] 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 0154/1190] 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 0155/1190] 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 0156/1190] 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 0157/1190] 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 0158/1190] 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 0159/1190] 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 cfa1b96d23e8d1f75074e6aa3233e3c4ecaedfbd Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 31 Oct 2023 05:18:25 +0000 Subject: [PATCH 0160/1190] remove comments. Signed-off-by: Jianfei Hu --- src/Coordination/KeeperDispatcher.cpp | 3 +- src/IO/S3/Client.cpp | 3 -- src/IO/S3/Credentials.cpp | 68 ++++++++++++--------------- src/IO/S3/Credentials.h | 3 +- 4 files changed, 34 insertions(+), 43 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 66a25ab8231..dfbcc882c6d 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -373,7 +373,8 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf String availability_zone; try{ availability_zone = DB::S3::getRunningAvailabilityZone(); - } catch (...) + } + catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index ea5af8b5fdc..3e5bc09e982 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -798,9 +798,6 @@ ClientFactory::ClientFactory() Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared(false)); Aws::Http::SetHttpClientFactory(std::make_shared()); - LOG_INFO(&Poco::Logger::get("Application"), "Factory is set"); - // Aws::Http::SetHttpClientFactory(const std::shared_ptr &factory); - // LOG_INFO(&Poco::Logger::get("JianfeiDebug"), "Factory is set {} ", Aws::Http::GetHttpClientFactory() == nullptr); } ClientFactory::~ClientFactory() diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 7d1e118bcfc..c42c3a5eff0 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -647,40 +647,6 @@ Aws::String SSOCredentialsProvider::loadAccessTokenFile(const Aws::String & sso_ } } -Aws::Client::ClientConfiguration getAwsClientConfig(const DB::S3::PocoHTTPClientConfiguration & configuration) -{ - DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( - configuration.region, - configuration.remote_host_filter, - configuration.s3_max_redirects, - configuration.s3_retry_attempts, - configuration.enable_s3_requests_logging, - configuration.for_disk_s3, - configuration.get_request_throttler, - configuration.put_request_throttler, - Aws::Http::SchemeMapper::ToString(Aws::Http::Scheme::HTTP)); - - /// See MakeDefaultHttpResourceClientConfiguration(). - /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside - /// of contrib/aws/aws-cpp-sdk-core/source/internal/AWSHttpResourceClient.cpp - aws_client_configuration.maxConnections = 2; - - /// Explicitly set the proxy settings to empty/zero to avoid relying on defaults that could potentially change - /// in the future. - aws_client_configuration.proxyHost = ""; - aws_client_configuration.proxyUserName = ""; - aws_client_configuration.proxyPassword = ""; - aws_client_configuration.proxyPort = 0; - - /// EC2MetadataService throttles by delaying the response so the service client should set a large read timeout. - /// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds. - aws_client_configuration.connectTimeoutMs = 1000; - aws_client_configuration.requestTimeoutMs = 1000; - - aws_client_configuration.retryStrategy = std::make_shared(1, 1000); - return aws_client_configuration; -} - S3CredentialsProviderChain::S3CredentialsProviderChain( const DB::S3::PocoHTTPClientConfiguration & configuration, const Aws::Auth::AWSCredentials & credentials, @@ -771,8 +737,36 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( } else if (Aws::Utils::StringUtils::ToLower(ec2_metadata_disabled.c_str()) != "true") { - auto aws_client_config = getAwsClientConfig(configuration); - auto ec2_metadata_client = InitEC2MetadataClient(aws_client_config); + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( + configuration.region, + configuration.remote_host_filter, + configuration.s3_max_redirects, + configuration.s3_retry_attempts, + configuration.enable_s3_requests_logging, + configuration.for_disk_s3, + configuration.get_request_throttler, + configuration.put_request_throttler, + Aws::Http::SchemeMapper::ToString(Aws::Http::Scheme::HTTP)); + + /// See MakeDefaultHttpResourceClientConfiguration(). + /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside + /// of contrib/aws/aws-cpp-sdk-core/source/internal/AWSHttpResourceClient.cpp + aws_client_configuration.maxConnections = 2; + + /// Explicitly set the proxy settings to empty/zero to avoid relying on defaults that could potentially change + /// in the future. + aws_client_configuration.proxyHost = ""; + aws_client_configuration.proxyUserName = ""; + aws_client_configuration.proxyPassword = ""; + aws_client_configuration.proxyPort = 0; + + /// EC2MetadataService throttles by delaying the response so the service client should set a large read timeout. + /// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds. + aws_client_configuration.connectTimeoutMs = 1000; + 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); AddProvider(std::make_shared(config_loader)); @@ -789,4 +783,4 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( } } -#endif +#endif \ No newline at end of file diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 9576718ea72..6e0bfba7961 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -171,7 +171,6 @@ struct CredentialsConfiguration bool no_sign_request = false; }; - class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain { public: @@ -183,4 +182,4 @@ public: } -#endif +#endif \ No newline at end of file From f902243c05c77ed3e6f8eb369de2a6821ca17ff0 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 31 Oct 2023 17:43:40 +0000 Subject: [PATCH 0161/1190] comments use static. Signed-off-by: Jianfei Hu --- src/IO/S3/Credentials.cpp | 17 +++++++++++++---- src/IO/S3/Credentials.h | 6 ++++-- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index c42c3a5eff0..decd5f24fc1 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -278,12 +279,11 @@ String getGCPAvailabilityZoneOrException() return zone_info[3]; } -String getRunningAvailabilityZone() +std::variant getRunningAvailabilityZoneImpl() { LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); - try{ - auto aws_az = AWSEC2MetadataClient::getAvailabilityZoneOrException(); + auto aws_az = AWSEC2MetadataClient::getAvailabilityZoneOrException(); return aws_az; } catch (const DB::Exception & aws_ex) @@ -300,6 +300,15 @@ String getRunningAvailabilityZone() } } +String getRunningAvailabilityZone() +{ + static auto az_or_exception = getRunningAvailabilityZoneImpl(); + 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_) , use_secure_pull(use_secure_pull_) @@ -783,4 +792,4 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( } } -#endif \ No newline at end of file +#endif diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 6e0bfba7961..8ca854de90d 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include "base/types.h" #include "config.h" @@ -54,7 +56,7 @@ public: virtual Aws::String getCurrentRegion() const; - friend String getRunningAvailabilityZone(); + friend std::variant getRunningAvailabilityZoneImpl(); private: std::pair getEC2MetadataToken(const std::string & user_agent_string) const; @@ -182,4 +184,4 @@ public: } -#endif \ No newline at end of file +#endif From 3fff81365916e4eac1266824be31742c462d24aa Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 31 Oct 2023 18:48:59 +0000 Subject: [PATCH 0162/1190] style fix and revert src/Coordination/KeeperSnapshotManagerS3.h Signed-off-by: Jianfei Hu --- src/Coordination/KeeperSnapshotManagerS3.h | 3 --- src/IO/S3/Credentials.cpp | 8 +++----- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index 97126e1ddb8..e17cf5a1cfb 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -9,7 +9,6 @@ #include #if USE_AWS_S3 -#include #include #include @@ -66,8 +65,6 @@ public: void updateS3Configuration(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} void uploadSnapshot(const SnapshotFileInfo &, [[maybe_unused]] bool async_upload = true) {} - std::string getAvaibilityZone() const { return ""; } - void startup(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} void shutdown() {} diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index decd5f24fc1..29354d188e0 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -261,7 +261,7 @@ String getGCPAvailabilityZoneOrException() 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"); + request.set("Metadata-Flavor", "Google"); session.sendRequest(request); std::istream& rs = session.receiveResponse(response); if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) { @@ -273,8 +273,7 @@ String getGCPAvailabilityZoneOrException() Strings zone_info; boost::split(zone_info, response_data, boost::is_any_of("/")); 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/, got {}", response_data); } return zone_info[3]; } @@ -294,8 +293,7 @@ std::variant getRunningAvailabilityZoneImpl() } catch(const DB::Exception & gcp_ex) { - throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Failed to find the availability zone, tried AWS, GCP. AWS Error {}\n GCP Error {}", aws_ex.displayText(), gcp_ex.displayText()); + throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Failed to find the availability zone, tried AWS, GCP. AWS Error {}\n GCP Error {}", aws_ex.displayText(), gcp_ex.displayText()); } } } From 045d67ffc120d899c69488a3dbdd29adab314dbd Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 31 Oct 2023 20:32:19 +0000 Subject: [PATCH 0163/1190] more style fix. Signed-off-by: Jianfei Hu space before catch. Signed-off-by: Jianfei Hu --- src/IO/S3/Client.cpp | 1 - src/IO/S3/Credentials.cpp | 23 +++++++++-------------- 2 files changed, 9 insertions(+), 15 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 3e5bc09e982..44ab01ba959 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -1,5 +1,4 @@ #include -#include #if USE_AWS_S3 diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 29354d188e0..9f7ca8800df 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -246,10 +246,8 @@ String AWSEC2MetadataClient::getAvailabilityZoneOrException() 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()); - } + 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; @@ -264,17 +262,14 @@ String getGCPAvailabilityZoneOrException() 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()); - } + 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("/")); - if (zone_info.size() != 4) { + 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]; } @@ -287,11 +282,12 @@ std::variant getRunningAvailabilityZoneImpl() } catch (const DB::Exception & aws_ex) { - try{ + try + { auto gcp_zone = getGCPAvailabilityZoneOrException(); return gcp_zone; } - catch(const DB::Exception & gcp_ex) + catch (const DB::Exception & gcp_ex) { throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Failed to find the availability zone, tried AWS, GCP. AWS Error {}\n GCP Error {}", aws_ex.displayText(), gcp_ex.displayText()); } @@ -786,8 +782,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( AddProvider(std::make_shared()); } - - } + } #endif From 6f94770a8e4d9c80e213e77a5830dd4d4eb0727f Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Wed, 1 Nov 2023 00:50:32 +0000 Subject: [PATCH 0164/1190] more fix -DUSE_AWS_S3=0 case. Signed-off-by: Jianfei Hu --- src/Coordination/KeeperDispatcher.cpp | 3 ++- src/IO/S3/Credentials.cpp | 19 ++++++++++++++++++- src/IO/S3/Credentials.h | 11 +++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index dfbcc882c6d..2a885c59c5f 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -371,7 +371,8 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); keeper_context = std::make_shared(standalone_keeper); String availability_zone; - try{ + try + { availability_zone = DB::S3::getRunningAvailabilityZone(); } catch (...) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 9f7ca8800df..68077b719d4 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -4,7 +4,6 @@ #include #include - #if USE_AWS_S3 # include @@ -785,4 +784,22 @@ 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 8ca854de90d..e45809eaa10 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -184,4 +184,15 @@ public: } +#else + +namespace DB +{ + +namespace S3 +{ +String getRunningAvailabilityZone(); +} + +} #endif 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 0165/1190] 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 0166/1190] 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 0167/1190] 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 0168/1190] 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 0169/1190] 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 3361c5d97327b50d40250ce43daa4a883bd45722 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 1 Nov 2023 12:05:41 +0100 Subject: [PATCH 0170/1190] Updated using read write buffer into modules --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 61 ++++++++++++++++---------- src/Common/ZooKeeper/ZooKeeperImpl.h | 14 ++++-- src/Server/KeeperTCPHandler.cpp | 46 ++++++++++++------- src/Server/KeeperTCPHandler.h | 8 +++- 4 files changed, 86 insertions(+), 43 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 5e1201d6ba2..9e1022e05c1 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -277,13 +277,34 @@ using namespace DB; template void ZooKeeper::write(const T & x) { - Coordination::write(x, *maybe_compressed_out); + Coordination::write(x, getWriteBuffer()); } template void ZooKeeper::read(T & x) { - Coordination::read(x, *maybe_compressed_in); + Coordination::read(x, getReadBuffer()); +} + +WriteBuffer & ZooKeeper::getWriteBuffer() +{ + if (compressed_out) + return *compressed_out; + return *out; +} + +void ZooKeeper::flushWriteBuffer() +{ + if (compressed_out) + compressed_out->next(); + out->next(); +} + +ReadBuffer & ZooKeeper::getReadBuffer() +{ + if (compressed_in) + return *compressed_in; + return *in; } static void removeRootPath(String & path, const String & chroot) @@ -374,8 +395,8 @@ ZooKeeper::ZooKeeper( send_thread = ThreadFromGlobalPool([this] { sendThread(); }); receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); - initFeatureFlags(); - keeper_feature_flags.logFlags(log); +// initFeatureFlags(); +// keeper_feature_flags.logFlags(log); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); } @@ -441,10 +462,10 @@ void ZooKeeper::connect( socket.setSendTimeout(args.operation_timeout_ms * 1000); socket.setNoDelay(true); - in = std::make_shared(socket); - out = std::make_shared(socket); - maybe_compressed_in = in; - maybe_compressed_out = out; + in.emplace(socket); + out.emplace(socket); + compressed_in.reset(); + compressed_out.reset(); try { @@ -469,10 +490,8 @@ void ZooKeeper::connect( connected = true; if (use_compression) { - maybe_compressed_in = std::make_shared(*in); - maybe_compressed_out = std::make_shared(*out, - CompressionCodecFactory::instance().get( - "ZSTD", {})); + compressed_in.emplace(*in); + compressed_out.emplace(*out,CompressionCodecFactory::instance().get("None", {})); } original_index = static_cast(node.original_index); @@ -549,8 +568,7 @@ void ZooKeeper::sendHandshake() write(timeout); write(previous_session_id); write(passwd); - - maybe_compressed_out->next(); + flushWriteBuffer(); } void ZooKeeper::receiveHandshake() @@ -598,8 +616,8 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) request.scheme = scheme; request.data = data; request.xid = AUTH_XID; - request.write(*maybe_compressed_out); - maybe_compressed_out->next(); + request.write(getWriteBuffer()); + flushWriteBuffer(); int32_t length; XID read_xid; @@ -678,9 +696,8 @@ void ZooKeeper::sendThread() info.request->addRootPath(args.chroot); info.request->probably_sent = true; - info.request->write(*maybe_compressed_out); - - maybe_compressed_out->next(); + info.request->write(getWriteBuffer()); + flushWriteBuffer(); logOperationIfNeeded(info.request); @@ -696,8 +713,8 @@ void ZooKeeper::sendThread() ZooKeeperHeartbeatRequest request; request.xid = PING_XID; - request.write(*maybe_compressed_out); - maybe_compressed_out->next(); + request.write(getWriteBuffer()); + flushWriteBuffer(); } ProfileEvents::increment(ProfileEvents::ZooKeeperBytesSent, out->count() - prev_bytes_sent); @@ -869,7 +886,7 @@ void ZooKeeper::receiveEvent() } else { - response->readImpl(*maybe_compressed_in); + response->readImpl(getReadBuffer()); response->removeRootPath(args.chroot); } /// Instead of setting the watch in sendEvent, set it in receiveEvent because need to check the response. diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index ceac8990904..13e1dc9e3cd 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include #include @@ -240,10 +242,10 @@ private: /// To avoid excessive getpeername(2) calls. Poco::Net::SocketAddress socket_address; - std::shared_ptr in; - std::shared_ptr out; - std::shared_ptr maybe_compressed_in; - std::shared_ptr maybe_compressed_out; + std::optional in; + std::optional out; + std::optional compressed_in; + std::optional compressed_out; bool use_compression = false; @@ -333,6 +335,10 @@ private: template void read(T &); + WriteBuffer & getWriteBuffer(); + void flushWriteBuffer(); + ReadBuffer & getReadBuffer(); + void logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response = nullptr, bool finalize = false, UInt64 elapsed_ms = 0); void initFeatureFlags(); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 84109453452..8c12707d88f 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -317,6 +317,8 @@ void KeeperTCPHandler::runImpl() in = std::make_shared(socket()); out = std::make_shared(socket()); + compressed_in.reset(); + compressed_out.reset(); bool use_compression = false; @@ -391,15 +393,8 @@ void KeeperTCPHandler::runImpl() if (use_compression) { - maybe_compressed_in = std::make_shared(*in); - maybe_compressed_out = std::make_shared(*out, - CompressionCodecFactory::instance().get("ZSTD", - {})); - } - else - { - maybe_compressed_in = in; - maybe_compressed_out = out; + compressed_in.emplace(*in); + compressed_out.emplace(*out,CompressionCodecFactory::instance().get("None",{})); } auto response_fd = poll_wrapper->getResponseFD(); @@ -488,8 +483,8 @@ void KeeperTCPHandler::runImpl() updateStats(response); packageSent(); - response->write(*maybe_compressed_out); - maybe_compressed_out->next(); + response->write(getWriteBuffer()); + flushWriteBuffer(); log_long_operation("Sending response"); if (response->error == Coordination::Error::ZSESSIONEXPIRED) { @@ -559,19 +554,40 @@ bool KeeperTCPHandler::tryExecuteFourLetterWordCmd(int32_t command) } } +WriteBuffer & KeeperTCPHandler::getWriteBuffer() +{ + if (compressed_out) + return *compressed_out; + return *out; +} + +void KeeperTCPHandler::flushWriteBuffer() +{ + if (compressed_out) + compressed_out->next(); + out->next(); +} + +ReadBuffer & KeeperTCPHandler::getReadBuffer() +{ + if (compressed_in) + return *compressed_in; + return *in; +} + std::pair KeeperTCPHandler::receiveRequest() { int32_t length; - Coordination::read(length, *maybe_compressed_in); + Coordination::read(length, getReadBuffer()); int32_t xid; - Coordination::read(xid, *maybe_compressed_in); + Coordination::read(xid, getReadBuffer()); Coordination::OpNum opnum; - Coordination::read(opnum, *maybe_compressed_in); + Coordination::read(opnum, getReadBuffer()); Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); request->xid = xid; - request->readImpl(*maybe_compressed_in); + request->readImpl(getReadBuffer()); if (!keeper_dispatcher->putRequest(request, session_id)) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id); diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index bd2f29d43d3..7f46f735f2f 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -82,13 +82,17 @@ private: /// Streams for reading/writing from/to client connection socket. std::shared_ptr in; std::shared_ptr out; - std::shared_ptr maybe_compressed_in; - std::shared_ptr maybe_compressed_out; + std::optional compressed_in; + std::optional compressed_out; std::atomic connected{false}; void runImpl(); + WriteBuffer & getWriteBuffer(); + void flushWriteBuffer(); + ReadBuffer & getReadBuffer(); + void sendHandshake(bool has_leader, bool & use_compression); Poco::Timespan receiveHandshake(int32_t handshake_length, bool & use_compression); 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 0171/1190] 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 942bf7f14791dee3c1cbc85ece0a036191a9bd93 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 1 Nov 2023 14:41:44 +0100 Subject: [PATCH 0172/1190] Uncommented feature flag code --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 9e1022e05c1..4f553b61493 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -395,8 +395,8 @@ ZooKeeper::ZooKeeper( send_thread = ThreadFromGlobalPool([this] { sendThread(); }); receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); -// initFeatureFlags(); -// keeper_feature_flags.logFlags(log); + initFeatureFlags(); + keeper_feature_flags.logFlags(log); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); } From b22773ccd1edab0789cfaab1f823e553b94a0bbc Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Wed, 1 Nov 2023 10:30:29 -0700 Subject: [PATCH 0173/1190] Apply suggestions from code review Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Coordination/KeeperContext.h | 2 +- src/Coordination/KeeperDispatcher.cpp | 2 +- src/IO/S3/Credentials.cpp | 9 +++++---- src/IO/S3/Credentials.h | 2 +- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index a7ef7d9e937..6b4d0695e2d 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_, std::string availability_zone); + void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, const std::string & availability_zone); Phase getServerState() const; void setServerState(Phase server_state_); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 2a885c59c5f..5fe9614f652 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -11,7 +11,7 @@ #include #include #include -#include "IO/S3/Credentials.h" +#include #include #include diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 68077b719d4..55dbef73325 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -192,7 +192,7 @@ Aws::String AWSEC2MetadataClient::getCurrentRegion() const return Aws::Region::AWS_GLOBAL; } -Aws::String awsMetadataEndpoint() +static Aws::String getAWSMetadataEndpoint() { auto * logger = &Poco::Logger::get("AWSEC2InstanceProfileConfigLoader"); Aws::String ec2_metadata_service_endpoint = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT"); @@ -260,7 +260,7 @@ String getGCPAvailabilityZoneOrException() Poco::Net::HTTPResponse response; request.set("Metadata-Flavor", "Google"); session.sendRequest(request); - std::istream& rs = session.receiveResponse(response); + 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; @@ -275,7 +275,8 @@ String getGCPAvailabilityZoneOrException() std::variant getRunningAvailabilityZoneImpl() { LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); - try{ + try + { auto aws_az = AWSEC2MetadataClient::getAvailabilityZoneOrException(); return aws_az; } @@ -288,7 +289,7 @@ std::variant getRunningAvailabilityZoneImpl() } catch (const DB::Exception & gcp_ex) { - throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Failed to find the availability zone, tried AWS, GCP. AWS Error {}\n GCP Error {}", aws_ex.displayText(), gcp_ex.displayText()); + throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Failed to find the availability zone, tried AWS, GCP. AWS Error: {}\nGCP Error: {}", aws_ex.displayText(), gcp_ex.displayText()); } } } diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index e45809eaa10..296e9c04452 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -2,7 +2,7 @@ #include #include -#include "base/types.h" +#include #include "config.h" #if USE_AWS_S3 From e29b7640c46b02c6d9b5e683478ea89add9dc225 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Wed, 1 Nov 2023 17:53:34 +0000 Subject: [PATCH 0174/1190] gcp metadata use dns. Signed-off-by: Jianfei Hu --- src/Coordination/KeeperContext.cpp | 2 +- src/IO/S3/Credentials.cpp | 9 +++++---- src/IO/S3/Credentials.h | 3 +++ 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 4ce8b2c732a..afbb7ebd13c 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -32,7 +32,7 @@ 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_, std::string availability_zone) +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, const std::string & availability_zone) { dispatcher = dispatcher_; diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 55dbef73325..fc8a911f53b 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -231,20 +231,20 @@ static Aws::String getAWSMetadataEndpoint() std::shared_ptr InitEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration) { - auto endpoint = awsMetadataEndpoint(); + auto endpoint = getAWSMetadataEndpoint(); return std::make_shared(client_configuration, endpoint.c_str()); } String AWSEC2MetadataClient::getAvailabilityZoneOrException() { - Poco::URI uri(awsMetadataEndpoint() + EC2_AVAILABILITY_ZONE_RESOURCE); + 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); + 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; @@ -254,7 +254,7 @@ String AWSEC2MetadataClient::getAvailabilityZoneOrException() String getGCPAvailabilityZoneOrException() { - Poco::URI uri("http://169.254.169.254/computeMetadata/v1/instance/zone"); + 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; @@ -267,6 +267,7 @@ String getGCPAvailabilityZoneOrException() 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]; diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 296e9c04452..5b2dccea01f 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -21,6 +21,9 @@ 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(); 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 0175/1190] 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 0176/1190] 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 0177/1190] 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 0178/1190] 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 0179/1190] 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 0180/1190] 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 0181/1190] 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 0182/1190] 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 0183/1190] 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 374545f4e20c52d6f393efe78b2a94801d9be0a0 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 2 Nov 2023 09:38:53 +0100 Subject: [PATCH 0184/1190] Removed unwanted flush & updated to use optional for buffers --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 2 -- src/Server/KeeperTCPHandler.cpp | 10 +++++----- src/Server/KeeperTCPHandler.h | 4 ++-- 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index e9803d165e1..592d142e925 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -27,7 +27,6 @@ void ZooKeeperResponse::write(WriteBuffer & out) const if (error == Error::ZOK) writeImpl(buf); Coordination::write(buf.str(), out); - out.next(); } std::string ZooKeeperRequest::toString() const @@ -49,7 +48,6 @@ void ZooKeeperRequest::write(WriteBuffer & out) const Coordination::write(getOpNum(), buf); writeImpl(buf); Coordination::write(buf.str(), out); - out.next(); } void ZooKeeperSyncRequest::writeImpl(WriteBuffer & out) const diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 8c12707d88f..0a5ef29b0fa 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -111,13 +111,13 @@ struct SocketInterruptablePollWrapper return pipe.fds_rw[1]; } - PollResult poll(Poco::Timespan remaining_time, const std::shared_ptr & in) + PollResult poll(Poco::Timespan remaining_time, const ReadBufferFromPocoSocket & in) { bool socket_ready = false; bool fd_ready = false; - if (in->available() != 0) + if (in.available() != 0) socket_ready = true; if (response_in.available() != 0) @@ -315,8 +315,8 @@ void KeeperTCPHandler::runImpl() socket().setSendTimeout(send_timeout); socket().setNoDelay(true); - in = std::make_shared(socket()); - out = std::make_shared(socket()); + in.emplace(socket()); + out.emplace(socket()); compressed_in.reset(); compressed_out.reset(); @@ -431,7 +431,7 @@ void KeeperTCPHandler::runImpl() { using namespace std::chrono_literals; - PollResult result = poll_wrapper->poll(session_timeout, in); + PollResult result = poll_wrapper->poll(session_timeout, *in); log_long_operation("Polling socket"); if (result.has_requests && !close_received) { diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index 7f46f735f2f..adb1baa084f 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -80,8 +80,8 @@ private: Coordination::XID close_xid = Coordination::CLOSE_XID; /// Streams for reading/writing from/to client connection socket. - std::shared_ptr in; - std::shared_ptr out; + std::optional in; + std::optional out; std::optional compressed_in; std::optional compressed_out; From 5b6bf58730827a48cf162942ecc3ff38caaa0999 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Nov 2023 11:19:37 +0000 Subject: [PATCH 0185/1190] Proper process aliases for aggregation-by-partition optimization. --- src/Interpreters/ActionsDAG.cpp | 96 +++++++++++++++---- src/Interpreters/ActionsDAG.h | 5 + .../useDataParallelAggregation.cpp | 13 +-- 3 files changed, 91 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 04dee2ed6e6..f681b4ce5cc 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -335,6 +335,28 @@ const ActionsDAG::Node * ActionsDAG::tryFindInOutputs(const std::string & name) return nullptr; } +ActionsDAG::NodeRawConstPtrs ActionsDAG::findInOutpus(const Names & names) const +{ + NodeRawConstPtrs required_nodes; + required_nodes.reserve(names.size()); + + std::unordered_map names_map; + for (const auto * node : outputs) + names_map[node->result_name] = node; + + for (const auto & name : names) + { + auto it = names_map.find(name); + if (it == names_map.end()) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Unknown column: {}, there are only columns {}", name, dumpDAG()); + + required_nodes.push_back(it->second); + } + + return required_nodes; +} + void ActionsDAG::addOrReplaceInOutputs(const Node & node) { for (auto & output_node : outputs) @@ -441,23 +463,7 @@ void ActionsDAG::removeUnusedActions(const NameSet & required_names, bool allow_ void ActionsDAG::removeUnusedActions(const Names & required_names, bool allow_remove_inputs, bool allow_constant_folding) { - NodeRawConstPtrs required_nodes; - required_nodes.reserve(required_names.size()); - - std::unordered_map names_map; - for (const auto * node : outputs) - names_map[node->result_name] = node; - - for (const auto & name : required_names) - { - auto it = names_map.find(name); - if (it == names_map.end()) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown column: {}, there are only columns {}", name, dumpDAG()); - - required_nodes.push_back(it->second); - } - + auto required_nodes = findInOutpus(required_names); outputs.swap(required_nodes); removeUnusedActions(allow_remove_inputs, allow_constant_folding); } @@ -528,6 +534,62 @@ void ActionsDAG::removeUnusedActions(bool allow_remove_inputs, bool allow_consta std::erase_if(inputs, [&](const Node * node) { return !visited_nodes.contains(node); }); } +ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) +{ + auto actions = std::make_shared(); + std::unordered_map copy_map; + + struct Frame + { + const Node * node = nullptr; + size_t next_child = 0; + }; + + std::stack stack; + + for (const auto * output : outputs) + { + if (copy_map.contains(output)) + continue; + + stack.push(Frame{output}); + while (!stack.empty()) + { + auto & frame = stack.top(); + const auto & children = frame.node->children; + while (frame.next_child < children.size() && copy_map.contains(children[frame.next_child])) + ++frame.next_child; + + if (frame.next_child < children.size()) + { + stack.push(Frame{children[frame.next_child]}); + continue; + } + + auto & copy_node = copy_map[frame.node]; + + if (remove_aliases && frame.node->type == ActionType::ALIAS) + copy_node = copy_map[frame.node->children.front()]; + else + copy_node = &actions->nodes.emplace_back(*frame.node); + + if (frame.node->type == ActionType::INPUT) + actions->inputs.push_back(copy_node); + + stack.pop(); + } + } + + for (auto & node : actions->nodes) + for (auto & child : node.children) + child = copy_map[child]; + + for (const auto * output : outputs) + actions->outputs.push_back(copy_map[output]); + + return actions; +} + static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * node, ColumnsWithTypeAndName arguments) { ColumnWithTypeAndName res_column; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 48ed03d7347..2f7e2f109df 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -157,6 +157,9 @@ public: /// Same, but return nullptr if node not found. const Node * tryFindInOutputs(const std::string & name) const; + /// Same, but for the list of names. + NodeRawConstPtrs findInOutpus(const Names & names) const; + /// Find first node with the same name in output nodes and replace it. /// If was not found, add node to outputs end. void addOrReplaceInOutputs(const Node & node); @@ -257,6 +260,8 @@ public: ActionsDAGPtr clone() const; + static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); + /// Execute actions for header. Input block must have empty columns. /// Result should be equal to the execution of ExpressionActions built from this DAG. /// Actions are not changed, no expressions are compiled. diff --git a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp index f90d10b31d5..0b53b6dd8a6 100644 --- a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp +++ b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp @@ -95,7 +95,7 @@ bool allOutputsDependsOnlyOnAllowedNodes( { const auto & match = matches.at(node); /// Function could be mapped into its argument. In this case .monotonicity != std::nullopt (see matchTrees) - if (match.node && match.node->result_name == node->result_name && !match.monotonicity) + if (match.node && !match.monotonicity) res = irreducible_nodes.contains(match.node); } @@ -155,9 +155,10 @@ bool isPartitionKeySuitsGroupByKey( return false; /// We are interested only in calculations required to obtain group by keys (and not aggregate function arguments for example). - group_by_actions->removeUnusedActions(aggregating.getParams().keys); + auto key_nodes = group_by_actions->findInOutpus(aggregating.getParams().keys); + auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, true); - const auto & gb_key_required_columns = group_by_actions->getRequiredColumnsNames(); + const auto & gb_key_required_columns = group_by_key_actions->getRequiredColumnsNames(); const auto & partition_actions = reading.getStorageMetadata()->getPartitionKey().expression->getActionsDAG(); @@ -166,9 +167,9 @@ bool isPartitionKeySuitsGroupByKey( if (std::ranges::find(gb_key_required_columns, col) == gb_key_required_columns.end()) return false; - const auto irreducibe_nodes = removeInjectiveFunctionsFromResultsRecursively(group_by_actions); + const auto irreducibe_nodes = removeInjectiveFunctionsFromResultsRecursively(group_by_key_actions); - const auto matches = matchTrees(group_by_actions->getOutputs(), partition_actions); + const auto matches = matchTrees(group_by_key_actions->getOutputs(), partition_actions); return allOutputsDependsOnlyOnAllowedNodes(partition_actions, irreducibe_nodes, matches); } @@ -206,7 +207,7 @@ size_t tryAggregatePartitionsIndependently(QueryPlan::Node * node, QueryPlan::No return 0; if (!reading->willOutputEachPartitionThroughSeparatePort() - && isPartitionKeySuitsGroupByKey(*reading, expression_step->getExpression()->clone(), *aggregating_step)) + && isPartitionKeySuitsGroupByKey(*reading, expression_step->getExpression(), *aggregating_step)) { if (reading->requestOutputEachPartitionThroughSeparatePort()) aggregating_step->skipMerging(); From 5095a2175874bb2dbae254f78c8ff71b4ac1d37f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Nov 2023 11:23:12 +0000 Subject: [PATCH 0186/1190] Update tests list. --- 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 ff93d2f4e30..7a73958c0f6 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -47,7 +47,6 @@ 02458_use_structure_from_insertion_table 02479_race_condition_between_insert_and_droppin_mv 02493_inconsistent_hex_and_binary_number -02521_aggregation_by_partitions 02554_fix_grouping_sets_predicate_push_down 02575_merge_prewhere_different_default_kind 02713_array_low_cardinality_string From 67e29d5eb95fa03307e566a81f56233f8a949197 Mon Sep 17 00:00:00 2001 From: Joe Lynch Date: Thu, 2 Nov 2023 13:40:50 +0100 Subject: [PATCH 0187/1190] Allow repeated adds and removes for delta lake files --- src/Storages/DataLakes/DeltaLakeMetadataParser.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp index eae17078577..b8bffb267e5 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp +++ b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp @@ -157,16 +157,12 @@ struct DeltaLakeMetadataParser::Impl if (json.has("add")) { const auto path = json["add"]["path"].getString(); - const auto [_, inserted] = result.insert(fs::path(configuration.getPath()) / path); - if (!inserted) - throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", path); + result.insert(fs::path(configuration.getPath()) / path); } else if (json.has("remove")) { const auto path = json["remove"]["path"].getString(); - const bool erase = result.erase(fs::path(configuration.getPath()) / path); - if (!erase) - throw Exception(ErrorCodes::INCORRECT_DATA, "File doesn't exist {}", path); + result.erase(fs::path(configuration.getPath()) / path); } } } From a716e12ec0535739cfa24479905c657952b0a340 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 2 Nov 2023 14:52:10 +0100 Subject: [PATCH 0188/1190] Addressed review comments --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Server/KeeperTCPHandler.cpp | 11 ++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 4f553b61493..5e84a431097 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -491,7 +491,7 @@ void ZooKeeper::connect( if (use_compression) { compressed_in.emplace(*in); - compressed_out.emplace(*out,CompressionCodecFactory::instance().get("None", {})); + compressed_out.emplace(*out, CompressionCodecFactory::instance().get("ZSTD", {})); } original_index = static_cast(node.original_index); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 0a5ef29b0fa..30233134fa8 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -394,7 +394,7 @@ void KeeperTCPHandler::runImpl() if (use_compression) { compressed_in.emplace(*in); - compressed_out.emplace(*out,CompressionCodecFactory::instance().get("None",{})); + compressed_out.emplace(*out, CompressionCodecFactory::instance().get("ZSTD",{})); } auto response_fd = poll_wrapper->getResponseFD(); @@ -577,17 +577,18 @@ ReadBuffer & KeeperTCPHandler::getReadBuffer() std::pair KeeperTCPHandler::receiveRequest() { + auto & read_buffer = getReadBuffer(); int32_t length; - Coordination::read(length, getReadBuffer()); + Coordination::read(length, read_buffer); int32_t xid; - Coordination::read(xid, getReadBuffer()); + Coordination::read(xid, read_buffer); Coordination::OpNum opnum; - Coordination::read(opnum, getReadBuffer()); + Coordination::read(opnum, read_buffer); Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); request->xid = xid; - request->readImpl(getReadBuffer()); + request->readImpl(read_buffer); if (!keeper_dispatcher->putRequest(request, session_id)) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id); From 06ff714e2e3dba727b71476447d78feb2ee5a720 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Wed, 1 Nov 2023 22:06:51 +0000 Subject: [PATCH 0189/1190] add configuration option and integration test(wip). Signed-off-by: Jianfei Hu --- src/Coordination/KeeperContext.cpp | 15 ++- src/Coordination/KeeperContext.h | 6 +- src/Coordination/KeeperStorage.cpp | 8 -- tests/config/config.d/keeper_port.xml | 1 + tests/integration/helpers/cluster.py | 1 + .../test_keeper_availability_zone/__init__.py | 0 .../configs/enable_keeper.xml | 39 +++++++ .../test_keeper_availability_zone/test.py | 108 ++++++++++++++++++ .../test_zookeeper_fallback_session/test.py | 2 + 9 files changed, 159 insertions(+), 21 deletions(-) create mode 100644 tests/integration/test_keeper_availability_zone/__init__.py create mode 100644 tests/integration/test_keeper_availability_zone/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_availability_zone/test.py diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index afbb7ebd13c..1cee2a8e446 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -32,12 +32,16 @@ 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 & availability_zone) +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, const std::string & environment_az) { dispatcher = dispatcher_; - running_availability_zone = availability_zone; - LOG_INFO(&Poco::Logger::get("KeeperContext"), "Initialize the KeeperContext with availability zone: '{}'. ", running_availability_zone); + /// 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); @@ -208,11 +212,6 @@ void KeeperContext::setStateFileDisk(DiskPtr disk) state_file_storage = std::move(disk); } -std::string KeeperContext::getRunningAvailabilityZone() const -{ - return running_availability_zone; -} - const std::unordered_map & KeeperContext::getSystemNodesWithData() const { return system_nodes_with_data; diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 6b4d0695e2d..1bae21dde9d 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 & availability_zone); + void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_, const std::string & environment_az); Phase getServerState() const; void setServerState(Phase server_state_); @@ -53,8 +53,6 @@ public: constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; } - std::string getRunningAvailabilityZone() const; - private: /// local disk defined using path or disk name using Storage = std::variant; @@ -90,8 +88,6 @@ private: KeeperFeatureFlags feature_flags; KeeperDispatcher * dispatcher{nullptr}; - - std::string running_availability_zone; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 4fb2b6b1834..b2bed2d483f 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1098,14 +1098,6 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); - /// AZ node information is a special case not persisted in the storage, so we handle it first. - if (request.path == Coordination::keeper_availability_zone_path) - { - response.data = storage.keeper_context->getRunningAvailabilityZone(); - response.error = Coordination::Error::ZOK; - return response_ptr; - } - if constexpr (!local) { if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 1e646cd07a7..35d310f7137 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -2,6 +2,7 @@ 9181 1 + never-down-42 1 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9b63a21529f..8192764c8e3 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -667,6 +667,7 @@ class ClickHouseCluster: self.zookeeper_certfile = zookeeper_certfile # available when with_zookeper == True + # NOTE: reason why needed. self.use_keeper = True self.zookeeper_port = 2181 self.keeper_instance_dir_prefix = p.join( 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/enable_keeper.xml b/tests/integration/test_keeper_availability_zone/configs/enable_keeper.xml new file mode 100644 index 00000000000..63e00ab66be --- /dev/null +++ b/tests/integration/test_keeper_availability_zone/configs/enable_keeper.xml @@ -0,0 +1,39 @@ + + 9000 + 8123 + 9000 + 127.0.0.1 + 9009 + 127.0.0.1 + + + 9181 + 1 + az-foo-42 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 10 + 5 + 5000 + 10000 + trace + + + 0 + 0 + 0 + + + + + + + 1 + localhost + 9234 + + + + 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..8157b5272a8 --- /dev/null +++ b/tests/integration/test_keeper_availability_zone/test.py @@ -0,0 +1,108 @@ +#!/usr/bin/env python3 + +# Plan: +# - Simplify the test +# - Check system.zookeeper to get result. +import pytest +import os +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from kazoo.client import KazooClient, KazooState + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def restart_clickhouse(feature_flags=[], expect_fail=True): + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + + if len(feature_flags) > 0: + feature_flags_config = "" + + for feature, is_enabled in feature_flags: + feature_flags_config += f"<{feature}>{is_enabled}<\\/{feature}>" + + feature_flags_config += "<\\/feature_flags>" + + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + feature_flags_config, + ) + + node.start_clickhouse(retry_start=not expect_fail) + keeper_utils.wait_until_connected(cluster, node) + + +def test_keeper_feature_flags(started_cluster): + restart_clickhouse() + + def assert_feature_flags(feature_flags): + res = keeper_utils.send_4lw_cmd(started_cluster, node, "ftfl") + + for feature, is_enabled in feature_flags: + node.wait_for_log_line( + f"ZooKeeperClient: Keeper feature flag {feature.upper()}: {'enabled' if is_enabled else 'disabled'}", + look_behind_lines=1000, + ) + + node.wait_for_log_line( + f"KeeperContext: Keeper feature flag {feature.upper()}: {'enabled' if is_enabled else 'disabled'}", + look_behind_lines=1000, + ) + + assert f"{feature}\t{1 if is_enabled else 0}" in res + + assert_feature_flags( + [("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)] + ) + + feature_flags = [("multi_read", 0), ("check_not_exists", 1)] + restart_clickhouse(feature_flags) + assert_feature_flags(feature_flags + [("filtered_list", 1)]) + + feature_flags = [("multi_read", 0), ("check_not_exists", 0), ("filtered_list", 0)] + restart_clickhouse(feature_flags) + assert_feature_flags(feature_flags) + +# Look at other test config to enable 9000. +# E AssertionError: assert 'Code: 210. DB::NetException: Connection refused (172.16.1.2:9000). (NETWORK_ERROR)\n\n' == 'never-down-42' +# E - never-down-42 +# E + Code: 210. DB::NetException: Connection refused (172.16.1.2:9000). (NETWORK_ERROR) +# E + + with pytest.raises(Exception): + restart_clickhouse([("invalid_feature", 1)], expect_fail=True) + + resp = node.query_and_get_error("SELECT value FROM system.zookeeper where path = '/keeper' and name='availability_zone'") + assert resp == 'never-down-42' + diff --git a/tests/integration/test_zookeeper_fallback_session/test.py b/tests/integration/test_zookeeper_fallback_session/test.py index 9afabfa3da3..3828a6afbb3 100644 --- a/tests/integration/test_zookeeper_fallback_session/test.py +++ b/tests/integration/test_zookeeper_fallback_session/test.py @@ -98,3 +98,5 @@ def test_fallback_session(started_cluster: ClickHouseCluster): ) == "2\n" ) + resp = node1.query_with_retry("SELECT value FROM system.zookeeper where path = '/keeper' and name='availability_zone'") + assert resp == 'never-down-42' \ No newline at end of file From b88048e6d9a5ce72475433ac709a01a4ae515cec Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 2 Nov 2023 16:51:50 +0000 Subject: [PATCH 0190/1190] 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 0191/1190] 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 0192/1190] 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 0193/1190] 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 0194/1190] 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 136efba321af48c985b17d97d4108ea00aa56841 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Fri, 3 Nov 2023 00:20:40 +0000 Subject: [PATCH 0195/1190] integration test working and std::exception_ptr. Signed-off-by: Jianfei Hu --- src/IO/S3/Credentials.cpp | 4 +- src/IO/S3/Credentials.h | 2 +- tests/config/config.d/keeper_port.xml | 1 - tests/integration/helpers/cluster.py | 1 - tests/integration/helpers/keeper_config1.xml | 1 + tests/integration/helpers/keeper_config2.xml | 1 + .../configs/enable_keeper.xml | 39 -------- .../configs/keeper_config.xml | 2 + .../test_keeper_availability_zone/test.py | 97 ++----------------- .../test_zookeeper_fallback_session/test.py | 2 - 10 files changed, 18 insertions(+), 132 deletions(-) delete mode 100644 tests/integration/test_keeper_availability_zone/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_availability_zone/configs/keeper_config.xml diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index fc8a911f53b..3b9a5bc48f8 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Common/Exception.h" #if USE_AWS_S3 @@ -290,7 +291,8 @@ std::variant getRunningAvailabilityZoneImpl() } catch (const DB::Exception & gcp_ex) { - throw DB::Exception(ErrorCodes::UNSUPPORTED_METHOD, "Failed to find the availability zone, tried AWS, GCP. AWS Error: {}\nGCP Error: {}", aws_ex.displayText(), gcp_ex.displayText()); + return std::make_exception_ptr(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())); } } } diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 5b2dccea01f..7df492c549a 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -1,8 +1,8 @@ #pragma once #include -#include #include +#include #include "config.h" #if USE_AWS_S3 diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 35d310f7137..1e646cd07a7 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -2,7 +2,6 @@ 9181 1 - never-down-42 1 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 8192764c8e3..9b63a21529f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -667,7 +667,6 @@ class ClickHouseCluster: self.zookeeper_certfile = zookeeper_certfile # available when with_zookeper == True - # NOTE: reason why needed. self.use_keeper = True self.zookeeper_port = 2181 self.keeper_instance_dir_prefix = p.join( 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/configs/enable_keeper.xml b/tests/integration/test_keeper_availability_zone/configs/enable_keeper.xml deleted file mode 100644 index 63e00ab66be..00000000000 --- a/tests/integration/test_keeper_availability_zone/configs/enable_keeper.xml +++ /dev/null @@ -1,39 +0,0 @@ - - 9000 - 8123 - 9000 - 127.0.0.1 - 9009 - 127.0.0.1 - - - 9181 - 1 - az-foo-42 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 10 - 5 - 5000 - 10000 - trace - - - 0 - 0 - 0 - - - - - - - 1 - localhost - 9234 - - - - 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 index 8157b5272a8..1836f0e679b 100644 --- a/tests/integration/test_keeper_availability_zone/test.py +++ b/tests/integration/test_keeper_availability_zone/test.py @@ -1,108 +1,31 @@ -#!/usr/bin/env python3 - -# Plan: -# - Simplify the test -# - Check system.zookeeper to get result. import pytest -import os from helpers.cluster import ClickHouseCluster -import helpers.keeper_utils as keeper_utils -from kazoo.client import KazooClient, KazooState +from helpers.keeper_utils import KeeperClient + -CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) cluster = ClickHouseCluster(__file__) -# clickhouse itself will use external zookeeper node = cluster.add_instance( "node", - main_configs=["configs/enable_keeper.xml"], + main_configs=["configs/keeper_config.xml"], + with_zookeeper=True, stay_alive=True, ) -@pytest.fixture(scope="module") +@pytest.fixture(scope="module", autouse=True) def started_cluster(): try: cluster.start() - yield cluster finally: cluster.shutdown() -def get_connection_zk(nodename, timeout=30.0): - _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout - ) - _fake_zk_instance.start() - return _fake_zk_instance - - -def restart_clickhouse(feature_flags=[], expect_fail=True): - node.stop_clickhouse() - node.copy_file_to_container( - os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), - "/etc/clickhouse-server/config.d/enable_keeper.xml", - ) - - if len(feature_flags) > 0: - feature_flags_config = "" - - for feature, is_enabled in feature_flags: - feature_flags_config += f"<{feature}>{is_enabled}<\\/{feature}>" - - feature_flags_config += "<\\/feature_flags>" - - node.replace_in_config( - "/etc/clickhouse-server/config.d/enable_keeper.xml", - "", - feature_flags_config, - ) - - node.start_clickhouse(retry_start=not expect_fail) - keeper_utils.wait_until_connected(cluster, node) - - -def test_keeper_feature_flags(started_cluster): - restart_clickhouse() - - def assert_feature_flags(feature_flags): - res = keeper_utils.send_4lw_cmd(started_cluster, node, "ftfl") - - for feature, is_enabled in feature_flags: - node.wait_for_log_line( - f"ZooKeeperClient: Keeper feature flag {feature.upper()}: {'enabled' if is_enabled else 'disabled'}", - look_behind_lines=1000, - ) - - node.wait_for_log_line( - f"KeeperContext: Keeper feature flag {feature.upper()}: {'enabled' if is_enabled else 'disabled'}", - look_behind_lines=1000, - ) - - assert f"{feature}\t{1 if is_enabled else 0}" in res - - assert_feature_flags( - [("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)] - ) - - feature_flags = [("multi_read", 0), ("check_not_exists", 1)] - restart_clickhouse(feature_flags) - assert_feature_flags(feature_flags + [("filtered_list", 1)]) - - feature_flags = [("multi_read", 0), ("check_not_exists", 0), ("filtered_list", 0)] - restart_clickhouse(feature_flags) - assert_feature_flags(feature_flags) - -# Look at other test config to enable 9000. -# E AssertionError: assert 'Code: 210. DB::NetException: Connection refused (172.16.1.2:9000). (NETWORK_ERROR)\n\n' == 'never-down-42' -# E - never-down-42 -# E + Code: 210. DB::NetException: Connection refused (172.16.1.2:9000). (NETWORK_ERROR) -# E + - with pytest.raises(Exception): - restart_clickhouse([("invalid_feature", 1)], expect_fail=True) - - resp = node.query_and_get_error("SELECT value FROM system.zookeeper where path = '/keeper' and name='availability_zone'") - assert resp == 'never-down-42' +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_zookeeper_fallback_session/test.py b/tests/integration/test_zookeeper_fallback_session/test.py index 3828a6afbb3..9afabfa3da3 100644 --- a/tests/integration/test_zookeeper_fallback_session/test.py +++ b/tests/integration/test_zookeeper_fallback_session/test.py @@ -98,5 +98,3 @@ def test_fallback_session(started_cluster: ClickHouseCluster): ) == "2\n" ) - resp = node1.query_with_retry("SELECT value FROM system.zookeeper where path = '/keeper' and name='availability_zone'") - assert resp == 'never-down-42' \ No newline at end of file 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 0196/1190] 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 0197/1190] 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 0198/1190] 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 ea5796fcb6860d7bb1854f8ad39ab7573941cb87 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Nov 2023 16:46:07 +0100 Subject: [PATCH 0199/1190] Fix --- src/Databases/DatabaseOnDisk.cpp | 4 ++-- src/Databases/DatabasesCommon.cpp | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 7 +++--- src/Interpreters/InterpreterCreateQuery.h | 8 ++++++- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- .../parseColumnsListForTableFunction.cpp | 4 ++-- ...07_backup_restore_flatten_nested.reference | 4 ++++ .../02907_backup_restore_flatten_nested.sh | 22 +++++++++++++++++++ 8 files changed, 42 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02907_backup_restore_flatten_nested.reference create mode 100755 tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 01afbdcaa57..96c084a261c 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -73,7 +73,7 @@ std::pair createTableFromAST( auto table_function = factory.get(table_function_ast, context); ColumnsDescription columns; if (ast_create_query.columns_list && ast_create_query.columns_list->columns) - columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true); + columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true, false); StoragePtr storage = table_function->execute(table_function_ast, context, ast_create_query.getTable(), std::move(columns)); storage->renameInMemory(ast_create_query); return {ast_create_query.getTable(), storage}; @@ -99,7 +99,7 @@ std::pair createTableFromAST( } else { - columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true); + columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true, false); constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints); } } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 3ffa08f8ec7..9b85e7194d3 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -372,6 +372,7 @@ void DatabaseWithOwnTablesBase::createTableRestoredFromBackup(const ASTPtr & cre /// Creates a table by executing a "CREATE TABLE" query. InterpreterCreateQuery interpreter{create_table_query, local_context}; interpreter.setInternal(true); + interpreter.setIsRestoreFromBackup(true); interpreter.execute(); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 252f45677ef..ca865264fcf 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -480,7 +480,7 @@ ASTPtr InterpreterCreateQuery::formatProjections(const ProjectionsDescription & } ColumnsDescription InterpreterCreateQuery::getColumnsDescription( - const ASTExpressionList & columns_ast, ContextPtr context_, bool attach) + const ASTExpressionList & columns_ast, ContextPtr context_, bool attach, bool is_restore_from_backup) { /// First, deduce implicit types. @@ -645,7 +645,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( res.add(std::move(column)); } - if (!attach && context_->getSettingsRef().flatten_nested) + if (!attach && !is_restore_from_backup && context_->getSettingsRef().flatten_nested) res.flattenNested(); if (res.getAllPhysical().empty()) @@ -692,7 +692,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti if (create.columns_list->columns) { - properties.columns = getColumnsDescription(*create.columns_list->columns, getContext(), create.attach); + properties.columns = getColumnsDescription(*create.columns_list->columns, getContext(), create.attach, is_restore_from_backup); } if (create.columns_list->indices) @@ -752,7 +752,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } else if (create.select) { - Block as_select_sample; if (getContext()->getSettingsRef().allow_experimental_analyzer) diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 67339dea928..0843a7ad15a 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -66,9 +66,14 @@ public: need_ddl_guard = false; } + void setIsRestoreFromBackup(bool is_restore_from_backup_) + { + is_restore_from_backup = is_restore_from_backup_; + } + /// Obtain information about columns, their types, default values and column comments, /// for case when columns in CREATE query is specified explicitly. - static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, ContextPtr context, bool attach); + static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, ContextPtr context, bool attach, bool is_restore_from_backup); static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints); static void prepareOnClusterQuery(ASTCreateQuery & create, ContextPtr context, const String & cluster_name); @@ -116,6 +121,7 @@ private: bool force_attach = false; bool load_database_without_tables = false; bool need_ddl_guard = true; + bool is_restore_from_backup = false; mutable String as_database_saved; mutable String as_table_saved; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 07a1ae7d170..ea4a5605256 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -745,7 +745,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, auto & create = create_ast->as(); create.attach = true; - auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context, true); + auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context, true, false); auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints); auto data_path = database->getTableDataPath(create); diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index 196053fe509..87f76f7f824 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -73,7 +73,7 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, con if (!columns_list) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not cast AST to ASTExpressionList"); - auto columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + auto columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false, false); auto validation_settings = DataTypeValidationSettings(context->getSettingsRef()); for (const auto & [name, type] : columns.getAll()) validateDataType(type, validation_settings); @@ -100,7 +100,7 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip try { - columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false, false); auto validation_settings = DataTypeValidationSettings(context->getSettingsRef()); for (const auto & [name, type] : columns.getAll()) validateDataType(type, validation_settings); diff --git a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.reference b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.reference new file mode 100644 index 00000000000..0375a7263a7 --- /dev/null +++ b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.reference @@ -0,0 +1,4 @@ +BACKUP_CREATED +CREATE TABLE test.test\n(\n `test` Array(Tuple(foo String, bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +RESTORED +CREATE TABLE test.test\n(\n `test` Array(Tuple(foo String, bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh new file mode 100755 index 00000000000..4ec269695a4 --- /dev/null +++ b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh @@ -0,0 +1,22 @@ +#!/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 --query " +drop table if exists test; +set flatten_nested = 0; +create table test (test Array(Tuple(foo String, bar Float64))) ENGINE = MergeTree() ORDER BY tuple(); +backup table test on cluster test_shard_localhost to Disk('default', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); +" | grep -o "BACKUP_CREATED" + +${CLICKHOUSE_CLIENT} --query "show create table test" + +${CLICKHOUSE_CLIENT} -nm --query " +drop table test sync; +set flatten_nested = 1; +restore table test on cluster test_shard_localhost from Disk('default', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); +" | grep -o "RESTORED" + +${CLICKHOUSE_CLIENT} --query "show create table test" From c5b1c200a9bd7b212c5f9642c1b0fd42c14ec412 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 3 Nov 2023 17:19:29 +0100 Subject: [PATCH 0200/1190] Update 02907_backup_restore_flatten_nested.sh --- .../0_stateless/02907_backup_restore_flatten_nested.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh index 4ec269695a4..a7d3ad23b1c 100755 --- a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh +++ b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh @@ -8,7 +8,7 @@ ${CLICKHOUSE_CLIENT} -nm --query " drop table if exists test; set flatten_nested = 0; create table test (test Array(Tuple(foo String, bar Float64))) ENGINE = MergeTree() ORDER BY tuple(); -backup table test on cluster test_shard_localhost to Disk('default', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); +backup table test on cluster test_shard_localhost to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "BACKUP_CREATED" ${CLICKHOUSE_CLIENT} --query "show create table test" @@ -16,7 +16,7 @@ ${CLICKHOUSE_CLIENT} --query "show create table test" ${CLICKHOUSE_CLIENT} -nm --query " drop table test sync; set flatten_nested = 1; -restore table test on cluster test_shard_localhost from Disk('default', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); +restore table test on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" ${CLICKHOUSE_CLIENT} --query "show create table test" From 9b9a6f8afcb652cd952344c844ec45fa3b1b018d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Nov 2023 17:31:23 +0100 Subject: [PATCH 0201/1190] Parallelize BackupEntriesCollector --- src/Backups/BackupEntriesCollector.cpp | 31 +++++++++++++++++++++----- src/Backups/BackupEntriesCollector.h | 8 ++++++- src/Backups/BackupsWorker.cpp | 4 +++- 3 files changed, 36 insertions(+), 7 deletions(-) diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index c216ae02ce2..ac5cd369556 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -82,7 +82,8 @@ BackupEntriesCollector::BackupEntriesCollector( const BackupSettings & backup_settings_, std::shared_ptr backup_coordination_, const ReadSettings & read_settings_, - const ContextPtr & context_) + const ContextPtr & context_, + ThreadPool & threadpool_) : backup_query_elements(backup_query_elements_) , backup_settings(backup_settings_) , backup_coordination(backup_coordination_) @@ -101,10 +102,14 @@ BackupEntriesCollector::BackupEntriesCollector( context->getSettingsRef().backup_restore_keeper_max_retries, context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms, context->getSettingsRef().backup_restore_keeper_retry_max_backoff_ms) + , threadpool(threadpool_) { } -BackupEntriesCollector::~BackupEntriesCollector() = default; +BackupEntriesCollector::~BackupEntriesCollector() +{ + threadpool.wait(); +} BackupEntries BackupEntriesCollector::run() { @@ -739,7 +744,13 @@ void BackupEntriesCollector::makeBackupEntriesForTablesData() return; for (const auto & table_name : table_infos | boost::adaptors::map_keys) - makeBackupEntriesForTableData(table_name); + { + threadpool.scheduleOrThrowOnError([&]() + { + makeBackupEntriesForTableData(table_name); + }); + } + threadpool.wait(); } void BackupEntriesCollector::makeBackupEntriesForTableData(const QualifiedTableName & table_name) @@ -775,20 +786,28 @@ void BackupEntriesCollector::makeBackupEntriesForTableData(const QualifiedTableN } } -void BackupEntriesCollector::addBackupEntry(const String & file_name, BackupEntryPtr backup_entry) +void BackupEntriesCollector::addBackupEntryUnlocked(const String & file_name, BackupEntryPtr backup_entry) { if (current_stage == Stage::WRITING_BACKUP) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding backup entries is not allowed"); backup_entries.emplace_back(file_name, backup_entry); } +void BackupEntriesCollector::addBackupEntry(const String & file_name, BackupEntryPtr backup_entry) +{ + std::lock_guard lock(mutex); + addBackupEntryUnlocked(file_name, backup_entry); +} + void BackupEntriesCollector::addBackupEntry(const std::pair & backup_entry) { - addBackupEntry(backup_entry.first, backup_entry.second); + std::lock_guard lock(mutex); + addBackupEntryUnlocked(backup_entry.first, backup_entry.second); } void BackupEntriesCollector::addBackupEntries(const BackupEntries & backup_entries_) { + std::lock_guard lock(mutex); if (current_stage == Stage::WRITING_BACKUP) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of backup entries is not allowed"); insertAtEnd(backup_entries, backup_entries_); @@ -796,6 +815,7 @@ void BackupEntriesCollector::addBackupEntries(const BackupEntries & backup_entri void BackupEntriesCollector::addBackupEntries(BackupEntries && backup_entries_) { + std::lock_guard lock(mutex); if (current_stage == Stage::WRITING_BACKUP) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of backup entries is not allowed"); insertAtEnd(backup_entries, std::move(backup_entries_)); @@ -803,6 +823,7 @@ void BackupEntriesCollector::addBackupEntries(BackupEntries && backup_entries_) void BackupEntriesCollector::addPostTask(std::function task) { + std::lock_guard lock(mutex); if (current_stage == Stage::WRITING_BACKUP) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of post tasks is not allowed"); post_tasks.push(std::move(task)); diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index 2501ff75bbd..33133cb3403 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -31,7 +31,8 @@ public: const BackupSettings & backup_settings_, std::shared_ptr backup_coordination_, const ReadSettings & read_settings_, - const ContextPtr & context_); + const ContextPtr & context_, + ThreadPool & threadpool_); ~BackupEntriesCollector(); /// Collects backup entries and returns the result. @@ -89,6 +90,8 @@ private: void makeBackupEntriesForTablesData(); void makeBackupEntriesForTableData(const QualifiedTableName & table_name); + void addBackupEntryUnlocked(const String & file_name, BackupEntryPtr backup_entry); + void runPostTasks(); Strings setStage(const String & new_stage, const String & message = ""); @@ -169,6 +172,9 @@ private: BackupEntries backup_entries; std::queue> post_tasks; std::vector access_counters; + + ThreadPool & threadpool; + std::mutex mutex; }; } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index da814dcbc08..7e00137739b 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -454,7 +454,9 @@ void BackupsWorker::doBackup( /// Prepare backup entries. BackupEntries backup_entries; { - BackupEntriesCollector backup_entries_collector{backup_query->elements, backup_settings, backup_coordination, backup_create_params.read_settings, context}; + BackupEntriesCollector backup_entries_collector( + backup_query->elements, backup_settings, backup_coordination, + backup_create_params.read_settings, context, *backups_thread_pool); backup_entries = backup_entries_collector.run(); } From bdf49a69f47ad4e1f63378297cb4f604d5da712f Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Fri, 3 Nov 2023 16:22:05 +0000 Subject: [PATCH 0202/1190] throw exception Signed-off-by: Jianfei Hu --- src/IO/S3/Credentials.cpp | 22 +++++++++++++++++----- src/IO/S3/Credentials.h | 2 +- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 3b9a5bc48f8..eb87568c8ab 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -274,7 +274,7 @@ String getGCPAvailabilityZoneOrException() return zone_info[3]; } -std::variant getRunningAvailabilityZoneImpl() +String getRunningAvailabilityZoneImpl() { LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); try @@ -291,16 +291,28 @@ std::variant getRunningAvailabilityZoneImpl() } catch (const DB::Exception & gcp_ex) { - return std::make_exception_ptr(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())); + 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 = getRunningAvailabilityZoneImpl(); - if (const auto * az = std::get_if(&az_or_exception)) + 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)); diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 7df492c549a..a978679348f 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -59,7 +59,7 @@ public: virtual Aws::String getCurrentRegion() const; - friend std::variant getRunningAvailabilityZoneImpl(); + friend String getRunningAvailabilityZoneImpl(); private: std::pair getEC2MetadataToken(const std::string & user_agent_string) const; From 3e8ad144232bf307e332e5c5258753b84d681d95 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 Nov 2023 16:07:05 +0100 Subject: [PATCH 0203/1190] Implement digest helpers for different objects --- tests/ci/ccache_utils.py | 4 +-- tests/ci/digest_helper.py | 63 +++++++++++++++++++++++++++++++++++++++ tests/ci/s3_helper.py | 10 ------- 3 files changed, 65 insertions(+), 12 deletions(-) create mode 100644 tests/ci/digest_helper.py diff --git a/tests/ci/ccache_utils.py b/tests/ci/ccache_utils.py index 75a026d2524..6ccaa8c80e0 100644 --- a/tests/ci/ccache_utils.py +++ b/tests/ci/ccache_utils.py @@ -3,13 +3,13 @@ import logging import os import shutil -from hashlib import md5 from pathlib import Path import requests # type: ignore from build_download_helper import download_build_with_progress, DownloadException from compress_files import decompress_fast, compress_fast +from digest_helper import digest_path from env_helper import S3_DOWNLOAD, S3_BUILDS_BUCKET from git_helper import git_runner from s3_helper import S3Helper @@ -108,7 +108,7 @@ class CargoCache: s3_helper: S3Helper, ): self._cargo_lock_file = Path(git_runner.cwd) / "rust" / "Cargo.lock" - self.lock_hash = md5(self._cargo_lock_file.read_bytes()).hexdigest() + self.lock_hash = digest_path(self._cargo_lock_file).hexdigest() self.directory = directory self.archive_name = f"Cargo_cache_{self.lock_hash}.tar.zst" self.temp_path = temp_path diff --git a/tests/ci/digest_helper.py b/tests/ci/digest_helper.py new file mode 100644 index 00000000000..a97c541cc65 --- /dev/null +++ b/tests/ci/digest_helper.py @@ -0,0 +1,63 @@ +#!/usr/bin/env python3 + +from hashlib import md5 +from logging import getLogger +from pathlib import Path +from typing import TYPE_CHECKING +from sys import modules + +if TYPE_CHECKING: + from hashlib import ( # pylint:disable=no-name-in-module,ungrouped-imports + _Hash as HASH, + ) +else: + HASH = "_Hash" + +logger = getLogger(__name__) + + +def _digest_file(file: Path) -> HASH: + assert file.is_file() + md5_hash = md5() + with open(file, "rb") as fd: + for chunk in iter(lambda: fd.read(4096), b""): + md5_hash.update(chunk) + return md5_hash + + +def _digest_directory(directory: Path) -> HASH: + assert directory.is_dir() + md5_hash = md5() + for p in sorted(directory.rglob("*")): + if p.is_symlink() and p.is_dir(): + # The symlink directory is not listed recursively, so we process it manually + md5_hash.update(_digest_directory(p).digest()) + if p.is_file(): + md5_hash.update(_digest_file(p).digest()) + return md5_hash + + +def digest_path(path: Path) -> HASH: + """Calculates md5 hash of the path, either it's directory or file""" + if path.is_dir(): + return _digest_directory(path) + if path.is_file(): + return _digest_file(path) + return md5() + + +def digest_script(path_str: str) -> HASH: + """Accepts value of the __file__ executed script and calculates the md5 hash for it""" + path = Path(path_str) + parent = path.parent + md5_hash = md5() + try: + for script in modules.values(): + script_path = getattr(script, "__file__", "") + if parent.absolute().as_posix() in script_path: + logger.debug("Updating the hash with %s", script_path) + md5_hash.update(_digest_file(Path(script_path)).digest()) + except RuntimeError: + logger.warning("The modules size has changed, retry calculating digest") + return digest_script(path_str) + return md5_hash diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index bb047b4f4ef..f94f7f60bb6 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -1,5 +1,4 @@ # -*- coding: utf-8 -*- -import hashlib import logging import re import shutil @@ -22,15 +21,6 @@ from env_helper import ( from compress_files import compress_file_fast -def _md5(fname): - hash_md5 = hashlib.md5() - with open(fname, "rb") as f: - for chunk in iter(lambda: f.read(4096), b""): - hash_md5.update(chunk) - logging.debug("MD5 for %s is %s", fname, hash_md5.hexdigest()) - return hash_md5.hexdigest() - - def _flatten_list(lst): result = [] for elem in lst: From f7c5602da45d14c17063a176be5db9da6be8bd89 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 Nov 2023 16:30:14 +0100 Subject: [PATCH 0204/1190] Add digest_paths --- tests/ci/digest_helper.py | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/ci/digest_helper.py b/tests/ci/digest_helper.py index a97c541cc65..21febeafc57 100644 --- a/tests/ci/digest_helper.py +++ b/tests/ci/digest_helper.py @@ -3,7 +3,7 @@ from hashlib import md5 from logging import getLogger from pathlib import Path -from typing import TYPE_CHECKING +from typing import TYPE_CHECKING, Iterable from sys import modules if TYPE_CHECKING: @@ -46,6 +46,15 @@ def digest_path(path: Path) -> HASH: return md5() +def digest_paths(paths: Iterable[Path]) -> HASH: + """Calculates aggregated md5 hash of passed paths. The order matters""" + md5_hash = md5() + for path in paths: + if path.exists(): + md5_hash.update(digest_path(path).digest()) + return md5_hash + + def digest_script(path_str: str) -> HASH: """Accepts value of the __file__ executed script and calculates the md5 hash for it""" path = Path(path_str) From e352e7bfba92d2dd209d5aa7fde516316abadb0d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 Nov 2023 17:41:53 +0100 Subject: [PATCH 0205/1190] Change digest API to update the single hash object --- tests/ci/digest_helper.py | 43 +++++++++++++++++++-------------------- 1 file changed, 21 insertions(+), 22 deletions(-) diff --git a/tests/ci/digest_helper.py b/tests/ci/digest_helper.py index 21febeafc57..69a62fa62b5 100644 --- a/tests/ci/digest_helper.py +++ b/tests/ci/digest_helper.py @@ -3,7 +3,7 @@ from hashlib import md5 from logging import getLogger from pathlib import Path -from typing import TYPE_CHECKING, Iterable +from typing import TYPE_CHECKING, Iterable, Optional from sys import modules if TYPE_CHECKING: @@ -16,43 +16,42 @@ else: logger = getLogger(__name__) -def _digest_file(file: Path) -> HASH: +def _digest_file(file: Path, hash_object: HASH) -> None: assert file.is_file() - md5_hash = md5() with open(file, "rb") as fd: for chunk in iter(lambda: fd.read(4096), b""): - md5_hash.update(chunk) - return md5_hash + hash_object.update(chunk) -def _digest_directory(directory: Path) -> HASH: +def _digest_directory(directory: Path, hash_object: HASH) -> None: assert directory.is_dir() - md5_hash = md5() for p in sorted(directory.rglob("*")): if p.is_symlink() and p.is_dir(): # The symlink directory is not listed recursively, so we process it manually - md5_hash.update(_digest_directory(p).digest()) + (_digest_directory(p, hash_object)) if p.is_file(): - md5_hash.update(_digest_file(p).digest()) - return md5_hash + (_digest_file(p, hash_object)) -def digest_path(path: Path) -> HASH: - """Calculates md5 hash of the path, either it's directory or file""" +def digest_path(path: Path, hash_object: Optional[HASH] = None) -> HASH: + """Calculates md5 (or updates existing hash_object) hash of the path, either it's + directory or file""" + hash_object = hash_object or md5() if path.is_dir(): - return _digest_directory(path) - if path.is_file(): - return _digest_file(path) - return md5() + _digest_directory(path, hash_object) + elif path.is_file(): + _digest_file(path, hash_object) + return hash_object -def digest_paths(paths: Iterable[Path]) -> HASH: - """Calculates aggregated md5 hash of passed paths. The order matters""" - md5_hash = md5() +def digest_paths(paths: Iterable[Path], hash_object: Optional[HASH] = None) -> HASH: + """Calculates aggregated md5 (or updates existing hash_object) hash of passed paths. + The order matters""" + hash_object = hash_object or md5() for path in paths: if path.exists(): - md5_hash.update(digest_path(path).digest()) - return md5_hash + digest_path(path, hash_object) + return hash_object def digest_script(path_str: str) -> HASH: @@ -65,7 +64,7 @@ def digest_script(path_str: str) -> HASH: script_path = getattr(script, "__file__", "") if parent.absolute().as_posix() in script_path: logger.debug("Updating the hash with %s", script_path) - md5_hash.update(_digest_file(Path(script_path)).digest()) + _digest_file(Path(script_path), md5_hash) except RuntimeError: logger.warning("The modules size has changed, retry calculating digest") return digest_script(path_str) From d95262622bc2fa1824957937225f553e3ddc03de Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 3 Nov 2023 16:47:06 +0000 Subject: [PATCH 0206/1190] init --- docs/en/operations/settings/settings.md | 11 +++ src/Core/Settings.h | 1 + .../QueryPlanOptimizationSettings.h | 1 + .../optimizeUseAggregateProjection.cpp | 29 ++++++- .../optimizeUseNormalProjection.cpp | 29 ++++++- ...eferred_optimize_projection_name.reference | 6 ++ ...2907_preferred_optimize_projection_name.sh | 85 +++++++++++++++++++ 7 files changed, 159 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02907_preferred_optimize_projection_name.reference create mode 100755 tests/queries/0_stateless/02907_preferred_optimize_projection_name.sh diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index bfe120be4a3..d25e2a1fef4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3954,6 +3954,17 @@ Possible values: Default value: `''`. +## preferred_optimize_projection_name {#preferred_optimize_projection_name} + +If it is set to a non-empty string, ClickHouse will try to apply specified projection in query. + + +Possible values: + +- string: name of preferred projection + +Default value: `''`. + ## alter_sync {#alter-sync} Allows to set up waiting for actions to be executed on replicas by [ALTER](../../sql-reference/statements/alter/index.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 609ade4cdc0..2161db5a422 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -605,6 +605,7 @@ class IColumn; M(Bool, optimize_use_implicit_projections, true, "Automatically choose implicit projections to perform SELECT query", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(String, force_optimize_projection_name, "", "If it is set to a non-empty string, check that this projection is used in the query at least once.", 0) \ + M(String, preferred_optimize_projection_name, "", "If it is set to a non-empty string, ClickHouse tries to apply specified projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ M(Bool, insert_null_as_default, true, "Insert DEFAULT values instead of NULL in INSERT SELECT (UNION ALL)", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 9315b4b9ee4..6e32a5495ec 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -42,6 +42,7 @@ struct QueryPlanOptimizationSettings bool optimize_projection = false; bool force_use_projection = false; String force_projection_name; + String preferred_projection_name; bool optimize_use_implicit_projections = false; static QueryPlanOptimizationSettings fromSettings(const Settings & from); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index cef72fa02b3..cf67ece2fbf 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -444,8 +444,33 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( const auto & projections = metadata->projections; std::vector agg_projections; - for (const auto & projection : projections) - if (projection.type == ProjectionDescription::Type::Aggregate) + + const auto & proj_name_from_settings = context->getSettings().preferred_optimize_projection_name.value; + bool is_projection_found = false; + + // Here we iterate over the projections and check if we have the same projections as we specified in preferred_projection_name + if (!proj_name_from_settings.empty()) + { + for (const auto & projection : projections) + { + if (projection.type == ProjectionDescription::Type::Aggregate) + { + size_t last_dot_pos = projection.name.find_last_of('.'); + std::string projection_name = (last_dot_pos != std::string::npos) ? projection.name.substr(last_dot_pos + 1) : projection.name; + if (proj_name_from_settings == projection_name) + { + agg_projections.push_back(&projection); + is_projection_found = true; + break; + } + } + } + if (!is_projection_found) + throw Exception(ErrorCodes::INCORRECT_DATA, "Projection {} is specified in setting force_optimize_projection_name but not used", + proj_name_from_settings); + } + else + for (const auto & projection : projections) agg_projections.push_back(&projection); bool can_use_minmax_projection = allow_implicit_projections && metadata->minmax_count_projection diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 727afcb1a99..b60a9995d13 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include @@ -131,6 +133,26 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); + // Here we iterate over the projections and check if we have the same projections as we specified in preferred_projection_name + bool is_projection_found = false; + const auto & proj_name_from_settings = context->getSettings().preferred_optimize_projection_name.value; + if (!proj_name_from_settings.empty()) + { + for (const auto * projection : normal_projections) + { + size_t last_dot_pos = projection->name.find_last_of('.'); + std::string projection_name = (last_dot_pos != std::string::npos) ? projection->name.substr(last_dot_pos + 1) : projection->name; + if (projection_name == proj_name_from_settings) + { + is_projection_found = true; + break; + } + } + if (!is_projection_found) + throw Exception(ErrorCodes::INCORRECT_DATA, "Projection {} is specified in setting force_optimize_projection_name but not used", + proj_name_from_settings); + } + for (const auto * projection : normal_projections) { if (!hasAllRequiredColumns(projection, required_columns)) @@ -153,7 +175,12 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (candidate.sum_marks >= ordinary_reading_marks) continue; - if (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks) + size_t last_dot_pos = projection->name.find_last_of('.'); + std::string projection_name = (last_dot_pos != std::string::npos) ? projection->name.substr(last_dot_pos + 1) : projection->name; + + if (!is_projection_found && (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks)) + best_candidate = &candidate; + else if (is_projection_found && projection_name == proj_name_from_settings) best_candidate = &candidate; } diff --git a/tests/queries/0_stateless/02907_preferred_optimize_projection_name.reference b/tests/queries/0_stateless/02907_preferred_optimize_projection_name.reference new file mode 100644 index 00000000000..3bbb71b6c71 --- /dev/null +++ b/tests/queries/0_stateless/02907_preferred_optimize_projection_name.reference @@ -0,0 +1,6 @@ +test +projection_test_by_string +Executing query with setting +test +projection_test_by_more +0 diff --git a/tests/queries/0_stateless/02907_preferred_optimize_projection_name.sh b/tests/queries/0_stateless/02907_preferred_optimize_projection_name.sh new file mode 100755 index 00000000000..6e7f55804bd --- /dev/null +++ b/tests/queries/0_stateless/02907_preferred_optimize_projection_name.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " +CREATE TABLE test ( + test_id UInt64, + test_name String, + test_count Nullable(Float64), + test_string String, + PROJECTION projection_test_by_string ( + SELECT test_string, + sum(test_count) + GROUP BY test_id, + test_string, + test_name + ), + PROJECTION projection_test_by_more ( + SELECT test_string, + test_name, + sum(test_count) + GROUP BY test_id, + test_string, + test_name + ) +) ENGINE = MergeTree +ORDER BY test_string;" + +$CLICKHOUSE_CLIENT -q " +INSERT INTO test +SELECT number, + 'test', + 1.* (number / 2), + 'test' +FROM numbers(100, 500);" + +$CLICKHOUSE_CLIENT --query_id '02907_test' -q " +SELECT test_string +FROM test +WHERE (test_id > 50) + AND (test_id < 150) +GROUP BY test_string;" + +$CLICKHOUSE_CLIENT -q " +SELECT projections +FROM system.query_log +WHERE query_id = '02907_test' AND arrayElement(projections, 1) LIKE '%projection_test_by_string' +LIMIT 1;" | grep -o "projection_test_by_string" || true + +$CLICKHOUSE_CLIENT -q " +SELECT projections +FROM system.query_log +WHERE query_id = '02907_test' AND arrayElement(projections, 1) LIKE '%projection_test_by_more' +LIMIT 1;" | grep -o "projection_test_by_more" || true + +echo "Executing query with setting" + +$CLICKHOUSE_CLIENT --query_id '02907_test_1' --preferred_optimize_projection_name 'projection_test_by_more' -q " +SELECT test_string +FROM test +WHERE (test_id > 50) + AND (test_id < 150) +GROUP BY test_string;" + +$CLICKHOUSE_CLIENT -q " +SELECT projections +FROM system.query_log +WHERE query_id = '02907_test_1' AND arrayElement(projections, 1) LIKE '%projection_test_by_more' +LIMIT 1;" | grep -o "projection_test_by_more" || true + +$CLICKHOUSE_CLIENT -q " +SELECT projections +FROM system.query_log +WHERE query_id = '02907_test_1' AND arrayElement(projections, 1) LIKE '%projection_test_by_string' +LIMIT 1" | grep -o "projection_test_by_string" || true + +$CLICKHOUSE_CLIENT --query_id '02907_test_1' --preferred_optimize_projection_name 'non_existing_projection' -q " +SELECT test_string +FROM test +WHERE (test_id > 50) + AND (test_id < 150) +GROUP BY test_string;" 2>&1 | grep -c "BAD_ARGUMENTS" || true From d4b1ae1c6099344d8e1d2bf3c85e02eaa1e30c51 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 3 Nov 2023 17:52:18 +0100 Subject: [PATCH 0207/1190] Update 02907_backup_restore_flatten_nested.sh --- .../0_stateless/02907_backup_restore_flatten_nested.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh index a7d3ad23b1c..d4c50502dc9 100755 --- a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh +++ b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh @@ -8,7 +8,7 @@ ${CLICKHOUSE_CLIENT} -nm --query " drop table if exists test; set flatten_nested = 0; create table test (test Array(Tuple(foo String, bar Float64))) ENGINE = MergeTree() ORDER BY tuple(); -backup table test on cluster test_shard_localhost to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); +backup table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "BACKUP_CREATED" ${CLICKHOUSE_CLIENT} --query "show create table test" @@ -16,7 +16,7 @@ ${CLICKHOUSE_CLIENT} --query "show create table test" ${CLICKHOUSE_CLIENT} -nm --query " drop table test sync; set flatten_nested = 1; -restore table test on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); +restore table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" ${CLICKHOUSE_CLIENT} --query "show create table test" From 92588dfac79693d64ad3ab36932a4ee4ef7c033b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 3 Nov 2023 18:07:19 +0100 Subject: [PATCH 0208/1190] style fix --- .../QueryPlan/Optimizations/optimizeUseNormalProjection.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index b60a9995d13..f6f4591da47 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -15,6 +15,11 @@ namespace DB::QueryPlanOptimizations { +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + /// Normal projection analysis result in case it can be applied. /// For now, it is empty. /// Normal projection can be used only if it contains all required source columns. From 6d9b517b194a83c17623b1f018b7c90b863d80a2 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 3 Nov 2023 17:21:37 +0000 Subject: [PATCH 0209/1190] 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 0210/1190] 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 0211/1190] 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 0212/1190] 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 0213/1190] 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 0214/1190] 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 0215/1190] 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 0216/1190] 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 0217/1190] 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 0218/1190] 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 8182d51978727e9e91b80764151228b9b3daf046 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Nov 2023 01:11:53 +0100 Subject: [PATCH 0219/1190] Add table `system.symbols` --- src/Storages/System/StorageSystemSymbols.cpp | 111 +++++++++++++++++++ src/Storages/System/StorageSystemSymbols.h | 33 ++++++ src/Storages/System/attachSystemTables.cpp | 2 + 3 files changed, 146 insertions(+) create mode 100644 src/Storages/System/StorageSystemSymbols.cpp create mode 100644 src/Storages/System/StorageSystemSymbols.h diff --git a/src/Storages/System/StorageSystemSymbols.cpp b/src/Storages/System/StorageSystemSymbols.cpp new file mode 100644 index 00000000000..62d8b7ca58f --- /dev/null +++ b/src/Storages/System/StorageSystemSymbols.cpp @@ -0,0 +1,111 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +StorageSystemSymbols::StorageSystemSymbols(const StorageID & table_id_) + : IStorage(table_id_) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription( + { + {"symbol", std::make_shared()}, + {"address_begin", std::make_shared()}, + {"address_end", std::make_shared()}, + })); + setInMemoryMetadata(storage_metadata); +} + + +namespace +{ + +class SymbolsBlockSource : public ISource +{ +private: + using Iterator = std::vector::const_iterator; + Iterator it; + const Iterator end; + std::vector columns_mask; + UInt64 max_block_size; + +public: + SymbolsBlockSource( + Iterator begin_, + Iterator end_, + std::vector columns_mask_, + Block header, + UInt64 max_block_size_) + : ISource(std::move(header)) + , it(begin_), end(end_), columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_) + { + } + + String getName() const override { return "Symbols"; } + +protected: + Chunk generate() override + { + if (it == end) + return {}; + + MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); + + size_t rows_count = 0; + while (rows_count < max_block_size && it != end) + { + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(it->name); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(reinterpret_cast(it->address_begin)); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(reinterpret_cast(it->address_end)); + + ++rows_count; + ++it; + } + + return Chunk(std::move(res_columns), rows_count); + } +}; + +} + + +Pipe StorageSystemSymbols::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /* query_info */, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t max_block_size, + const size_t /*num_streams*/) +{ + context->getAccess()->checkAccess(AccessType::INTROSPECTION); + + storage_snapshot->check(column_names); + Block sample_block = storage_snapshot->metadata->getSampleBlock(); + auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + + const auto & symbols = SymbolIndex::instance().symbols(); + + return Pipe(std::make_shared( + symbols.cbegin(), symbols.cend(), std::move(columns_mask), std::move(res_block), max_block_size)); +} + +} diff --git a/src/Storages/System/StorageSystemSymbols.h b/src/Storages/System/StorageSystemSymbols.h new file mode 100644 index 00000000000..808c406b91d --- /dev/null +++ b/src/Storages/System/StorageSystemSymbols.h @@ -0,0 +1,33 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/** Implements the system table `symbols` for introspection of symbols in the ClickHouse binary. + */ +class StorageSystemSymbols final : public IStorage +{ +public: + explicit StorageSystemSymbols(const StorageID & table_id_); + + std::string getName() const override { return "SystemSymbols"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + bool isSystemStorage() const override { return true; } +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index ffa225fb929..56525f5b948 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -85,6 +85,7 @@ #include #include #include +#include #if USE_RDKAFKA #include @@ -151,6 +152,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) attach(context, system_database, "schema_inference_cache"); attach(context, system_database, "dropped_tables"); attach(context, system_database, "scheduler"); + attach(context, system_database, "symbols"); #if USE_RDKAFKA attach(context, system_database, "kafka_consumers"); #endif From 99fe7a46fcb1b9bdb3429f0c3d721c80707053fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Nov 2023 01:12:13 +0100 Subject: [PATCH 0220/1190] Minor changes --- src/Storages/System/StorageSystemParts.cpp | 9 +++------ src/Storages/System/StorageSystemTables.cpp | 12 ++++++++---- src/Storages/System/StorageSystemTimeZones.cpp | 3 ++- 3 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index ac38c9c97b1..d8d85725e21 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -1,25 +1,22 @@ -#include "StorageSystemParts.h" +#include #include #include #include -#include -#include #include #include #include #include #include #include -#include -#include #include -#include #include #include + namespace { + std::string_view getRemovalStateDescription(DB::DataPartRemovalState state) { switch (state) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 715c98ee92a..d888813f6ce 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -70,7 +69,10 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) } -static ColumnPtr getFilteredDatabases(const SelectQueryInfo & query_info, ContextPtr context) +namespace +{ + +ColumnPtr getFilteredDatabases(const SelectQueryInfo & query_info, ContextPtr context) { MutableColumnPtr column = ColumnString::create(); @@ -88,7 +90,7 @@ static ColumnPtr getFilteredDatabases(const SelectQueryInfo & query_info, Contex return block.getByPosition(0).column; } -static ColumnPtr getFilteredTables(const ASTPtr & query, const ColumnPtr & filtered_databases_column, ContextPtr context) +ColumnPtr getFilteredTables(const ASTPtr & query, const ColumnPtr & filtered_databases_column, ContextPtr context) { MutableColumnPtr column = ColumnString::create(); @@ -110,7 +112,7 @@ static ColumnPtr getFilteredTables(const ASTPtr & query, const ColumnPtr & filte /// Avoid heavy operation on tables if we only queried columns that we can get without table object. /// Otherwise it will require table initialization for Lazy database. -static bool needTable(const DatabasePtr & database, const Block & header) +bool needTable(const DatabasePtr & database, const Block & header) { if (database->getEngineName() != "Lazy") return true; @@ -602,6 +604,8 @@ private: std::string database_name; }; +} + Pipe StorageSystemTables::read( const Names & column_names, diff --git a/src/Storages/System/StorageSystemTimeZones.cpp b/src/Storages/System/StorageSystemTimeZones.cpp index dc3711812a6..e0d7d2a5c42 100644 --- a/src/Storages/System/StorageSystemTimeZones.cpp +++ b/src/Storages/System/StorageSystemTimeZones.cpp @@ -10,7 +10,8 @@ namespace DB { NamesAndTypesList StorageSystemTimeZones::getNamesAndTypes() { - return { + return + { {"time_zone", std::make_shared()}, }; } From 995e8a91e535af1de62bc6551f2186278b17699d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Nov 2023 01:12:29 +0100 Subject: [PATCH 0221/1190] Remove fat symbols --- src/Core/callOnTypeIndex.h | 9 +++++---- src/Functions/array/arrayElement.cpp | 8 ++++++-- src/Functions/castTypeToEither.h | 6 ++++-- 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/src/Core/callOnTypeIndex.h b/src/Core/callOnTypeIndex.h index 39ce37c4c13..c6a9e4c842b 100644 --- a/src/Core/callOnTypeIndex.h +++ b/src/Core/callOnTypeIndex.h @@ -4,6 +4,7 @@ #include + namespace DB { @@ -16,7 +17,7 @@ struct TypePair template -bool callOnBasicType(TypeIndex number, F && f) +static bool NO_INLINE callOnBasicType(TypeIndex number, F && f) { if constexpr (_int) { @@ -86,7 +87,7 @@ bool callOnBasicType(TypeIndex number, F && f) /// Unroll template using TypeIndex template -inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) +static NO_INLINE bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) { if constexpr (_int) { @@ -170,7 +171,7 @@ template class DataTypeDecimal; template -bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... args) +static NO_INLINE bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... args) { switch (number) { @@ -219,7 +220,7 @@ bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... args) } template -static bool callOnTwoTypeIndexes(TypeIndex left_type, TypeIndex right_type, F && func) +static NO_INLINE bool callOnTwoTypeIndexes(TypeIndex left_type, TypeIndex right_type, F && func) { return callOnIndexAndDataType(left_type, [&](const auto & left_types) -> bool { diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index d7c29070c91..fe4353a8878 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -31,6 +31,9 @@ namespace ErrorCodes extern const int ZERO_ARRAY_OR_TUPLE_INDEX; } +namespace +{ + namespace ArrayImpl { class NullMapBuilder; @@ -130,7 +133,6 @@ class NullMapBuilder { public: explicit operator bool() const { return src_null_map; } - bool operator!() const { return !src_null_map; } void initSource(const UInt8 * src_null_map_) { @@ -949,7 +951,7 @@ static constexpr bool areConvertibleTypes = && std::is_convertible_v); template -static bool castColumnNumeric(const IColumn * column, F && f) +static NO_INLINE bool castColumnNumeric(const IColumn * column, F && f) { return castTypeToEither< ColumnVector, @@ -1250,6 +1252,8 @@ ColumnPtr FunctionArrayElement::perform(const ColumnsWithTypeAndName & arguments return res; } +} + REGISTER_FUNCTION(ArrayElement) { diff --git a/src/Functions/castTypeToEither.h b/src/Functions/castTypeToEither.h index aa8330366f1..e2e30c455ab 100644 --- a/src/Functions/castTypeToEither.h +++ b/src/Functions/castTypeToEither.h @@ -5,15 +5,17 @@ namespace DB { + template -static bool castTypeToEither(const T * type, F && f) +static NO_INLINE bool castTypeToEither(const T * type, F && f) { return ((typeid_cast(type) && f(*typeid_cast(type))) || ...); } template -constexpr bool castTypeToEither(TypeList, const auto * type, auto && f) +static NO_INLINE bool castTypeToEither(TypeList, const auto * type, auto && f) { return ((typeid_cast(type) != nullptr && std::forward(f)(*typeid_cast(type))) || ...); } + } From 9976006ee6700e7f4aa08b285889051ac726a99b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Nov 2023 01:17:42 +0100 Subject: [PATCH 0222/1190] Add a test --- tests/queries/0_stateless/02911_system_symbols.reference | 1 + tests/queries/0_stateless/02911_system_symbols.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02911_system_symbols.reference create mode 100644 tests/queries/0_stateless/02911_system_symbols.sql diff --git a/tests/queries/0_stateless/02911_system_symbols.reference b/tests/queries/0_stateless/02911_system_symbols.reference new file mode 100644 index 00000000000..df30df3ce57 --- /dev/null +++ b/tests/queries/0_stateless/02911_system_symbols.reference @@ -0,0 +1 @@ +DB::StorageSystemSymbols::StorageSystemSymbols(DB::StorageID const&) diff --git a/tests/queries/0_stateless/02911_system_symbols.sql b/tests/queries/0_stateless/02911_system_symbols.sql new file mode 100644 index 00000000000..398ce77fbd4 --- /dev/null +++ b/tests/queries/0_stateless/02911_system_symbols.sql @@ -0,0 +1 @@ +SELECT demangle(symbol) AS x FROM system.symbols WHERE symbol LIKE '%StorageSystemSymbols%' ORDER BY x LIMIT 1 SETTINGS allow_introspection_functions = 1; From 2d1351c3a63e863e18c327ba1813729690724c3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Nov 2023 01:34:06 +0100 Subject: [PATCH 0223/1190] Remove useless header --- .../AggregateFunctionSumMap.cpp | 645 ++++++++++++++++- .../AggregateFunctionSumMap.h | 656 ------------------ src/Functions/if.cpp | 12 +- 3 files changed, 647 insertions(+), 666 deletions(-) delete mode 100644 src/AggregateFunctions/AggregateFunctionSumMap.h diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 8204db3ff10..ecc8a978388 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -1,9 +1,23 @@ #include -#include #include -#include #include -#include + +#include + +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include namespace DB @@ -12,13 +26,636 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } namespace { +template +struct AggregateFunctionMapData +{ + // Map needs to be ordered to maintain function properties + std::map merged_maps; +}; + +/** Aggregate function, that takes at least two arguments: keys and values, and as a result, builds a tuple of at least 2 arrays - + * ordered keys and variable number of argument values aggregated by corresponding keys. + * + * sumMap function is the most useful when using SummingMergeTree to sum Nested columns, which name ends in "Map". + * + * Example: sumMap(k, v...) of: + * k v + * [1,2,3] [10,10,10] + * [3,4,5] [10,10,10] + * [4,5,6] [10,10,10] + * [6,7,8] [10,10,10] + * [7,5,3] [5,15,25] + * [8,9,10] [20,20,20] + * will return: + * ([1,2,3,4,5,6,7,8,9,10],[10,10,45,20,35,20,15,30,20,20]) + * + * minMap and maxMap share the same idea, but calculate min and max correspondingly. + * + * NOTE: The implementation of these functions are "amateur grade" - not efficient and low quality. + */ + +template +class AggregateFunctionMapBase : public IAggregateFunctionDataHelper< + AggregateFunctionMapData>, Derived> +{ +private: + static constexpr auto STATE_VERSION_1_MIN_REVISION = 54452; + + DataTypePtr keys_type; + SerializationPtr keys_serialization; + DataTypes values_types; + Serializations values_serializations; + Serializations promoted_values_serializations; + +public: + using Base = IAggregateFunctionDataHelper< + AggregateFunctionMapData>, Derived>; + + AggregateFunctionMapBase(const DataTypePtr & keys_type_, + const DataTypes & values_types_, const DataTypes & argument_types_) + : Base(argument_types_, {} /* parameters */, createResultType(keys_type_, values_types_, getName())) + , keys_type(keys_type_) + , keys_serialization(keys_type->getDefaultSerialization()) + , values_types(values_types_) + { + values_serializations.reserve(values_types.size()); + promoted_values_serializations.reserve(values_types.size()); + for (const auto & type : values_types) + { + values_serializations.emplace_back(type->getDefaultSerialization()); + if (type->canBePromoted()) + { + if (type->isNullable()) + promoted_values_serializations.emplace_back( + makeNullable(removeNullable(type)->promoteNumericType())->getDefaultSerialization()); + else + promoted_values_serializations.emplace_back(type->promoteNumericType()->getDefaultSerialization()); + } + else + { + promoted_values_serializations.emplace_back(type->getDefaultSerialization()); + } + } + } + + bool isVersioned() const override { return true; } + + size_t getDefaultVersion() const override { return 1; } + + size_t getVersionFromRevision(size_t revision) const override + { + if (revision >= STATE_VERSION_1_MIN_REVISION) + return 1; + else + return 0; + } + + static DataTypePtr createResultType( + const DataTypePtr & keys_type_, + const DataTypes & values_types_, + const String & name_) + { + DataTypes types; + types.emplace_back(std::make_shared(keys_type_)); + + for (const auto & value_type : values_types_) + { + if constexpr (std::is_same_v) + { + if (!value_type->isSummable()) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Values for {} cannot be summed, passed type {}", + name_, value_type->getName()}; + } + + DataTypePtr result_type; + + if constexpr (overflow) + { + if (value_type->onlyNull()) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Cannot calculate {} of type {}", + name_, value_type->getName()}; + + // Overflow, meaning that the returned type is the same as + // the input type. Nulls are skipped. + result_type = removeNullable(value_type); + } + else + { + auto value_type_without_nullable = removeNullable(value_type); + + // No overflow, meaning we promote the types if necessary. + if (!value_type_without_nullable->canBePromoted()) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Values for {} are expected to be Numeric, Float or Decimal, passed type {}", + name_, value_type->getName()}; + + WhichDataType value_type_to_check(value_type_without_nullable); + + /// Do not promote decimal because of implementation issues of this function design + /// Currently we cannot get result column type in case of decimal we cannot get decimal scale + /// in method void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + /// If we decide to make this function more efficient we should promote decimal type during summ + if (value_type_to_check.isDecimal()) + result_type = value_type_without_nullable; + else + result_type = value_type_without_nullable->promoteNumericType(); + } + + types.emplace_back(std::make_shared(result_type)); + } + + return std::make_shared(types); + } + + bool allocatesMemoryInArena() const override { return false; } + + static const auto & getArgumentColumns(const IColumn**& columns) + { + if constexpr (tuple_argument) + { + return assert_cast(columns[0])->getColumns(); + } + else + { + return columns; + } + } + + void add(AggregateDataPtr __restrict place, const IColumn ** columns_, const size_t row_num, Arena *) const override + { + const auto & columns = getArgumentColumns(columns_); + + // Column 0 contains array of keys of known type + const ColumnArray & array_column0 = assert_cast(*columns[0]); + const IColumn::Offsets & offsets0 = array_column0.getOffsets(); + const IColumn & key_column = array_column0.getData(); + const size_t keys_vec_offset = offsets0[row_num - 1]; + const size_t keys_vec_size = (offsets0[row_num] - keys_vec_offset); + + // Columns 1..n contain arrays of numeric values to sum + auto & merged_maps = this->data(place).merged_maps; + for (size_t col = 0, size = values_types.size(); col < size; ++col) + { + const auto & array_column = assert_cast(*columns[col + 1]); + const IColumn & value_column = array_column.getData(); + const IColumn::Offsets & offsets = array_column.getOffsets(); + const size_t values_vec_offset = offsets[row_num - 1]; + const size_t values_vec_size = (offsets[row_num] - values_vec_offset); + + // Expect key and value arrays to be of same length + if (keys_vec_size != values_vec_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sizes of keys and values arrays do not match"); + + // Insert column values for all keys + for (size_t i = 0; i < keys_vec_size; ++i) + { + auto value = value_column[values_vec_offset + i]; + T key = static_cast(key_column[keys_vec_offset + i].get()); + + if (!keepKey(key)) + continue; + + decltype(merged_maps.begin()) it; + if constexpr (is_decimal) + { + // FIXME why is storing NearestFieldType not enough, and we + // have to check for decimals again here? + UInt32 scale = static_cast &>(key_column).getScale(); + it = merged_maps.find(DecimalField(key, scale)); + } + else + it = merged_maps.find(key); + + if (it != merged_maps.end()) + { + if (!value.isNull()) + { + if (it->second[col].isNull()) + it->second[col] = value; + else + applyVisitor(Visitor(value), it->second[col]); + } + } + else + { + // Create a value array for this key + Array new_values; + new_values.resize(size); + new_values[col] = value; + + if constexpr (is_decimal) + { + UInt32 scale = static_cast &>(key_column).getScale(); + merged_maps.emplace(DecimalField(key, scale), std::move(new_values)); + } + else + { + merged_maps.emplace(key, std::move(new_values)); + } + } + } + } + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + auto & merged_maps = this->data(place).merged_maps; + const auto & rhs_maps = this->data(rhs).merged_maps; + + for (const auto & elem : rhs_maps) + { + const auto & it = merged_maps.find(elem.first); + if (it != merged_maps.end()) + { + for (size_t col = 0; col < values_types.size(); ++col) + if (!elem.second[col].isNull()) + applyVisitor(Visitor(elem.second[col]), it->second[col]); + } + else + merged_maps[elem.first] = elem.second; + } + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override + { + if (!version) + version = getDefaultVersion(); + + const auto & merged_maps = this->data(place).merged_maps; + size_t size = merged_maps.size(); + writeVarUInt(size, buf); + + std::function serialize; + switch (*version) + { + case 0: + { + serialize = [&](size_t col_idx, const Array & values){ values_serializations[col_idx]->serializeBinary(values[col_idx], buf, {}); }; + break; + } + case 1: + { + serialize = [&](size_t col_idx, const Array & values){ promoted_values_serializations[col_idx]->serializeBinary(values[col_idx], buf, {}); }; + break; + } + } + + for (const auto & elem : merged_maps) + { + keys_serialization->serializeBinary(elem.first, buf, {}); + for (size_t col = 0; col < values_types.size(); ++col) + serialize(col, elem.second); + } + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena *) const override + { + if (!version) + version = getDefaultVersion(); + + auto & merged_maps = this->data(place).merged_maps; + size_t size = 0; + readVarUInt(size, buf); + + std::function deserialize; + switch (*version) + { + case 0: + { + deserialize = [&](size_t col_idx, Array & values){ values_serializations[col_idx]->deserializeBinary(values[col_idx], buf, {}); }; + break; + } + case 1: + { + deserialize = [&](size_t col_idx, Array & values){ promoted_values_serializations[col_idx]->deserializeBinary(values[col_idx], buf, {}); }; + break; + } + } + + for (size_t i = 0; i < size; ++i) + { + Field key; + keys_serialization->deserializeBinary(key, buf, {}); + + Array values; + values.resize(values_types.size()); + + for (size_t col = 0; col < values_types.size(); ++col) + deserialize(col, values); + + if constexpr (is_decimal) + merged_maps[key.get>()] = values; + else + merged_maps[key.get()] = values; + } + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + size_t num_columns = values_types.size(); + + // Final step does compaction of keys that have zero values, this mutates the state + auto & merged_maps = this->data(place).merged_maps; + + // Remove keys which are zeros or empty. This should be enabled only for sumMap. + if constexpr (compact) + { + for (auto it = merged_maps.cbegin(); it != merged_maps.cend();) + { + // Key is not compacted if it has at least one non-zero value + bool erase = true; + for (size_t col = 0; col < num_columns; ++col) + { + if (!it->second[col].isNull() && it->second[col] != values_types[col]->getDefault()) + { + erase = false; + break; + } + } + + if (erase) + it = merged_maps.erase(it); + else + ++it; + } + } + + size_t size = merged_maps.size(); + + auto & to_tuple = assert_cast(to); + auto & to_keys_arr = assert_cast(to_tuple.getColumn(0)); + auto & to_keys_col = to_keys_arr.getData(); + + // Advance column offsets + auto & to_keys_offsets = to_keys_arr.getOffsets(); + to_keys_offsets.push_back(to_keys_offsets.back() + size); + to_keys_col.reserve(size); + + for (size_t col = 0; col < num_columns; ++col) + { + auto & to_values_arr = assert_cast(to_tuple.getColumn(col + 1)); + auto & to_values_offsets = to_values_arr.getOffsets(); + to_values_offsets.push_back(to_values_offsets.back() + size); + to_values_arr.getData().reserve(size); + } + + // Write arrays of keys and values + for (const auto & elem : merged_maps) + { + // Write array of keys into column + to_keys_col.insert(elem.first); + + // Write 0..n arrays of values + for (size_t col = 0; col < num_columns; ++col) + { + auto & to_values_col = assert_cast(to_tuple.getColumn(col + 1)).getData(); + if (elem.second[col].isNull()) + to_values_col.insertDefault(); + else + to_values_col.insert(elem.second[col]); + } + } + } + + bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } + String getName() const override { return Derived::getNameImpl(); } +}; + +template +class AggregateFunctionSumMap final : + public AggregateFunctionMapBase, FieldVisitorSum, overflow, tuple_argument, true> +{ +private: + using Self = AggregateFunctionSumMap; + using Base = AggregateFunctionMapBase; + +public: + AggregateFunctionSumMap(const DataTypePtr & keys_type_, + DataTypes & values_types_, const DataTypes & argument_types_, + const Array & params_) + : Base{keys_type_, values_types_, argument_types_} + { + // The constructor accepts parameters to have a uniform interface with + // sumMapFiltered, but this function doesn't have any parameters. + assertNoParameters(getNameImpl(), params_); + } + + static String getNameImpl() + { + if constexpr (overflow) + { + return "sumMapWithOverflow"; + } + else + { + return "sumMap"; + } + } + + bool keepKey(const T &) const { return true; } +}; + + +template +class AggregateFunctionSumMapFiltered final : + public AggregateFunctionMapBase, + FieldVisitorSum, + overflow, + tuple_argument, + true> +{ +private: + using Self = AggregateFunctionSumMapFiltered; + using Base = AggregateFunctionMapBase; + + using ContainerT = std::unordered_set; + + ContainerT keys_to_keep; + +public: + AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type_, + const DataTypes & values_types_, const DataTypes & argument_types_, + const Array & params_) + : Base{keys_type_, values_types_, argument_types_} + { + if (params_.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Aggregate function '{}' requires exactly one parameter " + "of Array type", getNameImpl()); + + Array keys_to_keep_values; + if (!params_.front().tryGet(keys_to_keep_values)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Aggregate function {} requires an Array as a parameter", + getNameImpl()); + + this->parameters = params_; + + keys_to_keep.reserve(keys_to_keep_values.size()); + + for (const Field & f : keys_to_keep_values) + keys_to_keep.emplace(f.safeGet()); + } + + static String getNameImpl() + { + if constexpr (overflow) + { + return "sumMapFilteredWithOverflow"; + } + else + { + return "sumMapFiltered"; + } + } + + bool keepKey(const T & key) const { return keys_to_keep.count(key); } +}; + + +/** Implements `Max` operation. + * Returns true if changed + */ +class FieldVisitorMax : public StaticVisitor +{ +private: + const Field & rhs; + + template + bool compareImpl(FieldType & x) const + { + auto val = rhs.get(); + if (val > x) + { + x = val; + return true; + } + + return false; + } + +public: + explicit FieldVisitorMax(const Field & rhs_) : rhs(rhs_) {} + + bool operator() (Null &) const + { + /// Do not update current value, skip nulls + return false; + } + + bool operator() (AggregateFunctionStateData &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot compare AggregateFunctionStates"); } + + bool operator() (Array & x) const { return compareImpl(x); } + bool operator() (Tuple & x) const { return compareImpl(x); } + template + bool operator() (DecimalField & x) const { return compareImpl>(x); } + template + bool operator() (T & x) const { return compareImpl(x); } +}; + +/** Implements `Min` operation. + * Returns true if changed + */ +class FieldVisitorMin : public StaticVisitor +{ +private: + const Field & rhs; + + template + bool compareImpl(FieldType & x) const + { + auto val = rhs.get(); + if (val < x) + { + x = val; + return true; + } + + return false; + } + +public: + explicit FieldVisitorMin(const Field & rhs_) : rhs(rhs_) {} + + + bool operator() (Null &) const + { + /// Do not update current value, skip nulls + return false; + } + + bool operator() (AggregateFunctionStateData &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot sum AggregateFunctionStates"); } + + bool operator() (Array & x) const { return compareImpl(x); } + bool operator() (Tuple & x) const { return compareImpl(x); } + template + bool operator() (DecimalField & x) const { return compareImpl>(x); } + template + bool operator() (T & x) const { return compareImpl(x); } +}; + + +template +class AggregateFunctionMinMap final : + public AggregateFunctionMapBase, FieldVisitorMin, true, tuple_argument, false> +{ +private: + using Self = AggregateFunctionMinMap; + using Base = AggregateFunctionMapBase; + +public: + AggregateFunctionMinMap(const DataTypePtr & keys_type_, + DataTypes & values_types_, const DataTypes & argument_types_, + const Array & params_) + : Base{keys_type_, values_types_, argument_types_} + { + // The constructor accepts parameters to have a uniform interface with + // sumMapFiltered, but this function doesn't have any parameters. + assertNoParameters(getNameImpl(), params_); + } + + static String getNameImpl() { return "minMap"; } + + bool keepKey(const T &) const { return true; } +}; + +template +class AggregateFunctionMaxMap final : + public AggregateFunctionMapBase, FieldVisitorMax, true, tuple_argument, false> +{ +private: + using Self = AggregateFunctionMaxMap; + using Base = AggregateFunctionMapBase; + +public: + AggregateFunctionMaxMap(const DataTypePtr & keys_type_, + DataTypes & values_types_, const DataTypes & argument_types_, + const Array & params_) + : Base{keys_type_, values_types_, argument_types_} + { + // The constructor accepts parameters to have a uniform interface with + // sumMapFiltered, but this function doesn't have any parameters. + assertNoParameters(getNameImpl(), params_); + } + + static String getNameImpl() { return "maxMap"; } + + bool keepKey(const T &) const { return true; } +}; + + auto parseArguments(const std::string & name, const DataTypes & arguments) { DataTypes args; diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h deleted file mode 100644 index b30f5ff5220..00000000000 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ /dev/null @@ -1,656 +0,0 @@ -#pragma once - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -struct Settings; - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int LOGICAL_ERROR; -} - -template -struct AggregateFunctionMapData -{ - // Map needs to be ordered to maintain function properties - std::map merged_maps; -}; - -/** Aggregate function, that takes at least two arguments: keys and values, and as a result, builds a tuple of at least 2 arrays - - * ordered keys and variable number of argument values aggregated by corresponding keys. - * - * sumMap function is the most useful when using SummingMergeTree to sum Nested columns, which name ends in "Map". - * - * Example: sumMap(k, v...) of: - * k v - * [1,2,3] [10,10,10] - * [3,4,5] [10,10,10] - * [4,5,6] [10,10,10] - * [6,7,8] [10,10,10] - * [7,5,3] [5,15,25] - * [8,9,10] [20,20,20] - * will return: - * ([1,2,3,4,5,6,7,8,9,10],[10,10,45,20,35,20,15,30,20,20]) - * - * minMap and maxMap share the same idea, but calculate min and max correspondingly. - * - * NOTE: The implementation of these functions are "amateur grade" - not efficient and low quality. - */ - -template -class AggregateFunctionMapBase : public IAggregateFunctionDataHelper< - AggregateFunctionMapData>, Derived> -{ -private: - static constexpr auto STATE_VERSION_1_MIN_REVISION = 54452; - - DataTypePtr keys_type; - SerializationPtr keys_serialization; - DataTypes values_types; - Serializations values_serializations; - Serializations promoted_values_serializations; - -public: - using Base = IAggregateFunctionDataHelper< - AggregateFunctionMapData>, Derived>; - - AggregateFunctionMapBase(const DataTypePtr & keys_type_, - const DataTypes & values_types_, const DataTypes & argument_types_) - : Base(argument_types_, {} /* parameters */, createResultType(keys_type_, values_types_, getName())) - , keys_type(keys_type_) - , keys_serialization(keys_type->getDefaultSerialization()) - , values_types(values_types_) - { - values_serializations.reserve(values_types.size()); - promoted_values_serializations.reserve(values_types.size()); - for (const auto & type : values_types) - { - values_serializations.emplace_back(type->getDefaultSerialization()); - if (type->canBePromoted()) - { - if (type->isNullable()) - promoted_values_serializations.emplace_back( - makeNullable(removeNullable(type)->promoteNumericType())->getDefaultSerialization()); - else - promoted_values_serializations.emplace_back(type->promoteNumericType()->getDefaultSerialization()); - } - else - { - promoted_values_serializations.emplace_back(type->getDefaultSerialization()); - } - } - } - - bool isVersioned() const override { return true; } - - size_t getDefaultVersion() const override { return 1; } - - size_t getVersionFromRevision(size_t revision) const override - { - if (revision >= STATE_VERSION_1_MIN_REVISION) - return 1; - else - return 0; - } - - static DataTypePtr createResultType( - const DataTypePtr & keys_type_, - const DataTypes & values_types_, - const String & name_) - { - DataTypes types; - types.emplace_back(std::make_shared(keys_type_)); - - for (const auto & value_type : values_types_) - { - if constexpr (std::is_same_v) - { - if (!value_type->isSummable()) - throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Values for {} cannot be summed, passed type {}", - name_, value_type->getName()}; - } - - DataTypePtr result_type; - - if constexpr (overflow) - { - if (value_type->onlyNull()) - throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Cannot calculate {} of type {}", - name_, value_type->getName()}; - - // Overflow, meaning that the returned type is the same as - // the input type. Nulls are skipped. - result_type = removeNullable(value_type); - } - else - { - auto value_type_without_nullable = removeNullable(value_type); - - // No overflow, meaning we promote the types if necessary. - if (!value_type_without_nullable->canBePromoted()) - throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Values for {} are expected to be Numeric, Float or Decimal, passed type {}", - name_, value_type->getName()}; - - WhichDataType value_type_to_check(value_type_without_nullable); - - /// Do not promote decimal because of implementation issues of this function design - /// Currently we cannot get result column type in case of decimal we cannot get decimal scale - /// in method void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override - /// If we decide to make this function more efficient we should promote decimal type during summ - if (value_type_to_check.isDecimal()) - result_type = value_type_without_nullable; - else - result_type = value_type_without_nullable->promoteNumericType(); - } - - types.emplace_back(std::make_shared(result_type)); - } - - return std::make_shared(types); - } - - bool allocatesMemoryInArena() const override { return false; } - - static const auto & getArgumentColumns(const IColumn**& columns) - { - if constexpr (tuple_argument) - { - return assert_cast(columns[0])->getColumns(); - } - else - { - return columns; - } - } - - void add(AggregateDataPtr __restrict place, const IColumn ** columns_, const size_t row_num, Arena *) const override - { - const auto & columns = getArgumentColumns(columns_); - - // Column 0 contains array of keys of known type - const ColumnArray & array_column0 = assert_cast(*columns[0]); - const IColumn::Offsets & offsets0 = array_column0.getOffsets(); - const IColumn & key_column = array_column0.getData(); - const size_t keys_vec_offset = offsets0[row_num - 1]; - const size_t keys_vec_size = (offsets0[row_num] - keys_vec_offset); - - // Columns 1..n contain arrays of numeric values to sum - auto & merged_maps = this->data(place).merged_maps; - for (size_t col = 0, size = values_types.size(); col < size; ++col) - { - const auto & array_column = assert_cast(*columns[col + 1]); - const IColumn & value_column = array_column.getData(); - const IColumn::Offsets & offsets = array_column.getOffsets(); - const size_t values_vec_offset = offsets[row_num - 1]; - const size_t values_vec_size = (offsets[row_num] - values_vec_offset); - - // Expect key and value arrays to be of same length - if (keys_vec_size != values_vec_size) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sizes of keys and values arrays do not match"); - - // Insert column values for all keys - for (size_t i = 0; i < keys_vec_size; ++i) - { - auto value = value_column[values_vec_offset + i]; - T key = static_cast(key_column[keys_vec_offset + i].get()); - - if (!keepKey(key)) - continue; - - decltype(merged_maps.begin()) it; - if constexpr (is_decimal) - { - // FIXME why is storing NearestFieldType not enough, and we - // have to check for decimals again here? - UInt32 scale = static_cast &>(key_column).getScale(); - it = merged_maps.find(DecimalField(key, scale)); - } - else - it = merged_maps.find(key); - - if (it != merged_maps.end()) - { - if (!value.isNull()) - { - if (it->second[col].isNull()) - it->second[col] = value; - else - applyVisitor(Visitor(value), it->second[col]); - } - } - else - { - // Create a value array for this key - Array new_values; - new_values.resize(size); - new_values[col] = value; - - if constexpr (is_decimal) - { - UInt32 scale = static_cast &>(key_column).getScale(); - merged_maps.emplace(DecimalField(key, scale), std::move(new_values)); - } - else - { - merged_maps.emplace(key, std::move(new_values)); - } - } - } - } - } - - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override - { - auto & merged_maps = this->data(place).merged_maps; - const auto & rhs_maps = this->data(rhs).merged_maps; - - for (const auto & elem : rhs_maps) - { - const auto & it = merged_maps.find(elem.first); - if (it != merged_maps.end()) - { - for (size_t col = 0; col < values_types.size(); ++col) - if (!elem.second[col].isNull()) - applyVisitor(Visitor(elem.second[col]), it->second[col]); - } - else - merged_maps[elem.first] = elem.second; - } - } - - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override - { - if (!version) - version = getDefaultVersion(); - - const auto & merged_maps = this->data(place).merged_maps; - size_t size = merged_maps.size(); - writeVarUInt(size, buf); - - std::function serialize; - switch (*version) - { - case 0: - { - serialize = [&](size_t col_idx, const Array & values){ values_serializations[col_idx]->serializeBinary(values[col_idx], buf, {}); }; - break; - } - case 1: - { - serialize = [&](size_t col_idx, const Array & values){ promoted_values_serializations[col_idx]->serializeBinary(values[col_idx], buf, {}); }; - break; - } - } - - for (const auto & elem : merged_maps) - { - keys_serialization->serializeBinary(elem.first, buf, {}); - for (size_t col = 0; col < values_types.size(); ++col) - serialize(col, elem.second); - } - } - - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena *) const override - { - if (!version) - version = getDefaultVersion(); - - auto & merged_maps = this->data(place).merged_maps; - size_t size = 0; - readVarUInt(size, buf); - - std::function deserialize; - switch (*version) - { - case 0: - { - deserialize = [&](size_t col_idx, Array & values){ values_serializations[col_idx]->deserializeBinary(values[col_idx], buf, {}); }; - break; - } - case 1: - { - deserialize = [&](size_t col_idx, Array & values){ promoted_values_serializations[col_idx]->deserializeBinary(values[col_idx], buf, {}); }; - break; - } - } - - for (size_t i = 0; i < size; ++i) - { - Field key; - keys_serialization->deserializeBinary(key, buf, {}); - - Array values; - values.resize(values_types.size()); - - for (size_t col = 0; col < values_types.size(); ++col) - deserialize(col, values); - - if constexpr (is_decimal) - merged_maps[key.get>()] = values; - else - merged_maps[key.get()] = values; - } - } - - void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override - { - size_t num_columns = values_types.size(); - - // Final step does compaction of keys that have zero values, this mutates the state - auto & merged_maps = this->data(place).merged_maps; - - // Remove keys which are zeros or empty. This should be enabled only for sumMap. - if constexpr (compact) - { - for (auto it = merged_maps.cbegin(); it != merged_maps.cend();) - { - // Key is not compacted if it has at least one non-zero value - bool erase = true; - for (size_t col = 0; col < num_columns; ++col) - { - if (!it->second[col].isNull() && it->second[col] != values_types[col]->getDefault()) - { - erase = false; - break; - } - } - - if (erase) - it = merged_maps.erase(it); - else - ++it; - } - } - - size_t size = merged_maps.size(); - - auto & to_tuple = assert_cast(to); - auto & to_keys_arr = assert_cast(to_tuple.getColumn(0)); - auto & to_keys_col = to_keys_arr.getData(); - - // Advance column offsets - auto & to_keys_offsets = to_keys_arr.getOffsets(); - to_keys_offsets.push_back(to_keys_offsets.back() + size); - to_keys_col.reserve(size); - - for (size_t col = 0; col < num_columns; ++col) - { - auto & to_values_arr = assert_cast(to_tuple.getColumn(col + 1)); - auto & to_values_offsets = to_values_arr.getOffsets(); - to_values_offsets.push_back(to_values_offsets.back() + size); - to_values_arr.getData().reserve(size); - } - - // Write arrays of keys and values - for (const auto & elem : merged_maps) - { - // Write array of keys into column - to_keys_col.insert(elem.first); - - // Write 0..n arrays of values - for (size_t col = 0; col < num_columns; ++col) - { - auto & to_values_col = assert_cast(to_tuple.getColumn(col + 1)).getData(); - if (elem.second[col].isNull()) - to_values_col.insertDefault(); - else - to_values_col.insert(elem.second[col]); - } - } - } - - bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } - String getName() const override { return Derived::getNameImpl(); } -}; - -template -class AggregateFunctionSumMap final : - public AggregateFunctionMapBase, FieldVisitorSum, overflow, tuple_argument, true> -{ -private: - using Self = AggregateFunctionSumMap; - using Base = AggregateFunctionMapBase; - -public: - AggregateFunctionSumMap(const DataTypePtr & keys_type_, - DataTypes & values_types_, const DataTypes & argument_types_, - const Array & params_) - : Base{keys_type_, values_types_, argument_types_} - { - // The constructor accepts parameters to have a uniform interface with - // sumMapFiltered, but this function doesn't have any parameters. - assertNoParameters(getNameImpl(), params_); - } - - static String getNameImpl() - { - if constexpr (overflow) - { - return "sumMapWithOverflow"; - } - else - { - return "sumMap"; - } - } - - bool keepKey(const T &) const { return true; } -}; - - -template -class AggregateFunctionSumMapFiltered final : - public AggregateFunctionMapBase, - FieldVisitorSum, - overflow, - tuple_argument, - true> -{ -private: - using Self = AggregateFunctionSumMapFiltered; - using Base = AggregateFunctionMapBase; - - using ContainerT = std::unordered_set; - - ContainerT keys_to_keep; - -public: - AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type_, - const DataTypes & values_types_, const DataTypes & argument_types_, - const Array & params_) - : Base{keys_type_, values_types_, argument_types_} - { - if (params_.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Aggregate function '{}' requires exactly one parameter " - "of Array type", getNameImpl()); - - Array keys_to_keep_values; - if (!params_.front().tryGet(keys_to_keep_values)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Aggregate function {} requires an Array as a parameter", - getNameImpl()); - - this->parameters = params_; - - keys_to_keep.reserve(keys_to_keep_values.size()); - - for (const Field & f : keys_to_keep_values) - keys_to_keep.emplace(f.safeGet()); - } - - static String getNameImpl() - { - if constexpr (overflow) - { - return "sumMapFilteredWithOverflow"; - } - else - { - return "sumMapFiltered"; - } - } - - bool keepKey(const T & key) const { return keys_to_keep.count(key); } -}; - - -/** Implements `Max` operation. - * Returns true if changed - */ -class FieldVisitorMax : public StaticVisitor -{ -private: - const Field & rhs; - - template - bool compareImpl(FieldType & x) const - { - auto val = rhs.get(); - if (val > x) - { - x = val; - return true; - } - - return false; - } - -public: - explicit FieldVisitorMax(const Field & rhs_) : rhs(rhs_) {} - - bool operator() (Null &) const - { - /// Do not update current value, skip nulls - return false; - } - - bool operator() (AggregateFunctionStateData &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot compare AggregateFunctionStates"); } - - bool operator() (Array & x) const { return compareImpl(x); } - bool operator() (Tuple & x) const { return compareImpl(x); } - template - bool operator() (DecimalField & x) const { return compareImpl>(x); } - template - bool operator() (T & x) const { return compareImpl(x); } -}; - -/** Implements `Min` operation. - * Returns true if changed - */ -class FieldVisitorMin : public StaticVisitor -{ -private: - const Field & rhs; - - template - bool compareImpl(FieldType & x) const - { - auto val = rhs.get(); - if (val < x) - { - x = val; - return true; - } - - return false; - } - -public: - explicit FieldVisitorMin(const Field & rhs_) : rhs(rhs_) {} - - - bool operator() (Null &) const - { - /// Do not update current value, skip nulls - return false; - } - - bool operator() (AggregateFunctionStateData &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot sum AggregateFunctionStates"); } - - bool operator() (Array & x) const { return compareImpl(x); } - bool operator() (Tuple & x) const { return compareImpl(x); } - template - bool operator() (DecimalField & x) const { return compareImpl>(x); } - template - bool operator() (T & x) const { return compareImpl(x); } -}; - - -template -class AggregateFunctionMinMap final : - public AggregateFunctionMapBase, FieldVisitorMin, true, tuple_argument, false> -{ -private: - using Self = AggregateFunctionMinMap; - using Base = AggregateFunctionMapBase; - -public: - AggregateFunctionMinMap(const DataTypePtr & keys_type_, - DataTypes & values_types_, const DataTypes & argument_types_, - const Array & params_) - : Base{keys_type_, values_types_, argument_types_} - { - // The constructor accepts parameters to have a uniform interface with - // sumMapFiltered, but this function doesn't have any parameters. - assertNoParameters(getNameImpl(), params_); - } - - static String getNameImpl() { return "minMap"; } - - bool keepKey(const T &) const { return true; } -}; - -template -class AggregateFunctionMaxMap final : - public AggregateFunctionMapBase, FieldVisitorMax, true, tuple_argument, false> -{ -private: - using Self = AggregateFunctionMaxMap; - using Base = AggregateFunctionMapBase; - -public: - AggregateFunctionMaxMap(const DataTypePtr & keys_type_, - DataTypes & values_types_, const DataTypes & argument_types_, - const Array & params_) - : Base{keys_type_, values_types_, argument_types_} - { - // The constructor accepts parameters to have a uniform interface with - // sumMapFiltered, but this function doesn't have any parameters. - assertNoParameters(getNameImpl(), params_); - } - - static String getNameImpl() { return "maxMap"; } - - bool keepKey(const T &) const { return true; } -}; - -} diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index eba1733c683..983825a0e68 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -234,7 +234,7 @@ private: } template - ColumnPtr executeRightType( + NO_INLINE ColumnPtr executeRightType( [[maybe_unused]] const ColumnUInt8 * cond_col, [[maybe_unused]] const ColumnsWithTypeAndName & arguments, [[maybe_unused]] const ColVecT0 * col_left) const @@ -266,7 +266,7 @@ private: } template - ColumnPtr executeConstRightType( + NO_INLINE ColumnPtr executeConstRightType( [[maybe_unused]] const ColumnUInt8 * cond_col, [[maybe_unused]] const ColumnsWithTypeAndName & arguments, [[maybe_unused]] const ColumnConst * col_left) const @@ -298,7 +298,7 @@ private: } template - ColumnPtr executeRightTypeArray( + NO_INLINE ColumnPtr executeRightTypeArray( [[maybe_unused]] const ColumnUInt8 * cond_col, [[maybe_unused]] const ColumnsWithTypeAndName & arguments, [[maybe_unused]] const DataTypePtr result_type, @@ -355,7 +355,7 @@ private: } template - ColumnPtr executeConstRightTypeArray( + NO_INLINE ColumnPtr executeConstRightTypeArray( [[maybe_unused]] const ColumnUInt8 * cond_col, [[maybe_unused]] const ColumnsWithTypeAndName & arguments, [[maybe_unused]] const DataTypePtr & result_type, @@ -413,7 +413,7 @@ private: } template - ColumnPtr executeTyped( + NO_INLINE ColumnPtr executeTyped( const ColumnUInt8 * cond_col, const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { using ColVecT0 = ColumnVectorOrDecimal; @@ -1086,7 +1086,7 @@ public: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " "Must be ColumnUInt8 or ColumnConstUInt8.", arg_cond.column->getName(), getName()); - auto call = [&](const auto & types) -> bool + auto call = [&](const auto & types) NO_INLINE -> bool { using Types = std::decay_t; using T0 = typename Types::LeftType; From 99d90a1430437319ceab57e56c6bbd7cf594ede2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Nov 2023 01:58:20 +0100 Subject: [PATCH 0224/1190] Attempt to remove garbage --- .../AggregateFunctionSumMap.cpp | 50 ++++++++----------- 1 file changed, 21 insertions(+), 29 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index ecc8a978388..f89af3a0dae 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -20,8 +20,19 @@ #include +namespace std +{ + template + struct hash> + { + size_t operator()(const DB::DecimalField & x) const { return hash()(x.getValue()); } + }; +} + + namespace DB { + struct Settings; namespace ErrorCodes @@ -65,7 +76,7 @@ struct AggregateFunctionMapData template class AggregateFunctionMapBase : public IAggregateFunctionDataHelper< - AggregateFunctionMapData>, Derived> + AggregateFunctionMapData, Derived> { private: static constexpr auto STATE_VERSION_1_MIN_REVISION = 54452; @@ -78,7 +89,7 @@ private: public: using Base = IAggregateFunctionDataHelper< - AggregateFunctionMapData>, Derived>; + AggregateFunctionMapData, Derived>; AggregateFunctionMapBase(const DataTypePtr & keys_type_, const DataTypes & values_types_, const DataTypes & argument_types_) @@ -227,15 +238,7 @@ public: continue; decltype(merged_maps.begin()) it; - if constexpr (is_decimal) - { - // FIXME why is storing NearestFieldType not enough, and we - // have to check for decimals again here? - UInt32 scale = static_cast &>(key_column).getScale(); - it = merged_maps.find(DecimalField(key, scale)); - } - else - it = merged_maps.find(key); + it = merged_maps.find(key); if (it != merged_maps.end()) { @@ -254,15 +257,7 @@ public: new_values.resize(size); new_values[col] = value; - if constexpr (is_decimal) - { - UInt32 scale = static_cast &>(key_column).getScale(); - merged_maps.emplace(DecimalField(key, scale), std::move(new_values)); - } - else - { - merged_maps.emplace(key, std::move(new_values)); - } + merged_maps.emplace(key, std::move(new_values)); } } } @@ -354,10 +349,7 @@ public: for (size_t col = 0; col < values_types.size(); ++col) deserialize(col, values); - if constexpr (is_decimal) - merged_maps[key.get>()] = values; - else - merged_maps[key.get()] = values; + merged_maps[key.get()] = values; } } @@ -711,7 +703,7 @@ auto parseArguments(const std::string & name, const DataTypes & arguments) // The template parameter MappedFunction is an aggregate // function template that allows to choose the aggregate function variant that // accepts either normal arguments or tuple argument. -template