From e51bee58ecf8f684abfb0f1f9ac1806fb47b5efd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Apr 2022 22:32:45 +0200 Subject: [PATCH 0001/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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/1097] 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 f5c9d278ad7be8a90d92d66546134575ad54c7e7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 9 Aug 2023 22:57:49 +0200 Subject: [PATCH 0024/1097] use statistic to order prewhere conditions better --- src/Access/Common/AccessType.h | 5 + src/AggregateFunctions/QuantileTDigest.h | 11 + src/CMakeLists.txt | 1 + src/Databases/DatabasesCommon.cpp | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 15 + src/Interpreters/InterpreterCreateQuery.cpp | 17 ++ src/Interpreters/InterpreterCreateQuery.h | 2 + src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Interpreters/MutationsInterpreter.cpp | 36 ++- src/Interpreters/MutationsInterpreter.h | 5 +- src/Parsers/ASTAlterQuery.cpp | 38 +++ src/Parsers/ASTAlterQuery.h | 9 + src/Parsers/ASTCreateQuery.cpp | 12 + src/Parsers/ASTCreateQuery.h | 3 +- src/Parsers/ASTStatisticDeclaration.cpp | 35 +++ src/Parsers/ASTStatisticDeclaration.h | 26 ++ src/Parsers/ParserAlterQuery.cpp | 70 +++++ src/Parsers/ParserCreateQuery.cpp | 46 +++ src/Parsers/ParserCreateQuery.h | 12 + .../Optimizations/optimizePrewhere.cpp | 1 + src/Storages/AlterCommands.cpp | 100 +++++++ src/Storages/AlterCommands.h | 6 + src/Storages/ColumnDependency.h | 5 +- src/Storages/IStorage.h | 3 + src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 + src/Storages/MergeTree/MergeTask.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartCompact.h | 1 + .../MergeTree/MergeTreeDataPartInMemory.cpp | 5 +- .../MergeTree/MergeTreeDataPartInMemory.h | 1 + .../MergeTree/MergeTreeDataPartWide.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.h | 1 + .../MergeTreeDataPartWriterCompact.cpp | 6 +- .../MergeTreeDataPartWriterCompact.h | 1 + .../MergeTreeDataPartWriterOnDisk.cpp | 126 +++++++-- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 30 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 14 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 1 + .../MergeTree/MergeTreeDataWriter.cpp | 2 + .../MergeTree/MergeTreeWhereOptimizer.cpp | 131 ++++----- .../MergeTree/MergeTreeWhereOptimizer.h | 10 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 3 +- .../MergeTree/MergedBlockOutputStream.h | 2 + .../MergedColumnOnlyOutputStream.cpp | 2 + .../MergeTree/MergedColumnOnlyOutputStream.h | 2 + src/Storages/MergeTree/MutateTask.cpp | 64 ++++- .../MergeTree/registerStorageMergeTree.cpp | 5 + src/Storages/MutationCommands.cpp | 11 + src/Storages/MutationCommands.h | 5 +- src/Storages/Statistic/Statistic.cpp | 155 +++++++++++ src/Storages/Statistic/Statistic.h | 262 ++++++++++++++++++ src/Storages/StatisticsDescription.cpp | 120 ++++++++ src/Storages/StatisticsDescription.h | 46 +++ src/Storages/StorageInMemoryMetadata.cpp | 13 + src/Storages/StorageInMemoryMetadata.h | 8 + 58 files changed, 1391 insertions(+), 118 deletions(-) create mode 100644 src/Parsers/ASTStatisticDeclaration.cpp create mode 100644 src/Parsers/ASTStatisticDeclaration.h create mode 100644 src/Storages/Statistic/Statistic.cpp create mode 100644 src/Storages/Statistic/Statistic.h create mode 100644 src/Storages/StatisticsDescription.cpp create mode 100644 src/Storages/StatisticsDescription.h diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index f65a77c1d6a..dac3f813dd6 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -51,6 +51,11 @@ enum class AccessType M(ALTER_CLEAR_INDEX, "CLEAR INDEX", TABLE, ALTER_INDEX) \ M(ALTER_INDEX, "INDEX", GROUP, ALTER_TABLE) /* allows to execute ALTER ORDER BY or ALTER {ADD|DROP...} INDEX */\ \ + M(ALTER_ADD_STATISTIC, "ALTER ADD STATISTIC", TABLE, ALTER_STATISTIC) \ + M(ALTER_DROP_STATISTIC, "ALTER DROP STATISTIC", TABLE, ALTER_STATISTIC) \ + M(ALTER_MATERIALIZE_STATISTIC, "ALTER MATERIALIZE STATISTIC", TABLE, ALTER_STATISTIC) \ + M(ALTER_STATISTIC, "STATISTIC", GROUP, ALTER_TABLE) /* allows to execute ALTER ORDER BY or ALTER {ADD|DROP...} INDEX */\ + \ M(ALTER_ADD_PROJECTION, "ADD PROJECTION", TABLE, ALTER_PROJECTION) \ M(ALTER_DROP_PROJECTION, "DROP PROJECTION", TABLE, ALTER_PROJECTION) \ M(ALTER_MATERIALIZE_PROJECTION, "MATERIALIZE PROJECTION", TABLE, ALTER_PROJECTION) \ diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index 915f6763e52..8706f77c12d 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -43,6 +43,7 @@ namespace ErrorCodes template class QuantileTDigest { + friend class TDigestStatistic; using Value = Float32; using Count = Float32; using BetterFloat = Float64; // For intermediate results and sum(Count). Must have better precision, than Count @@ -334,6 +335,16 @@ public: compress(); // Allows reading/writing TDigests with different epsilon/max_centroids params } + Float64 getCountLessThan(Float64 value) const + { + + ///Count sum = 0; + ///Value prev_mean = centroids.front().mean; + ///Count prev_count = centroids.front().count; + + return value; + } + /** Calculates the quantile q [0, 1] based on the digest. * For an empty digest returns NaN. */ diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index ca428fbff3a..df49992595d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -247,6 +247,7 @@ add_object_library(clickhouse_storages Storages) add_object_library(clickhouse_storages_mysql Storages/MySQL) add_object_library(clickhouse_storages_distributed Storages/Distributed) add_object_library(clickhouse_storages_mergetree Storages/MergeTree) +add_object_library(clickhouse_storages_statistic Storages/Statistic) add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_storages_windowview Storages/WindowView) add_object_library(clickhouse_client Client) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index bb98e2bd3bb..de27c4fd8e7 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -46,11 +46,13 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo { ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); + ASTPtr new_statistics = InterpreterCreateQuery::formatStatistics(metadata.statistics); ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints); ASTPtr new_projections = InterpreterCreateQuery::formatProjections(metadata.projections); ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->stats, new_statistics); ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints); ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->projections, new_projections); } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index e82415f1aca..7f30ee9337d 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -287,6 +287,21 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_SAMPLE_BY, database, table); break; } + case ASTAlterCommand::ADD_STATISTIC: + { + required_access.emplace_back(AccessType::ALTER_ADD_STATISTIC, database, table); + break; + } + case ASTAlterCommand::DROP_STATISTIC: + { + required_access.emplace_back(AccessType::ALTER_DROP_STATISTIC, database, table); + break; + } + case ASTAlterCommand::MATERIALIZE_STATISTIC: + { + required_access.emplace_back(AccessType::ALTER_MATERIALIZE_STATISTIC, database, table); + break; + } case ASTAlterCommand::ADD_INDEX: { required_access.emplace_back(AccessType::ALTER_ADD_INDEX, database, table); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d0bb3dd389f..5c0b58eddc8 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -455,6 +455,16 @@ ASTPtr InterpreterCreateQuery::formatIndices(const IndicesDescription & indices) return res; } +ASTPtr InterpreterCreateQuery::formatStatistics(const StatisticsDescriptions & statistics) +{ + auto res = std::make_shared(); + + for (const auto & statistic : statistics) + res->children.push_back(statistic.definition_ast->clone()); + + return res; +} + ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription & constraints) { auto res = std::make_shared(); @@ -706,6 +716,11 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.indices.push_back(index_desc); } + if (create.columns_list->stats) + for (const auto & statistic : create.columns_list->stats->children) + properties.stats.push_back( + StatisticDescription::getStatisticFromAST(statistic->clone(), properties.columns, getContext())); + if (create.columns_list->projections) for (const auto & projection_ast : create.columns_list->projections->children) { @@ -791,11 +806,13 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti ASTPtr new_columns = formatColumns(properties.columns); ASTPtr new_indices = formatIndices(properties.indices); + ASTPtr new_statistics = formatStatistics(properties.stats); ASTPtr new_constraints = formatConstraints(properties.constraints); ASTPtr new_projections = formatProjections(properties.projections); create.columns_list->setOrReplace(create.columns_list->columns, new_columns); create.columns_list->setOrReplace(create.columns_list->indices, new_indices); + create.columns_list->setOrReplace(create.columns_list->stats, new_statistics); create.columns_list->setOrReplace(create.columns_list->constraints, new_constraints); create.columns_list->setOrReplace(create.columns_list->projections, new_projections); diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index a5fa6576091..88eba3e0d79 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -38,6 +38,7 @@ public: static ASTPtr formatColumns(const NamesAndTypesList & columns, const NamesAndAliases & alias_columns); static ASTPtr formatColumns(const ColumnsDescription & columns); static ASTPtr formatIndices(const IndicesDescription & indices); + static ASTPtr formatStatistics(const StatisticsDescriptions & statistics); static ASTPtr formatConstraints(const ConstraintsDescription & constraints); static ASTPtr formatProjections(const ProjectionsDescription & projections); @@ -80,6 +81,7 @@ private: { ColumnsDescription columns; IndicesDescription indices; + StatisticsDescriptions stats; ConstraintsDescription constraints; ProjectionsDescription projections; }; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6ea15312ec4..be8443d56a7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -687,6 +687,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( MergeTreeWhereOptimizer where_optimizer{ std::move(column_compressed_sizes), metadata_snapshot, + storage->getConditionEstimatorByPredicate(query_info, context), queried_columns, supported_prewhere_columns, log}; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 25c52ad8925..22105f063fa 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -479,6 +479,7 @@ void MutationsInterpreter::prepare(bool dry_run) /// TODO Should we get columns, indices and projections from the part itself? Table metadata may be different const ColumnsDescription & columns_desc = metadata_snapshot->getColumns(); const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices(); + const StatisticsDescriptions & statistics_desc = metadata_snapshot->getStatistics(); const ProjectionsDescription & projections_desc = metadata_snapshot->getProjections(); auto storage_snapshot = std::make_shared(*source.getStorage(), metadata_snapshot); @@ -682,7 +683,7 @@ void MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::MATERIALIZE_INDEX) { - mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); + mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); auto it = std::find_if( std::cbegin(indices_desc), std::end(indices_desc), [&](const IndexDescription & index) @@ -703,9 +704,25 @@ void MutationsInterpreter::prepare(bool dry_run) materialized_indices.emplace(command.index_name); } } + else if (command.type == MutationCommand::MATERIALIZE_STATISTIC) + { + mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); + auto it = std::find_if( + std::cbegin(statistics_desc), std::end(statistics_desc), + [&](const StatisticDescription & statistic) + { + return statistic.name == command.statistic_name; + }); + if (it == std::cend(statistics_desc)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown statistic: {}", command.statistic_name); + + for (const auto & column : it->column_names) + dependencies.emplace(column, ColumnDependency::STATISTIC); + materialized_statistics.emplace(command.statistic_name); + } else if (command.type == MutationCommand::MATERIALIZE_PROJECTION) { - mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); + mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); const auto & projection = projections_desc.get(command.projection_name); if (!source.hasIndexOrProjection(projection.getDirectoryName())) { @@ -716,12 +733,17 @@ void MutationsInterpreter::prepare(bool dry_run) } else if (command.type == MutationCommand::DROP_INDEX) { - mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); + mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); materialized_indices.erase(command.index_name); } + else if (command.type == MutationCommand::DROP_STATISTIC) + { + mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); + materialized_statistics.erase(command.statistic_name); + } else if (command.type == MutationCommand::DROP_PROJECTION) { - mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); + mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); materialized_projections.erase(command.projection_name); } else if (command.type == MutationCommand::MATERIALIZE_TTL) @@ -770,7 +792,9 @@ void MutationsInterpreter::prepare(bool dry_run) auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); for (const auto & dependency : new_dependencies) { - if (dependency.kind == ColumnDependency::SKIP_INDEX || dependency.kind == ColumnDependency::PROJECTION) + if (dependency.kind == ColumnDependency::SKIP_INDEX + || dependency.kind == ColumnDependency::PROJECTION + || dependency.kind == ColumnDependency::STATISTIC) dependencies.insert(dependency); } } @@ -1288,7 +1312,7 @@ QueryPipelineBuilder MutationsInterpreter::execute() Block MutationsInterpreter::getUpdatedHeader() const { // If it's an index/projection materialization, we don't write any data columns, thus empty header is used - return mutation_kind.mutation_kind == MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : *updated_header; + return mutation_kind.mutation_kind == MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION ? Block{} : *updated_header; } const ColumnDependencies & MutationsInterpreter::getColumnDependencies() const diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index d783b503531..4d95f56ee71 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -91,6 +91,8 @@ public: NameSet grabMaterializedIndices() { return std::move(materialized_indices); } + NameSet grabMaterializedStatistics() { return std::move(materialized_statistics); } + NameSet grabMaterializedProjections() { return std::move(materialized_projections); } struct MutationKind @@ -98,7 +100,7 @@ public: enum MutationKindEnum { MUTATE_UNKNOWN, - MUTATE_INDEX_PROJECTION, + MUTATE_INDEX_STATISTIC_PROJECTION, MUTATE_OTHER, } mutation_kind = MUTATE_UNKNOWN; @@ -212,6 +214,7 @@ private: NameSet materialized_indices; NameSet materialized_projections; + NameSet materialized_statistics; MutationKind mutation_kind; /// Do we meet any index or projection mutation. diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 61e5903fad5..b04c0efa85c 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -123,6 +123,9 @@ const char * ASTAlterCommand::typeToString(ASTAlterCommand::Type type) case LIVE_VIEW_REFRESH: return "LIVE_VIEW_REFRESH"; case MODIFY_DATABASE_SETTING: return "MODIFY_DATABASE_SETTING"; case MODIFY_COMMENT: return "MODIFY_COMMENT"; + case ADD_STATISTIC: return "ADD_STATISTIC"; + case DROP_STATISTIC: return "DROP_STATISTIC"; + case MATERIALIZE_STATISTIC: return "MATERIALIZE_STATISTIC"; } UNREACHABLE(); } @@ -248,6 +251,41 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & partition->formatImpl(settings, state, frame); } } + else if (type == ASTAlterCommand::ADD_STATISTIC) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD STATISTIC " << (if_not_exists ? "IF NOT EXISTS " : "") + << (settings.hilite ? hilite_none : ""); + statistic_decl->formatImpl(settings, state, frame); + + if (first) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : ""); + else if (statistic) /// AFTER + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : ""); + statistic->formatImpl(settings, state, frame); + } + } + else if (type == ASTAlterCommand::DROP_STATISTIC) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_statistic ? "CLEAR " : "DROP ") << "INDEX " + << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + statistic->formatImpl(settings, state, frame); + if (partition) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } + } + else if (type == ASTAlterCommand::MATERIALIZE_STATISTIC) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE STATISTIC " << (settings.hilite ? hilite_none : ""); + statistic->formatImpl(settings, state, frame); + if (partition) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } + } else if (type == ASTAlterCommand::ADD_CONSTRAINT) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD CONSTRAINT " << (if_not_exists ? "IF NOT EXISTS " : "") diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 1400113fa9c..251e8c233e2 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -54,6 +54,10 @@ public: DROP_PROJECTION, MATERIALIZE_PROJECTION, + ADD_STATISTIC, + DROP_STATISTIC, + MATERIALIZE_STATISTIC, + DROP_PARTITION, DROP_DETACHED_PARTITION, ATTACH_PARTITION, @@ -129,6 +133,9 @@ public: */ ASTPtr projection; + ASTPtr statistic_decl; + ASTPtr statistic; + /** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE queries. * The value or ID of the partition is stored here. */ @@ -167,6 +174,8 @@ public: bool clear_index = false; /// for CLEAR INDEX (do not drop index from metadata) + bool clear_statistic = false; /// for CLEAR STATISTIC (do not drop statistic from metadata) + bool clear_projection = false; /// for CLEAR PROJECTION (do not drop projection from metadata) bool if_not_exists = false; /// option for ADD_COLUMN diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 196681a8801..5c34841e6a7 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -132,6 +132,8 @@ ASTPtr ASTColumns::clone() const res->set(res->columns, columns->clone()); if (indices) res->set(res->indices, indices->clone()); + if (stats) + res->set(res->stats, stats->clone()); if (constraints) res->set(res->constraints, constraints->clone()); if (projections) @@ -166,6 +168,16 @@ void ASTColumns::formatImpl(const FormatSettings & s, FormatState & state, Forma list.children.push_back(elem); } } + if (stats) + { + for (const auto & stat : stats->children) + { + auto elem = std::make_shared(); + elem->prefix = "STATISTIC"; + elem->set(elem->elem, stat->clone()); + list.children.push_back(elem); + } + } if (constraints) { for (const auto & constraint : constraints->children) diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 230996f610e..0c1a139c2eb 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -53,6 +53,7 @@ class ASTColumns : public IAST public: ASTExpressionList * columns = nullptr; ASTExpressionList * indices = nullptr; + ASTExpressionList * stats = nullptr; ASTExpressionList * constraints = nullptr; ASTExpressionList * projections = nullptr; IAST * primary_key = nullptr; @@ -66,7 +67,7 @@ public: bool empty() const { return (!columns || columns->children.empty()) && (!indices || indices->children.empty()) && (!constraints || constraints->children.empty()) - && (!projections || projections->children.empty()); + && (!projections || projections->children.empty()) && (!stats || stats->children.empty()); } void forEachPointerToChild(std::function f) override diff --git a/src/Parsers/ASTStatisticDeclaration.cpp b/src/Parsers/ASTStatisticDeclaration.cpp new file mode 100644 index 00000000000..53b20b167b7 --- /dev/null +++ b/src/Parsers/ASTStatisticDeclaration.cpp @@ -0,0 +1,35 @@ +#include + +#include +#include +#include + + +namespace DB +{ + +ASTPtr ASTStatisticDeclaration::clone() const +{ + auto res = std::make_shared(); + + res->name = name; + + if (columns) + res->set(res->columns, columns->clone()); + if (type) + res->set(res->type, type->clone()); + return std::move(res); +} + + +void ASTStatisticDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const +{ + s.ostr << backQuoteIfNeed(name); + s.ostr << " "; + columns->formatImpl(s, state, frame); + s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); + type->formatImpl(s, state, frame); +} + +} + diff --git a/src/Parsers/ASTStatisticDeclaration.h b/src/Parsers/ASTStatisticDeclaration.h new file mode 100644 index 00000000000..0d5ab7723e9 --- /dev/null +++ b/src/Parsers/ASTStatisticDeclaration.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +namespace DB +{ + +class ASTFunction; + +/** name BY columns TYPE typename(args) in create query + */ +class ASTStatisticDeclaration : public IAST +{ +public: + String name; + IAST * columns; + ASTFunction * type; + + /** Get the text that identifies this element. */ + String getID(char) const override { return "Stat"; } + + ASTPtr clone() const override; + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 8292b52f092..bb94d98d587 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -44,6 +44,11 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_clear_index("CLEAR INDEX"); ParserKeyword s_materialize_index("MATERIALIZE INDEX"); + ParserKeyword s_add_statistic("ADD STATISTIC"); + ParserKeyword s_drop_statistic("DROP STATISTIC"); + ParserKeyword s_clear_statistic("CLEAR STATISTIC"); + ParserKeyword s_materialize_statistic("MATERIALIZE STATISTIC"); + ParserKeyword s_add_constraint("ADD CONSTRAINT"); ParserKeyword s_drop_constraint("DROP CONSTRAINT"); @@ -112,6 +117,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserIdentifier parser_remove_property; ParserCompoundColumnDeclaration parser_col_decl; ParserIndexDeclaration parser_idx_decl; + ParserStatisticDeclaration parser_stat_decl; ParserConstraintDeclaration parser_constraint_decl; ParserProjectionDeclaration parser_projection_decl; ParserCompoundColumnDeclaration parser_modify_col_decl(false, false, true); @@ -327,6 +333,70 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; } } + else if (s_add_statistic.ignore(pos, expected)) + { + if (s_if_not_exists.ignore(pos, expected)) + command->if_not_exists = true; + + if (!parser_stat_decl.parse(pos, command->statistic_decl, expected)) + return false; + + command->type = ASTAlterCommand::ADD_STATISTIC; + + if (s_first.ignore(pos, expected)) + command->first = true; + else if (s_after.ignore(pos, expected)) + { + if (!parser_name.parse(pos, command->statistic, expected)) + return false; + } + } + else if (s_drop_statistic.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->statistic, expected)) + return false; + + command->type = ASTAlterCommand::DROP_STATISTIC; + command->detach = false; + } + else if (s_clear_statistic.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->statistic, expected)) + return false; + + command->type = ASTAlterCommand::DROP_STATISTIC; + command->clear_statistic = true; + command->detach = false; + + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } + } + else if (s_materialize_statistic.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->statistic, expected)) + return false; + + command->type = ASTAlterCommand::MATERIALIZE_STATISTIC; + command->detach = false; + + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } + } else if (s_add_projection.ignore(pos, expected)) { if (s_if_not_exists.ignore(pos, expected)) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index adf3513ba40..eb79e250a1d 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -157,6 +158,39 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return true; } +bool ParserStatisticDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_type("TYPE"); + + ParserIdentifier name_p; + ParserDataType data_type_p; + ParserExpression expression_p; + + ASTPtr name; + ASTPtr columns; + ASTPtr type; + + if (!name_p.parse(pos, name, expected)) + return false; + + if (!expression_p.parse(pos, columns, expected)) + return false; + + if (!s_type.ignore(pos, expected)) + return false; + + if (!data_type_p.parse(pos, type, expected)) + return false; + + auto stat = std::make_shared(); + stat->name = name->as().name(); + stat->set(stat->columns, columns); + stat->set(stat->type, type); + node = stat; + + return true; +} + bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_check("CHECK"); @@ -226,11 +260,13 @@ bool ParserProjectionDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_index("INDEX"); + ParserKeyword s_stat("STATISTIC"); ParserKeyword s_constraint("CONSTRAINT"); ParserKeyword s_projection("PROJECTION"); ParserKeyword s_primary_key("PRIMARY KEY"); ParserIndexDeclaration index_p; + ParserStatisticDeclaration stat_p; ParserConstraintDeclaration constraint_p; ParserProjectionDeclaration projection_p; ParserColumnDeclaration column_p{true, true}; @@ -248,6 +284,11 @@ bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte if (!constraint_p.parse(pos, new_node, expected)) return false; } + else if (s_stat.ignore(pos, expected)) + { + if (!stat_p.parse(pos, new_node, expected)) + return false; + } else if (s_projection.ignore(pos, expected)) { if (!projection_p.parse(pos, new_node, expected)) @@ -297,6 +338,7 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr columns = std::make_shared(); ASTPtr indices = std::make_shared(); + ASTPtr stats = std::make_shared(); ASTPtr constraints = std::make_shared(); ASTPtr projections = std::make_shared(); ASTPtr primary_key; @@ -307,6 +349,8 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E columns->children.push_back(elem); else if (elem->as()) indices->children.push_back(elem); + else if (elem->as()) + stats->children.push_back(elem); else if (elem->as()) constraints->children.push_back(elem); else if (elem->as()) @@ -330,6 +374,8 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E res->set(res->columns, columns); if (!indices->children.empty()) res->set(res->indices, indices); + if (!stats->children.empty()) + res->set(res->stats, stats); if (!constraints->children.empty()) res->set(res->constraints, constraints); if (!projections->children.empty()) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 5f79a4b68f6..4e85e3456f3 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -380,6 +380,18 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/** name BY columns TYPE typename(arg1, arg2, ...) */ +/** name BY columns */ +class ParserStatisticDeclaration : public IParserBase +{ +public: + ParserStatisticDeclaration() = default; + +protected: + const char * getName() const override { return "statistics declaration"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + class ParserConstraintDeclaration : public IParserBase { protected: diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index ca8a412bf2e..7c542733927 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -158,6 +158,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) MergeTreeWhereOptimizer where_optimizer{ std::move(column_compressed_sizes), storage_metadata, + storage.getConditionEstimatorByPredicate(read_from_merge_tree->getQueryInfo(), context), queried_columns, storage.supportedPrewhereColumns(), &Poco::Logger::get("QueryPlanOptimizePrewhere")}; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index a9247f9b898..98a5ae3bb2f 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -232,6 +233,25 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } + else if (command_ast->type == ASTAlterCommand::ADD_STATISTIC) + { + AlterCommand command; + command.ast = command_ast->clone(); + command.statistic_decl = command_ast->statistic_decl; + command.type = AlterCommand::ADD_STATISTIC; + + const auto & ast_stat_decl = command_ast->statistic_decl->as(); + + command.statistic_name = ast_stat_decl.name; + + if (command_ast->statistic) + command.after_statistic_name = command_ast->statistic->as().name(); + + command.if_not_exists = command_ast->if_not_exists; + command.first = command_ast->first; + + return command; + } else if (command_ast->type == ASTAlterCommand::ADD_CONSTRAINT) { AlterCommand command; @@ -291,6 +311,20 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } + else if (command_ast->type == ASTAlterCommand::DROP_STATISTIC) + { + AlterCommand command; + command.ast = command_ast->clone(); + command.type = AlterCommand::DROP_STATISTIC; + command.statistic_name = command_ast->statistic->as().name(); + command.if_exists = command_ast->if_exists; + command.clear = command_ast->clear_statistic; + + if (command_ast->partition) + command.partition = command_ast->partition; + + return command; + } else if (command_ast->type == ASTAlterCommand::DROP_PROJECTION) { AlterCommand command; @@ -553,6 +587,68 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) metadata.secondary_indices.erase(erase_it); } } + else if (type == ADD_STATISTIC) + { + if (std::any_of( + metadata.statistics.cbegin(), + metadata.statistics.cend(), + [this](const auto & statistic) + { + return statistic.name == statistic_name; + })) + { + if (if_not_exists) + return; + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add statistic {} : statistic with this name already exists", statistic_name); + } + + auto insert_it = metadata.statistics.end(); + + /// insert the index in the beginning of the indices list + if (first) + insert_it = metadata.statistics.begin(); + + if (!after_statistic_name.empty()) + { + insert_it = std::find_if( + metadata.statistics.begin(), + metadata.statistics.end(), + [this](const auto & statistic) + { + return statistic.name == after_statistic_name; + }); + + if (insert_it == metadata.statistics.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong statistic name. Cannot find statistic {} to insert after", backQuote(after_statistic_name)); + + ++insert_it; + } + + metadata.statistics.emplace(insert_it, StatisticDescription::getStatisticFromAST(statistic_decl, metadata.columns, context)); + } + else if (type == DROP_STATISTIC) + { + if (!partition && !clear) + { + auto erase_it = std::find_if( + metadata.statistics.begin(), + metadata.statistics.end(), + [this](const auto & statistic) + { + return statistic.name == statistic_name; + }); + + if (erase_it == metadata.statistics.end()) + { + if (if_exists) + return; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong statistic name. Cannot find statistic {} to drop", backQuote(statistic_name)); + } + + metadata.statistics.erase(erase_it); + } + } else if (type == ADD_CONSTRAINT) { auto constraints = metadata.constraints.getConstraints(); @@ -877,6 +973,10 @@ std::optional AlterCommand::tryConvertToMutationCommand(Storage result.partition = partition; result.predicate = nullptr; + } + else if (type == DROP_STATISTIC) + { + } else if (type == DROP_PROJECTION) { diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 3e526dcc0bb..eae538815f0 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -38,6 +38,8 @@ struct AlterCommand DROP_CONSTRAINT, ADD_PROJECTION, DROP_PROJECTION, + ADD_STATISTIC, + DROP_STATISTIC, MODIFY_TTL, MODIFY_SETTING, RESET_SETTING, @@ -118,6 +120,10 @@ struct AlterCommand /// For ADD/DROP PROJECTION String projection_name; + ASTPtr statistic_decl = nullptr; + String after_statistic_name; + String statistic_name; + /// For MODIFY TTL ASTPtr ttl = nullptr; diff --git a/src/Storages/ColumnDependency.h b/src/Storages/ColumnDependency.h index 6c3c96ec62a..b9088dd0227 100644 --- a/src/Storages/ColumnDependency.h +++ b/src/Storages/ColumnDependency.h @@ -24,7 +24,10 @@ struct ColumnDependency TTL_EXPRESSION, /// TTL is set for @column_name. - TTL_TARGET + TTL_TARGET, + + /// Exists any statistic, that requires @column_name + STATISTIC, }; ColumnDependency(const String & column_name_, Kind kind_) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b262d88db57..9231b0c3286 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -135,6 +136,8 @@ public: /// Returns true if the storage supports queries with the PREWHERE section. virtual bool supportsPrewhere() const { return false; } + virtual ConditionEstimator getConditionEstimatorByPredicate(const SelectQueryInfo &, ContextPtr) const { return {}; } + /// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported. /// This is needed for engines whose aggregates data from multiple tables, like Merge. virtual std::optional supportedPrewhereColumns() const { return std::nullopt; } diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 23bbc1c7f9d..48fb3303445 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -736,7 +736,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( } MergedBlockOutputStream part_out( - new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, + new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, {}, CompressionCodecFactory::instance().get("NONE", {}), NO_TRANSACTION_PTR); part_out.write(block); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index fd73d802579..2b1cb5fc2de 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -102,6 +103,7 @@ public: const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) = 0; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 8f39c31eae0..c5aa74b3a92 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1,4 +1,5 @@ -#include "Storages/MergeTree/IDataPartStorage.h" +#include +#include #include #include @@ -365,6 +366,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->metadata_snapshot, global_ctx->merging_columns, MergeTreeIndexFactory::instance().getMany(global_ctx->metadata_snapshot->getSecondaryIndices()), + MergeTreeStatisticFactory::instance().getMany(global_ctx->metadata_snapshot->getStatistics()), ctx->compression_codec, global_ctx->txn, /*reset_columns=*/ true, @@ -580,6 +582,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const /// because all of them were already recalculated and written /// as key part of vertical merge std::vector{}, + std::vector{}, /// TODO: think about it &global_ctx->written_offset_columns, global_ctx->to->getIndexGranularity()); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e9c3a7f66ae..31a48d92083 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8458,7 +8458,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, - index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, txn); + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), + Statistics{}, + compression_codec, txn); bool sync_on_insert = settings->fsync_after_insert; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 07e20f16a9f..6f5320062b2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -52,6 +52,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) @@ -66,7 +67,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( return std::make_unique( shared_from_this(), ordered_columns_list, metadata_snapshot, - indices_to_recalc, getMarksFileExtension(), + indices_to_recalc, stats_to_recalc_, getMarksFileExtension(), default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index b115692a7cf..341d464a9da 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -43,6 +43,7 @@ public: const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 468747a6c36..a23b4395df6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -52,6 +52,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & /* indices_to_recalc */, + const Statistics & /* stats_to_recalc_ */, const CompressionCodecPtr & /* default_codec */, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & /* computed_index_granularity */) @@ -92,7 +93,8 @@ MutableDataPartStoragePtr MergeTreeDataPartInMemory::flushToDisk(const String & auto compression_codec = storage.getContext()->chooseCompressionCodec(0, 0); auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); - MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec, NO_TRANSACTION_PTR); + auto stats = MergeTreeStatisticFactory::instance().getMany(metadata_snapshot->getStatistics()); + MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, stats, compression_codec, NO_TRANSACTION_PTR); out.write(block); const auto & projections = metadata_snapshot->getProjections(); @@ -125,6 +127,7 @@ MutableDataPartStoragePtr MergeTreeDataPartInMemory::flushToDisk(const String & MergedBlockOutputStream projection_out( new_projection_part, desc.metadata, new_projection_part->getColumns(), projection_indices, + {}, projection_compression_codec, NO_TRANSACTION_PTR); projection_out.write(old_projection_part->block); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index db7244d8e99..e6ef05319c5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -32,6 +32,7 @@ public: const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index f44cbdd8628..20f430bed8f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -50,13 +50,14 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) { return std::make_unique( shared_from_this(), columns_list, - metadata_snapshot, indices_to_recalc, + metadata_snapshot, indices_to_recalc, stats_to_recalc_, getMarksFileExtension(), default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 5ee497b9b21..6fc195bbfe7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -38,6 +38,7 @@ public: const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 5e1da21da5b..5024ff0217d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -21,12 +21,13 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, + const Statistics & stats_to_recalc, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, - indices_to_recalc_, marks_file_extension_, + indices_to_recalc_, stats_to_recalc, marks_file_extension_, default_codec_, settings_, index_granularity_) , plain_file(data_part_->getDataPartStorage().writeFile( MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, @@ -176,6 +177,7 @@ void MergeTreeDataPartWriterCompact::write(const Block & block, const IColumn::P auto granules_to_write = getGranulesToWrite(index_granularity, flushed_block.rows(), getCurrentMark(), /* last_block = */ false); writeDataBlockPrimaryIndexAndSkipIndices(flushed_block, granules_to_write); setCurrentMark(getCurrentMark() + granules_to_write.size()); + calculateAndSerializeStatistics(flushed_block); } } @@ -422,6 +424,7 @@ void MergeTreeDataPartWriterCompact::fillChecksums(IMergeTreeDataPart::Checksums fillPrimaryIndexChecksums(checksums); fillSkipIndicesChecksums(checksums); + fillStatisticsChecksums(checksums); } void MergeTreeDataPartWriterCompact::finish(bool sync) @@ -434,6 +437,7 @@ void MergeTreeDataPartWriterCompact::finish(bool sync) finishPrimaryIndexSerialization(sync); finishSkipIndicesSerialization(sync); + finishStatisticsSerialization(sync); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 06f8122393f..c5a045c42d0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -15,6 +15,7 @@ public: const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, const MergeTreeWriterSettings & settings, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index f57ffa5ee14..c4f037f65c3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -1,8 +1,6 @@ #include #include #include -#include -#include "IO/WriteBufferFromFileDecorator.h" namespace DB { @@ -11,7 +9,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() +template +void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() { /// Here the main goal is to do preFinalize calls for plain_file and marks_file /// Before that all hashing and compression buffers have to be finalized @@ -22,36 +21,45 @@ void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() compressor.finalize(); plain_hashing.finalize(); - if (compress_marks) + if constexpr (!only_plain_file) { - marks_compressed_hashing.finalize(); - marks_compressor.finalize(); + if (compress_marks) + { + marks_compressed_hashing.finalize(); + marks_compressor.finalize(); + } + + marks_hashing.finalize(); } - marks_hashing.finalize(); - plain_file->preFinalize(); - marks_file->preFinalize(); + if constexpr (!only_plain_file) + marks_file->preFinalize(); is_prefinalized = true; } -void MergeTreeDataPartWriterOnDisk::Stream::finalize() +template +void MergeTreeDataPartWriterOnDisk::Stream::finalize() { if (!is_prefinalized) preFinalize(); plain_file->finalize(); - marks_file->finalize(); + if constexpr (!only_plain_file) + marks_file->finalize(); } -void MergeTreeDataPartWriterOnDisk::Stream::sync() const +template +void MergeTreeDataPartWriterOnDisk::Stream::sync() const { plain_file->sync(); - marks_file->sync(); + if constexpr (!only_plain_file) + marks_file->sync(); } -MergeTreeDataPartWriterOnDisk::Stream::Stream( +template<> +MergeTreeDataPartWriterOnDisk::Stream::Stream( const String & escaped_column_name_, const MutableDataPartStoragePtr & data_part_storage, const String & data_path_, @@ -78,7 +86,27 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( { } -void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums) +template<> +MergeTreeDataPartWriterOnDisk::Stream::Stream( + const String & escaped_column_name_, + const MutableDataPartStoragePtr & data_part_storage, + const String & data_path_, + const std::string & data_file_extension_, + const CompressionCodecPtr & compression_codec_, + size_t max_compress_block_size_, + const WriteSettings & query_write_settings) : + escaped_column_name(escaped_column_name_), + data_file_extension{data_file_extension_}, + plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)), + plain_hashing(*plain_file), + compressor(plain_hashing, compression_codec_, max_compress_block_size_), + compressed_hashing(compressor), + compress_marks(false) +{ +} + +template +void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums) { String name = escaped_column_name; @@ -88,15 +116,18 @@ void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPa checksums.files[name + data_file_extension].file_size = plain_hashing.count(); checksums.files[name + data_file_extension].file_hash = plain_hashing.getHash(); - if (compress_marks) + if constexpr (!only_plain_file) { - checksums.files[name + marks_file_extension].is_compressed = true; - checksums.files[name + marks_file_extension].uncompressed_size = marks_compressed_hashing.count(); - checksums.files[name + marks_file_extension].uncompressed_hash = marks_compressed_hashing.getHash(); - } + if (compress_marks) + { + checksums.files[name + marks_file_extension].is_compressed = true; + checksums.files[name + marks_file_extension].uncompressed_size = marks_compressed_hashing.count(); + checksums.files[name + marks_file_extension].uncompressed_hash = marks_compressed_hashing.getHash(); + } - checksums.files[name + marks_file_extension].file_size = marks_hashing.count(); - checksums.files[name + marks_file_extension].file_hash = marks_hashing.getHash(); + checksums.files[name + marks_file_extension].file_size = marks_hashing.count(); + checksums.files[name + marks_file_extension].file_hash = marks_hashing.getHash(); + } } @@ -105,12 +136,14 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeIndices & indices_to_recalc_, + const Statistics & stats_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter(data_part_, columns_list_, metadata_snapshot_, settings_, index_granularity_) , skip_indices(indices_to_recalc_) + , stats(stats_to_recalc_) , marks_file_extension(marks_file_extension_) , default_codec(default_codec_) , compute_granularity(index_granularity.empty()) @@ -126,6 +159,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( if (settings.rewrite_primary_key) initPrimaryIndex(); initSkipIndices(); + initStatistics(); } // Implementation is split into static functions for ability @@ -207,6 +241,20 @@ void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() } } +void MergeTreeDataPartWriterOnDisk::initStatistics() +{ + for (const auto & stat_ptr : stats) + { + String stats_name = stat_ptr->getFileName(); + stats_streams.emplace_back(std::make_unique>( + stats_name, + data_part->getDataPartStoragePtr(), + stats_name, STAT_FILE_SUFFIX, + default_codec, settings.max_compress_block_size, + settings.query_write_settings)); + } +} + void MergeTreeDataPartWriterOnDisk::initSkipIndices() { ParserCodec codec_parser; @@ -217,7 +265,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() { String stream_name = skip_index->getFileName(); skip_indices_streams.emplace_back( - std::make_unique( + std::make_unique>( stream_name, data_part->getDataPartStoragePtr(), stream_name, skip_index->getSerializedFileExtension(), @@ -279,6 +327,14 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc last_block_index_columns[j] = primary_index_block.getByPosition(j).column; } +void MergeTreeDataPartWriterOnDisk::calculateAndSerializeStatistics(const Block & block) +{ + for (const auto & stat_ptr : stats) + { + stat_ptr->update(block); + } +} + void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block & skip_indexes_block, const Granules & granules_to_write) { /// Filling and writing skip indices like in MergeTreeDataPartWriterWide::writeColumn @@ -417,6 +473,27 @@ void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::Data } } +void MergeTreeDataPartWriterOnDisk::finishStatisticsSerialization(bool sync) +{ + for (auto & stream : stats_streams) + { + stream->finalize(); + if (sync) + stream->sync(); + } +} + +void MergeTreeDataPartWriterOnDisk::fillStatisticsChecksums(MergeTreeData::DataPart::Checksums & checksums) +{ + for (size_t i = 0; i < stats.size(); i++) + { + auto & stream = *stats_streams[i]; + stats[i]->serialize(stream.compressed_hashing); + stream.preFinalize(); + stream.addToChecksums(checksums); + } +} + void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync) { for (auto & stream : skip_indices_streams) @@ -442,4 +519,7 @@ Names MergeTreeDataPartWriterOnDisk::getSkipIndicesColumns() const return Names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end()); } +template struct MergeTreeDataPartWriterOnDisk::Stream; +template struct MergeTreeDataPartWriterOnDisk::Stream; + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index b76b74ab717..30d43b9c180 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -46,6 +47,7 @@ public: /// Helper class, which holds chain of buffers to write data file with marks. /// It is used to write: one column, skip index or all columns (in compact format). + template struct Stream { Stream( @@ -61,6 +63,15 @@ public: size_t marks_compress_block_size_, const WriteSettings & query_write_settings); + Stream( + const String & escaped_column_name_, + const MutableDataPartStoragePtr & data_part_storage, + const String & data_path_, + const std::string & data_file_extension_, + const CompressionCodecPtr & compression_codec_, + size_t max_compress_block_size_, + const WriteSettings & query_write_settings); + String escaped_column_name; std::string data_file_extension; std::string marks_file_extension; @@ -73,9 +84,9 @@ public: /// marks_compressed_hashing -> marks_compressor -> marks_hashing -> marks_file std::unique_ptr marks_file; - HashingWriteBuffer marks_hashing; - CompressedWriteBuffer marks_compressor; - HashingWriteBuffer marks_compressed_hashing; + std::conditional_t marks_hashing; + std::conditional_t marks_compressor; + std::conditional_t marks_compressed_hashing; bool compress_marks; bool is_prefinalized = false; @@ -89,13 +100,15 @@ public: void addToChecksums(IMergeTreeDataPart::Checksums & checksums); }; - using StreamPtr = std::unique_ptr; + using StreamPtr = std::unique_ptr>; + using StatisticStreamPtr = std::unique_ptr>; MergeTreeDataPartWriterOnDisk( const MergeTreeMutableDataPartPtr & data_part_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, const String & marks_file_extension, const CompressionCodecPtr & default_codec, const MergeTreeWriterSettings & settings, @@ -117,6 +130,8 @@ protected: /// require additional state: skip_indices_aggregators and skip_index_accumulated_marks void calculateAndSerializeSkipIndices(const Block & skip_indexes_block, const Granules & granules_to_write); + void calculateAndSerializeStatistics(const Block & stats_block); + /// Finishes primary index serialization: write final primary index row (if required) and compute checksums void fillPrimaryIndexChecksums(MergeTreeData::DataPart::Checksums & checksums); void finishPrimaryIndexSerialization(bool sync); @@ -124,6 +139,9 @@ protected: void fillSkipIndicesChecksums(MergeTreeData::DataPart::Checksums & checksums); void finishSkipIndicesSerialization(bool sync); + void fillStatisticsChecksums(MergeTreeData::DataPart::Checksums & checksums); + void finishStatisticsSerialization(bool sync); + /// Get global number of the current which we are writing (or going to start to write) size_t getCurrentMark() const { return current_mark; } @@ -134,6 +152,9 @@ protected: const MergeTreeIndices skip_indices; + const Statistics stats; + std::vector stats_streams; + const String marks_file_extension; const CompressionCodecPtr default_codec; @@ -166,6 +187,7 @@ protected: private: void initSkipIndices(); void initPrimaryIndex(); + void initStatistics(); virtual void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) = 0; }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index f9fe6f2c8ab..aa97f515074 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -77,12 +77,13 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, + const Statistics & stats_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, - indices_to_recalc_, marks_file_extension_, + indices_to_recalc_, stats_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) { const auto & columns = metadata_snapshot->getColumns(); @@ -116,7 +117,7 @@ void MergeTreeDataPartWriterWide::addStreams( auto ast = parseQuery(codec_parser, "(" + Poco::toUpper(settings.marks_compression_codec) + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); CompressionCodecPtr marks_compression_codec = CompressionCodecFactory::instance().get(ast, nullptr); - column_streams[stream_name] = std::make_unique( + column_streams[stream_name] = std::make_unique>( stream_name, data_part->getDataPartStoragePtr(), stream_name, DATA_FILE_EXTENSION, @@ -256,6 +257,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm calculateAndSerializePrimaryIndex(primary_key_block, granules_to_write); calculateAndSerializeSkipIndices(skip_indexes_block, granules_to_write); + calculateAndSerializeStatistics(block); shiftCurrentMark(granules_to_write); } @@ -272,7 +274,7 @@ void MergeTreeDataPartWriterWide::writeSingleMark( void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stream_with_mark, size_t rows_in_mark) { - Stream & stream = *column_streams[stream_with_mark.stream_name]; + auto & stream = *column_streams[stream_with_mark.stream_name]; WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing; writeIntBinary(stream_with_mark.mark.offset_in_compressed_file, marks_out); @@ -296,7 +298,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( if (is_offsets && offset_columns.contains(stream_name)) return; - Stream & stream = *column_streams[stream_name]; + auto & stream = *column_streams[stream_name]; /// There could already be enough data to compress into the new block. if (stream.compressed_hashing.offset() >= settings.min_compress_block_size) @@ -632,6 +634,8 @@ void MergeTreeDataPartWriterWide::fillChecksums(IMergeTreeDataPart::Checksums & fillPrimaryIndexChecksums(checksums); fillSkipIndicesChecksums(checksums); + + fillStatisticsChecksums(checksums); } void MergeTreeDataPartWriterWide::finish(bool sync) @@ -644,6 +648,8 @@ void MergeTreeDataPartWriterWide::finish(bool sync) finishPrimaryIndexSerialization(sync); finishSkipIndicesSerialization(sync); + + finishStatisticsSerialization(sync); } void MergeTreeDataPartWriterWide::writeFinalMark( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 633b5119474..574225b9614 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -22,6 +22,7 @@ public: const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, const String & marks_file_extension, const CompressionCodecPtr & default_codec, const MergeTreeWriterSettings & settings, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 7e306880e9c..da67bff07f3 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -538,6 +538,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( metadata_snapshot, columns, indices, + MergeTreeStatisticFactory::instance().getMany(metadata_snapshot->getStatistics()), compression_codec, context->getCurrentTransaction(), false, @@ -670,6 +671,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( metadata_snapshot, columns, MergeTreeIndices{}, + Statistics{}, /// TODO(hanfei): It should be helpful to write statistics for projection result. compression_codec, NO_TRANSACTION_PTR, false, false, data.getContext()->getWriteSettings()); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 25a4579c73e..c4046133542 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -20,16 +20,18 @@ namespace DB /// Conditions like "x = N" are considered good if abs(N) > threshold. /// This is used to assume that condition is likely to have good selectivity. -static constexpr auto threshold = 2; +/// static constexpr auto threshold = 2; MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( std::unordered_map column_sizes_, const StorageMetadataPtr & metadata_snapshot, + const ConditionEstimator & estimator_, const Names & queried_columns_, const std::optional & supported_columns_, Poco::Logger * log_) - : table_columns{collections::map( + : estimator(estimator_) + , table_columns{collections::map( metadata_snapshot->getColumns().getAllPhysical(), [](const NameAndTypePair & col) { return col.name; })} , queried_columns{queried_columns_} , supported_columns{supported_columns_} @@ -132,66 +134,66 @@ static void collectColumns(const RPNBuilderTreeNode & node, const NameSet & colu } } -static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet & columns_names) -{ - if (!condition.isFunction()) - return false; - - auto function_node = condition.toFunctionNode(); - - /** We are only considering conditions of form `equals(one, another)` or `one = another`, - * especially if either `one` or `another` is ASTIdentifier - */ - if (function_node.getFunctionName() != "equals" || function_node.getArgumentsSize() != 2) - return false; - - auto lhs_argument = function_node.getArgumentAt(0); - auto rhs_argument = function_node.getArgumentAt(1); - - auto lhs_argument_column_name = lhs_argument.getColumnName(); - auto rhs_argument_column_name = rhs_argument.getColumnName(); - - bool lhs_argument_is_column = columns_names.contains(lhs_argument_column_name); - bool rhs_argument_is_column = columns_names.contains(rhs_argument_column_name); - - bool lhs_argument_is_constant = lhs_argument.isConstant(); - bool rhs_argument_is_constant = rhs_argument.isConstant(); - - RPNBuilderTreeNode * constant_node = nullptr; - - if (lhs_argument_is_column && rhs_argument_is_constant) - constant_node = &rhs_argument; - else if (lhs_argument_is_constant && rhs_argument_is_column) - constant_node = &lhs_argument; - else - return false; - - Field output_value; - DataTypePtr output_type; - if (!constant_node->tryGetConstant(output_value, output_type)) - return false; - - const auto type = output_value.getType(); - - /// check the value with respect to threshold - if (type == Field::Types::UInt64) - { - const auto value = output_value.get(); - return value > threshold; - } - else if (type == Field::Types::Int64) - { - const auto value = output_value.get(); - return value < -threshold || threshold < value; - } - else if (type == Field::Types::Float64) - { - const auto value = output_value.get(); - return value < threshold || threshold < value; - } - - return false; -} +/// static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet & columns_names) +/// { +/// if (!condition.isFunction()) +/// return false; +/// +/// auto function_node = condition.toFunctionNode(); +/// +/// /** We are only considering conditions of form `equals(one, another)` or `one = another`, +/// * especially if either `one` or `another` is ASTIdentifier +/// */ +/// if (function_node.getFunctionName() != "equals" || function_node.getArgumentsSize() != 2) +/// return false; +/// +/// auto lhs_argument = function_node.getArgumentAt(0); +/// auto rhs_argument = function_node.getArgumentAt(1); +/// +/// auto lhs_argument_column_name = lhs_argument.getColumnName(); +/// auto rhs_argument_column_name = rhs_argument.getColumnName(); +/// +/// bool lhs_argument_is_column = columns_names.contains(lhs_argument_column_name); +/// bool rhs_argument_is_column = columns_names.contains(rhs_argument_column_name); +/// +/// bool lhs_argument_is_constant = lhs_argument.isConstant(); +/// bool rhs_argument_is_constant = rhs_argument.isConstant(); +/// +/// RPNBuilderTreeNode * constant_node = nullptr; +/// +/// if (lhs_argument_is_column && rhs_argument_is_constant) +/// constant_node = &rhs_argument; +/// else if (lhs_argument_is_constant && rhs_argument_is_column) +/// constant_node = &lhs_argument; +/// else +/// return false; +/// +/// Field output_value; +/// DataTypePtr output_type; +/// if (!constant_node->tryGetConstant(output_value, output_type)) +/// return false; +/// +/// const auto type = output_value.getType(); +/// +/// /// check the value with respect to threshold +/// if (type == Field::Types::UInt64) +/// { +/// const auto value = output_value.get(); +/// return value > threshold; +/// } +/// else if (type == Field::Types::Int64) +/// { +/// const auto value = output_value.get(); +/// return value < -threshold || threshold < value; +/// } +/// else if (type == Field::Types::Float64) +/// { +/// const auto value = output_value.get(); +/// return value < threshold || threshold < value; +/// } +/// +/// return false; +/// } void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const { @@ -229,7 +231,10 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree && cond.table_columns.size() < queried_columns.size(); if (cond.viable) - cond.good = isConditionGood(node, table_columns); + cond.selectivity = estimator.estimateSelectivity(node); + + ///if (cond.viable) + /// cond.good = isConditionGood(node, table_columns); res.emplace_back(std::move(cond)); } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 18555a72db1..6985237a7c6 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -6,6 +6,7 @@ #include #include +#include "Storages/Statistic/Statistic.h" #include #include @@ -37,6 +38,7 @@ public: MergeTreeWhereOptimizer( std::unordered_map column_sizes_, const StorageMetadataPtr & metadata_snapshot, + const ConditionEstimator & estimator_, const Names & queried_columns_, const std::optional & supported_columns_, Poco::Logger * log_); @@ -69,12 +71,12 @@ private: /// Can condition be moved to prewhere? bool viable = false; - /// Does the condition presumably have good selectivity? - bool good = false; + /// the lower the better + Float64 selectivity = 0; auto tuple() const { - return std::make_tuple(!viable, !good, columns_size, table_columns.size()); + return std::make_tuple(!viable, selectivity, columns_size, table_columns.size()); } /// Is condition a better candidate for moving to PREWHERE? @@ -137,6 +139,8 @@ private: static NameSet determineArrayJoinedNames(const ASTSelectQuery & select); + const ConditionEstimator estimator; + const NameSet table_columns; const Names queried_columns; const std::optional supported_columns; diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 39c4157a42e..a3e0f6bf77b 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -198,7 +198,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore( part, metadata_snapshot, block.getNamesAndTypesList(), - {}, + {}, {}, CompressionCodecFactory::instance().get("NONE", {}), NO_TRANSACTION_PTR); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index bfd9e92b4eb..f4bff1cd42c 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -19,6 +19,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, + const Statistics & statistics, CompressionCodecPtr default_codec_, const MergeTreeTransactionPtr & txn, bool reset_columns_, @@ -47,7 +48,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( data_part->version.setCreationTID(tid, nullptr); data_part->storeVersionMetadata(); - writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, writer_settings, {}); + writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, statistics, default_codec, writer_settings, {}); } /// If data is pre-sorted. diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 20e6de5a99b..48eca3e71f6 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -19,6 +20,7 @@ public: const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, + const Statistics & statistics, CompressionCodecPtr default_codec_, const MergeTreeTransactionPtr & txn, bool reset_columns_ = false, diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 3b2eb96f2d4..492a573a738 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -16,6 +16,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const Block & header_, CompressionCodecPtr default_codec, const MergeTreeIndices & indices_to_recalc, + const Statistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) @@ -36,6 +37,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( header.getNamesAndTypesList(), metadata_snapshot_, indices_to_recalc, + stats_to_recalc_, default_codec, writer_settings, index_granularity); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index f382b0fef60..1a2c56a4f7b 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -19,6 +20,7 @@ public: const Block & header_, CompressionCodecPtr default_codec_, const MergeTreeIndices & indices_to_recalc_, + const Statistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_ = nullptr, const MergeTreeIndexGranularity & index_granularity = {}, const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b98b0844ee7..d16fde50f0a 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -71,6 +72,7 @@ static void splitAndModifyMutationCommands( for (const auto & command : commands) { if (command.type == MutationCommand::Type::MATERIALIZE_INDEX + || command.type == MutationCommand::Type::MATERIALIZE_STATISTIC || command.type == MutationCommand::Type::MATERIALIZE_COLUMN || command.type == MutationCommand::Type::MATERIALIZE_PROJECTION || command.type == MutationCommand::Type::MATERIALIZE_TTL @@ -190,6 +192,7 @@ static void splitAndModifyMutationCommands( { if (command.type == MutationCommand::Type::MATERIALIZE_INDEX || command.type == MutationCommand::Type::MATERIALIZE_COLUMN + || command.type == MutationCommand::Type::MATERIALIZE_STATISTIC || command.type == MutationCommand::Type::MATERIALIZE_PROJECTION || command.type == MutationCommand::Type::MATERIALIZE_TTL || command.type == MutationCommand::Type::DELETE @@ -437,6 +440,20 @@ static ExecuteTTLType shouldExecuteTTL(const StorageMetadataPtr & metadata_snaps return has_ttl_expression ? ExecuteTTLType::RECALCULATE : ExecuteTTLType::NONE; } +static std::set getStatisticsToRecalculate(const StorageMetadataPtr & metadata_snapshot, const NameSet & materialized_stats) +{ + const auto & stats_factory = MergeTreeStatisticFactory::instance(); + std::set stats_to_recalc; + const auto & stats = metadata_snapshot->getStatistics(); + for (const auto & stat_desc : stats) + { + if (materialized_stats.contains(stat_desc.name)) + { + stats_to_recalc.insert(stats_factory.get(stat_desc)); + } + } + return stats_to_recalc; +} /// Return set of indices which should be recalculated during mutation also /// wraps input stream into additional expression stream @@ -527,7 +544,8 @@ static NameSet collectFilesToSkip( const Block & updated_header, const std::set & indices_to_recalc, const String & mrk_extension, - const std::set & projections_to_recalc) + const std::set & projections_to_recalc, + const std::set & stats_to_recalc) { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); @@ -544,6 +562,9 @@ static NameSet collectFilesToSkip( for (const auto & projection : projections_to_recalc) files_to_skip.insert(projection->getDirectoryName()); + for (const auto & stat : stats_to_recalc) + files_to_skip.insert(stat->getFileName() + STAT_FILE_SUFFIX); + if (isWidePart(source_part)) { auto new_stream_counts = getStreamCounts(new_part, new_part->getColumns().getNames()); @@ -620,6 +641,11 @@ static NameToNameVector collectFilesForRenames( if (source_part->checksums.has(command.column_name + ".proj")) add_rename(command.column_name + ".proj", ""); } + //else if (command.type == MutationCommand::Type::DROP_STATISTICS) + //{ + // if (source_part->checksums.has(command.column_name + ".stat")) + // add_rename(command.column_name + ".stat", ""); + //} else if (isWidePart(source_part)) { if (command.type == MutationCommand::Type::DROP_COLUMN) @@ -830,6 +856,7 @@ struct MutationContext NamesAndTypesList storage_columns; NameSet materialized_indices; NameSet materialized_projections; + NameSet materialized_statistics; MergeTreeData::MutableDataPartPtr new_data_part; IMergedBlockOutputStreamPtr out{nullptr}; @@ -840,6 +867,7 @@ struct MutationContext IMergeTreeDataPart::MinMaxIndexPtr minmax_idx{nullptr}; std::set indices_to_recalc; + std::set stats_to_recalc; std::set projections_to_recalc; MergeTreeData::DataPart::Checksums existing_indices_checksums; NameSet files_to_skip; @@ -1292,6 +1320,30 @@ private: } } + Statistics stats; + const auto & statistics = ctx->metadata_snapshot->getStatistics(); + for (const auto & stat : statistics) + { + if (ctx->materialized_statistics.contains(stat.name)) + { + stats.push_back(MergeTreeStatisticFactory::instance().get(stat)); + } + else + { + auto prefix = fmt::format("{}{}.", STAT_FILE_PREFIX, stat.name); + auto it = ctx->source_part->checksums.files.upper_bound(prefix); + while (it != ctx->source_part->checksums.files.end()) + { + if (!startsWith(it->first, prefix)) + break; + + entries_to_hardlink.insert(it->first); + ctx->existing_indices_checksums.addFile(it->first, it->second.file_size, it->second.file_hash); + ++it; + } + } + } + NameSet removed_projections; for (const auto & command : ctx->for_file_renames) { @@ -1376,11 +1428,13 @@ private: ctx->minmax_idx = std::make_shared(); + LOG_TRACE(ctx->log, "going to write {} stats", stats.size()); ctx->out = std::make_shared( ctx->new_data_part, ctx->metadata_snapshot, ctx->new_data_part->getColumns(), skip_indices, + stats, ctx->compression_codec, ctx->txn, /*reset_columns=*/ true, @@ -1575,6 +1629,7 @@ private: ctx->updated_header, ctx->compression_codec, std::vector(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), + Statistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()), nullptr, ctx->source_part->index_granularity, &ctx->source_part->index_granularity_info @@ -1840,6 +1895,8 @@ bool MutateTask::prepare() ctx->metadata_snapshot->getColumns().getNamesOfPhysical(), context_for_reading, settings); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); + ctx->materialized_statistics = ctx->interpreter->grabMaterializedStatistics(); + LOG_INFO(ctx->log, "stats number {}", ctx->materialized_statistics.size()); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); @@ -1904,13 +1961,16 @@ bool MutateTask::prepare() ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate(ctx->metadata_snapshot, ctx->materialized_projections); + ctx->stats_to_recalc = MutationHelpers::getStatisticsToRecalculate(ctx->metadata_snapshot, ctx->materialized_statistics); + ctx->files_to_skip = MutationHelpers::collectFilesToSkip( ctx->source_part, ctx->new_data_part, ctx->updated_header, ctx->indices_to_recalc, ctx->mrk_extension, - ctx->projections_to_recalc); + ctx->projections_to_recalc, + ctx->stats_to_recalc); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 75f1542e30e..b93604bcac6 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -579,6 +579,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) for (auto & index : args.query.columns_list->indices->children) metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, columns, context)); + if (args.query.columns_list && args.query.columns_list->stats) + for (const auto & stat : args.query.columns_list->stats->children) + metadata.statistics.push_back( + StatisticDescription::getStatisticFromAST(stat, columns, args.getContext())); + if (args.query.columns_list && args.query.columns_list->projections) for (auto & projection_ast : args.query.columns_list->projections->children) { diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 6eb345b449e..b00dca95c56 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -10,6 +10,7 @@ #include #include #include +#include "Parsers/ASTAlterQuery.h" #include #include @@ -68,6 +69,16 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.index_name = command->index->as().name(); return res; } + else if (command->type == ASTAlterCommand::MATERIALIZE_STATISTIC) + { + MutationCommand res; + res.ast = command->ptr(); + res.type = MATERIALIZE_STATISTIC; + res.partition = command->partition; + res.predicate = nullptr; + res.statistic_name = command->statistic->as().name(); + return res; + } else if (command->type == ASTAlterCommand::MATERIALIZE_PROJECTION) { MutationCommand res; diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index 5ef0cfda1be..c9fa59bc309 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -30,10 +30,12 @@ struct MutationCommand UPDATE, MATERIALIZE_INDEX, MATERIALIZE_PROJECTION, + MATERIALIZE_STATISTIC, READ_COLUMN, /// Read column and apply conversions (MODIFY COLUMN alter query). DROP_COLUMN, DROP_INDEX, DROP_PROJECTION, + DROP_STATISTIC, MATERIALIZE_TTL, RENAME_COLUMN, MATERIALIZE_COLUMN, @@ -48,9 +50,10 @@ struct MutationCommand /// Columns with corresponding actions std::unordered_map column_to_update_expression; - /// For MATERIALIZE INDEX and PROJECTION + /// For MATERIALIZE INDEX and PROJECTION and STATISTIC String index_name; String projection_name; + String statistic_name; /// For MATERIALIZE INDEX, UPDATE and DELETE. ASTPtr partition; diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp new file mode 100644 index 00000000000..eaf7d828e1f --- /dev/null +++ b/src/Storages/Statistic/Statistic.cpp @@ -0,0 +1,155 @@ +#include +#include +#include +#include +#include "Storages/MergeTree/RPNBuilder.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; +} + + +std::optional ConditionEstimator::extractSingleColumn(const RPNBuilderTreeNode & node) const +{ + if (node.isConstant()) + { + return std::nullopt; + } + + if (!node.isFunction()) + { + auto column_name = node.getColumnName(); + return {column_name}; + } + + auto function_node = node.toFunctionNode(); + size_t arguments_size = function_node.getArgumentsSize(); + std::optional result; + for (size_t i = 0; i < arguments_size; ++i) + { + auto function_argument = function_node.getArgumentAt(i); + auto subresult = extractSingleColumn(function_argument); + if (subresult == std::nullopt) + continue; + else if (subresult == "") + return ""; + else if (result == std::nullopt) + result = subresult; + else if (result.value() != subresult.value()) + return ""; + } + return result; +} + +std::pair ConditionEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const +{ + if (!node.isFunction()) + return {}; + + auto function_node = node.toFunctionNode(); + if (function_node.getArgumentsSize() != 2) + return {}; + + std::string function_name = function_node.getFunctionName(); + + auto lhs_argument = function_node.getArgumentAt(0); + auto rhs_argument = function_node.getArgumentAt(1); + + auto lhs_argument_column_name = lhs_argument.getColumnName(); + auto rhs_argument_column_name = rhs_argument.getColumnName(); + + bool lhs_argument_is_column = column_name == (lhs_argument_column_name); + bool rhs_argument_is_column = column_name == (rhs_argument_column_name); + + bool lhs_argument_is_constant = lhs_argument.isConstant(); + bool rhs_argument_is_constant = rhs_argument.isConstant(); + + RPNBuilderTreeNode * constant_node = nullptr; + + if (lhs_argument_is_column && rhs_argument_is_constant) + constant_node = &rhs_argument; + else if (lhs_argument_is_constant && rhs_argument_is_column) + constant_node = &lhs_argument; + else + return {}; + + Field output_value; + DataTypePtr output_type; + if (!constant_node->tryGetConstant(output_value, output_type)) + return {}; + + const auto type = output_value.getType(); + Float64 value; + if (type == Field::Types::Int64) + value = output_value.get(); + else if (type == Field::Types::UInt64) + value = output_value.get(); + else if (type == Field::Types::Float64) + value = output_value.get(); + return std::make_pair(function_name, value); +} + +StatisticPtr TDigestCreator(const StatisticDescription & stat) +{ + if (stat.column_names.size() != 1) + { + /// throw + } + + /// TODO: check column data types. + return StatisticPtr(new TDigestStatistic(stat)); +} + +void MergeTreeStatisticFactory::registerCreator(const std::string & stat_type, Creator creator) +{ + if (!creators.emplace(stat_type, std::move(creator)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticFactory: the statistic creator type {} is not unique", stat_type); +} + +MergeTreeStatisticFactory::MergeTreeStatisticFactory() +{ + registerCreator("t_digest", TDigestCreator); + + ///registerCreator("cm_sketch", CMSketchCreator); +} + +MergeTreeStatisticFactory & MergeTreeStatisticFactory::instance() +{ + static MergeTreeStatisticFactory instance; + return instance; +} + +StatisticPtr MergeTreeStatisticFactory::get(const StatisticDescription & stat) const +{ + auto it = creators.find(stat.type); + if (it == creators.end()) + { + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Unknown Statistic type '{}'. Available types: {}", stat.type, + std::accumulate(creators.cbegin(), creators.cend(), std::string{}, + [] (auto && left, const auto & right) -> std::string + { + if (left.empty()) + return right.first; + else + return left + ", " + right.first; + }) + ); + } + return std::make_shared(stat); +} + +Statistics MergeTreeStatisticFactory::getMany(const std::vector & stats) const +{ + Statistics result; + for (const auto & stat : stats) + result.push_back(get(stat)); + return result; +} + +} diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h new file mode 100644 index 00000000000..2d16ef90ebe --- /dev/null +++ b/src/Storages/Statistic/Statistic.h @@ -0,0 +1,262 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include "Common/Exception.h" +#include +#include "Storages/MergeTree/RPNBuilder.h" + +#include + +/// this is for user-defined statistic. +/// For auto collected statisic, we can use 'auto_statistic_' +constexpr auto STAT_FILE_PREFIX = "statistic_"; +constexpr auto STAT_FILE_SUFFIX = ".stat"; + +namespace DB +{ + +class IStatistic; +using StatisticPtr = std::shared_ptr; +using Statistics = std::vector; + +class IStatistic +{ +public: + explicit IStatistic(const StatisticDescription & stat_) + : statistics(stat_) + { + } + virtual ~IStatistic() = default; + + String getFileName() const + { + return STAT_FILE_PREFIX + name(); + } + + const String & name() const + { + return statistics.name; + } + + const String & columnName() const + { + return statistics.column_names[0]; + } + /// const String& type() const = 0; + /// virtual StatisticType statisticType() const = 0; + + virtual void serialize(WriteBuffer & buf) = 0; + virtual void deserialize(ReadBuffer & buf) = 0; + virtual void update(const Block & block) = 0; + virtual UInt64 count() = 0; + +protected: + + const StatisticDescription & statistics; + +}; + +class TDigestStatistic : public IStatistic +{ + QuantileTDigest data; +public: + explicit TDigestStatistic(const StatisticDescription & stat) : IStatistic(stat) + { + } + + struct Range + { + Float64 left, right; + }; + + /// FIXME: implement correct count estimate method. + Float64 estimateLess(Float64 val) const + { + return data.getCountLessThan(val); + } + + void serialize(WriteBuffer & buf) override + { + data.serialize(buf); + LOG_DEBUG(&Poco::Logger::get("t-digest"), "serialize into {} data", buf.offset()); + } + + void deserialize(ReadBuffer & buf) override + { + data.deserialize(buf); + } + + void update(const Block & block) override + { + const auto & column_with_type = block.getByName(statistics.column_names[0]); + size_t size = block.rows(); + + for (size_t i = 0; i < size; ++i) + { + /// TODO: support more types. + Float64 value = column_with_type.column->getFloat64(i); + data.add(value, 1); + } + + LOG_DEBUG(&Poco::Logger::get("t-digest"), "write into {} data", size); + } + + UInt64 count() override + { + return static_cast(data.count); + } +}; + +class MergeTreeStatisticFactory : private boost::noncopyable +{ +public: + static MergeTreeStatisticFactory & instance(); + + using Creator = std::function; + + StatisticPtr get(const StatisticDescription & stat) const; + + Statistics getMany(const std::vector & stats) const; + + void registerCreator(const std::string & type, Creator creator); + +protected: + MergeTreeStatisticFactory(); + +private: + using Creators = std::unordered_map; + Creators creators; +}; + +class ConditionEstimator +{ +private: + + static constexpr auto default_good_cond_factor = 0.1; + static constexpr auto default_normal_cond_factor = 0.5; + static constexpr auto default_unknown_cond_factor = 1.0; + /// Conditions like "x = N" are considered good if abs(N) > threshold. + /// This is used to assume that condition is likely to have good selectivity. + static constexpr auto threshold = 2; + + UInt64 total_count; + + struct PartColumnEstimator + { + UInt64 part_count; + + std::shared_ptr t_digest; + + void merge(StatisticPtr statistic) + { + UInt64 cur_part_count = statistic->count(); + if (part_count == 0) + part_count = cur_part_count; + + if (typeid_cast(statistic.get())) + { + t_digest = std::static_pointer_cast(statistic); + } + } + + Float64 estimateLess(Float64 val) const + { + if (t_digest != nullptr) + return t_digest -> estimateLess(val); + return part_count * default_normal_cond_factor; + } + + Float64 estimateGreator(Float64 val) const + { + if (t_digest != nullptr) + return part_count - t_digest -> estimateLess(val); + return part_count * default_normal_cond_factor; + } + }; + + struct ColumnEstimator + { + std::map estimators; + + void merge(std::string part_name, StatisticPtr statistic) + { + estimators[part_name].merge(statistic); + } + Float64 estimateLess(Float64 val) const + { + if (estimators.empty()) + return default_normal_cond_factor; + Float64 result = 0; + for (const auto & [key, estimator] : estimators) + result += estimator.estimateLess(val); + return result; + } + + Float64 estimateGreater(Float64 val) const + { + if (estimators.empty()) + return default_normal_cond_factor; + Float64 result = 0; + for (const auto & [key, estimator] : estimators) + result += estimator.estimateGreator(val); + return result; + } + }; + + std::map column_estimators; + std::optional extractSingleColumn(const RPNBuilderTreeNode & node) const; + std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const; + +public: + + ConditionEstimator() = default; + + /// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ... + /// Right now we only support simple condition like col = val / col < val + Float64 estimateSelectivity(const RPNBuilderTreeNode & node) const + { + auto col = extractSingleColumn(node); + if (col == std::nullopt || col == "") + { + return default_unknown_cond_factor; + } + auto it = column_estimators.find(col.value()); + ColumnEstimator estimator; + if (it != column_estimators.end()) + { + estimator = it->second; + } + auto [op, val] = extractBinaryOp(node, col.value()); + if (op == "equals") + { + if (val < - threshold || val > threshold) + return default_normal_cond_factor; + else + return default_good_cond_factor; + } + else if (op == "less" || op == "lessThan") + { + return estimator.estimateLess(val) / total_count; + } + else if (op == "greater" || op == "greaterThan") + { + return estimator.estimateLess(val) / total_count; + } + else + return default_unknown_cond_factor; + } + void merge(std::string part_name, StatisticPtr statistic) + { + column_estimators[statistic->columnName()].merge(part_name, statistic); + } + +}; + + +} diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp new file mode 100644 index 00000000000..6c7e1244fcd --- /dev/null +++ b/src/Storages/StatisticsDescription.cpp @@ -0,0 +1,120 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; + extern const int LOGICAL_ERROR; +}; + +StatisticDescription StatisticDescription::getStatisticFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context) +{ + const auto * stat_definition = definition_ast->as(); + if (!stat_definition) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create statistic from non ASTStatisticDeclaration AST"); + + if (stat_definition->name.empty()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistic must have name in definition."); + + // type == nullptr => auto + if (!stat_definition->type) + throw Exception(ErrorCodes::INCORRECT_QUERY, "TYPE is required for statistics"); + + if (stat_definition->type->parameters && !stat_definition->type->parameters->children.empty()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistics type cannot have parameters"); + + StatisticDescription stat; + stat.definition_ast = definition_ast->clone(); + stat.name = stat_definition->name; + stat.type = Poco::toLower(stat_definition->type->name); + + ASTPtr expr_list = extractKeyExpressionList(stat_definition->columns->clone()); + for (const auto & ast : expr_list->children) + { + ASTIdentifier* ident = ast->as(); + if (!ident || !columns.hasPhysical(ident->getColumnName())) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column"); + const auto & column = columns.get(ident->getColumnName()); + stat.column_names.push_back(column.name); + stat.data_types.push_back(column.type); + } + + UNUSED(context); + + return stat; +} + +StatisticDescription::StatisticDescription(const StatisticDescription & other) + : definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr) + , name(other.name) + , type(other.type) + , column_names(other.column_names) +{ +} + +StatisticDescription & StatisticDescription::operator=(const StatisticDescription & other) +{ + if (&other == this) + return *this; + + if (other.definition_ast) + definition_ast = other.definition_ast->clone(); + else + definition_ast.reset(); + + name = other.name; + type = other.type; + column_names = other.column_names; + + return *this; +} + + +bool StatisticsDescriptions::has(const String & name) const +{ + for (const auto & statistic : *this) + if (statistic.name == name) + return true; + return false; +} + +String StatisticsDescriptions::toString() const +{ + if (empty()) + return {}; + + ASTExpressionList list; + for (const auto & statistic : *this) + list.children.push_back(statistic.definition_ast); + + return serializeAST(list, true); +} + +StatisticsDescriptions StatisticsDescriptions::parse(const String & str, const ColumnsDescription & columns, ContextPtr context) +{ + StatisticsDescriptions result; + if (str.empty()) + return result; + + ParserStatisticDeclaration parser; + ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + + for (const auto & index : list->children) + result.emplace_back(StatisticDescription::getStatisticFromAST(index, columns, context)); + + return result; +} + +} diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h new file mode 100644 index 00000000000..2cbce381990 --- /dev/null +++ b/src/Storages/StatisticsDescription.h @@ -0,0 +1,46 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct StatisticDescription +{ + /// Definition AST of statistic + ASTPtr definition_ast; + + /// Statistic name + String name; + + String type; + + /// Names of statistic columns + Names column_names; + + /// Data types of statistic columns + DataTypes data_types; + + static StatisticDescription getStatisticFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context); + + StatisticDescription() = default; + + /// We need custom copy constructors because we don't want + /// unintentionaly share AST variables and modify them. + StatisticDescription(const StatisticDescription & other); + StatisticDescription & operator=(const StatisticDescription & other); +}; + +struct StatisticsDescriptions : public std::vector +{ + /// Stat with name exists + bool has(const String & name) const; + /// Convert description to string + String toString() const; + /// Parse description from string + static StatisticsDescriptions parse(const String & str, const ColumnsDescription & columns, ContextPtr context); +}; + +} diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index afe75349864..4546d9a8bda 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -4,6 +4,7 @@ #include #include #include +#include "Storages/StatisticsDescription.h" #include #include #include @@ -28,6 +29,7 @@ namespace ErrorCodes StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & other) : columns(other.columns) , secondary_indices(other.secondary_indices) + , statistics(other.statistics) , constraints(other.constraints) , projections(other.projections.clone()) , minmax_count_projection( @@ -52,6 +54,7 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo columns = other.columns; secondary_indices = other.secondary_indices; + statistics = other.statistics; constraints = other.constraints; projections = other.projections.clone(); if (other.minmax_count_projection) @@ -91,6 +94,11 @@ void StorageInMemoryMetadata::setSecondaryIndices(IndicesDescription secondary_i secondary_indices = std::move(secondary_indices_); } +void StorageInMemoryMetadata::setStatistics(StatisticsDescriptions statistics_) +{ + statistics = std::move(statistics_); +} + void StorageInMemoryMetadata::setConstraints(ConstraintsDescription constraints_) { constraints = std::move(constraints_); @@ -146,6 +154,11 @@ const IndicesDescription & StorageInMemoryMetadata::getSecondaryIndices() const return secondary_indices; } +const StatisticsDescriptions & StorageInMemoryMetadata::getStatistics() const +{ + return statistics; +} + bool StorageInMemoryMetadata::hasSecondaryIndices() const { return !secondary_indices.empty(); diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 4ed7eb8bf29..761788949fb 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -24,6 +25,8 @@ struct StorageInMemoryMetadata ColumnsDescription columns; /// Table indices. Currently supported for MergeTree only. IndicesDescription secondary_indices; + + StatisticsDescriptions statistics; /// Table constraints. Currently supported for MergeTree only. ConstraintsDescription constraints; /// Table projections. Currently supported for MergeTree only. @@ -75,6 +78,9 @@ struct StorageInMemoryMetadata /// Sets secondary indices void setSecondaryIndices(IndicesDescription secondary_indices_); + /// Sets statistics + void setStatistics(StatisticsDescriptions statistics_); + /// Sets constraints void setConstraints(ConstraintsDescription constraints_); @@ -105,6 +111,8 @@ struct StorageInMemoryMetadata /// Returns secondary indices const IndicesDescription & getSecondaryIndices() const; + const StatisticsDescriptions & getStatistics() const; + /// Has at least one non primary index bool hasSecondaryIndices() const; From 0fb68f41c2afb774524896749a6092da7884339e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 11 Aug 2023 16:45:06 +0200 Subject: [PATCH 0025/1097] fix tests --- src/Core/Settings.h | 2 + .../MergeTree/MergeTreeWhereOptimizer.cpp | 137 ++++++++++-------- .../MergeTree/MergeTreeWhereOptimizer.h | 8 +- .../01271_show_privileges.reference | 4 + .../02117_show_create_table_system.reference | 6 +- 5 files changed, 91 insertions(+), 66 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6c3d339b4be..0936d092f99 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -132,6 +132,8 @@ class IColumn; M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ \ + M(Bool, allow_statistic_optimize, false, "use statistic to optimize queries", 0) \ + \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ \ diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index d19e9c410b2..6c331a22b1d 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -10,6 +10,7 @@ #include #include #include +#include "Common/logger_useful.h" #include #include #include @@ -20,7 +21,7 @@ namespace DB /// Conditions like "x = N" are considered good if abs(N) > threshold. /// This is used to assume that condition is likely to have good selectivity. -/// static constexpr auto threshold = 2; +static constexpr auto threshold = 2; static NameToIndexMap fillNamesPositions(const Names & names) { @@ -74,6 +75,8 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( if (it != column_sizes.end()) total_size_of_queried_columns += it->second; } + + LOG_DEBUG(log, "pk columns size : {}", metadata_snapshot->getPrimaryKey().column_names.size()); } void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, const ContextPtr & context) const @@ -92,6 +95,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = select.final(); + where_optimizer_context.use_statistic = context->getSettingsRef().allow_statistic_optimize; RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/); RPNBuilderTreeNode node(select.where().get(), tree_context); @@ -122,6 +126,7 @@ std::optional MergeTreeWhe where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = is_final; + where_optimizer_context.use_statistic = context->getSettingsRef().allow_statistic_optimize; RPNBuilderTreeContext tree_context(context); RPNBuilderTreeNode node(&filter_dag->findInOutputs(filter_column_name), tree_context); @@ -167,66 +172,66 @@ static void collectColumns(const RPNBuilderTreeNode & node, const NameSet & colu } } -/// static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet & columns_names) -/// { -/// if (!condition.isFunction()) -/// return false; -/// -/// auto function_node = condition.toFunctionNode(); -/// -/// /** We are only considering conditions of form `equals(one, another)` or `one = another`, -/// * especially if either `one` or `another` is ASTIdentifier -/// */ -/// if (function_node.getFunctionName() != "equals" || function_node.getArgumentsSize() != 2) -/// return false; -/// -/// auto lhs_argument = function_node.getArgumentAt(0); -/// auto rhs_argument = function_node.getArgumentAt(1); -/// -/// auto lhs_argument_column_name = lhs_argument.getColumnName(); -/// auto rhs_argument_column_name = rhs_argument.getColumnName(); -/// -/// bool lhs_argument_is_column = columns_names.contains(lhs_argument_column_name); -/// bool rhs_argument_is_column = columns_names.contains(rhs_argument_column_name); -/// -/// bool lhs_argument_is_constant = lhs_argument.isConstant(); -/// bool rhs_argument_is_constant = rhs_argument.isConstant(); -/// -/// RPNBuilderTreeNode * constant_node = nullptr; -/// -/// if (lhs_argument_is_column && rhs_argument_is_constant) -/// constant_node = &rhs_argument; -/// else if (lhs_argument_is_constant && rhs_argument_is_column) -/// constant_node = &lhs_argument; -/// else -/// return false; -/// -/// Field output_value; -/// DataTypePtr output_type; -/// if (!constant_node->tryGetConstant(output_value, output_type)) -/// return false; -/// -/// const auto type = output_value.getType(); -/// -/// /// check the value with respect to threshold -/// if (type == Field::Types::UInt64) -/// { -/// const auto value = output_value.get(); -/// return value > threshold; -/// } -/// else if (type == Field::Types::Int64) -/// { -/// const auto value = output_value.get(); -/// return value < -threshold || threshold < value; -/// } -/// else if (type == Field::Types::Float64) -/// { -/// const auto value = output_value.get(); -/// return value < threshold || threshold < value; -/// } -/// -/// return false; -/// } +static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet & columns_names) +{ + if (!condition.isFunction()) + return false; + + auto function_node = condition.toFunctionNode(); + + /** We are only considering conditions of form `equals(one, another)` or `one = another`, + * especially if either `one` or `another` is ASTIdentifier + */ + if (function_node.getFunctionName() != "equals" || function_node.getArgumentsSize() != 2) + return false; + + auto lhs_argument = function_node.getArgumentAt(0); + auto rhs_argument = function_node.getArgumentAt(1); + + auto lhs_argument_column_name = lhs_argument.getColumnName(); + auto rhs_argument_column_name = rhs_argument.getColumnName(); + + bool lhs_argument_is_column = columns_names.contains(lhs_argument_column_name); + bool rhs_argument_is_column = columns_names.contains(rhs_argument_column_name); + + bool lhs_argument_is_constant = lhs_argument.isConstant(); + bool rhs_argument_is_constant = rhs_argument.isConstant(); + + RPNBuilderTreeNode * constant_node = nullptr; + + if (lhs_argument_is_column && rhs_argument_is_constant) + constant_node = &rhs_argument; + else if (lhs_argument_is_constant && rhs_argument_is_column) + constant_node = &lhs_argument; + else + return false; + + Field output_value; + DataTypePtr output_type; + if (!constant_node->tryGetConstant(output_value, output_type)) + return false; + + const auto type = output_value.getType(); + + /// check the value with respect to threshold + if (type == Field::Types::UInt64) + { + const auto value = output_value.get(); + return value > threshold; + } + else if (type == Field::Types::Int64) + { + const auto value = output_value.get(); + return value < -threshold || threshold < value; + } + else if (type == Field::Types::Float64) + { + const auto value = output_value.get(); + return value < threshold || threshold < value; + } + + return false; +} void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const { @@ -264,13 +269,23 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree && cond.table_columns.size() < queried_columns.size(); if (cond.viable) + cond.good = isConditionGood(node, table_columns); + + if (where_optimizer_context.use_statistic) + { + cond.good = cond.viable; + cond.selectivity = estimator.estimateSelectivity(node); + } if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) { + /// Consider all conditions good with this setting enabled. + cond.good = cond.viable; /// Find min position in PK of any column that is used in this condition. cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); } + LOG_DEBUG(log, "node {}, min pos : {}", node.getASTNode()->dumpTree(), cond.min_position_in_primary_key); res.emplace_back(std::move(cond)); } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index f5e6b6fbd49..e64ae1ba89c 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -71,8 +71,11 @@ private: /// Can condition be moved to prewhere? bool viable = false; + /// Does the condition presumably have good selectivity? + bool good = false; + /// the lower the better - Float64 selectivity = 0; + Float64 selectivity = 1.0; /// Does the condition contain primary key column? /// If so, it is better to move it further to the end of PREWHERE chain depending on minimal position in PK of any @@ -81,7 +84,7 @@ private: auto tuple() const { - return std::make_tuple(!viable, selectivity, -min_position_in_primary_key, columns_size, table_columns.size()); + return std::make_tuple(!viable, !good, -min_position_in_primary_key, selectivity, columns_size, table_columns.size()); } /// Is condition a better candidate for moving to PREWHERE? @@ -100,6 +103,7 @@ private: bool move_all_conditions_to_prewhere = false; bool move_primary_key_columns_to_end_of_prewhere = false; bool is_final = false; + bool use_statistic = false; }; struct OptimizeResult diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index a3e15f0793c..c096a182f2b 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -24,6 +24,10 @@ ALTER DROP INDEX ['DROP INDEX'] TABLE ALTER INDEX ALTER MATERIALIZE INDEX ['MATERIALIZE INDEX'] TABLE ALTER INDEX ALTER CLEAR INDEX ['CLEAR INDEX'] TABLE ALTER INDEX ALTER INDEX ['INDEX'] \N ALTER TABLE +ALTER ADD STATISTIC ['ALTER ADD STATISTIC'] TABLE ALTER STATISTIC +ALTER DROP STATISTIC ['ALTER DROP STATISTIC'] TABLE ALTER STATISTIC +ALTER MATERIALIZE STATISTIC ['ALTER MATERIALIZE STATISTIC'] TABLE ALTER STATISTIC +ALTER STATISTIC ['STATISTIC'] \N ALTER TABLE ALTER ADD PROJECTION ['ADD PROJECTION'] TABLE ALTER PROJECTION ALTER DROP PROJECTION ['DROP PROJECTION'] TABLE ALTER PROJECTION ALTER MATERIALIZE PROJECTION ['MATERIALIZE PROJECTION'] TABLE ALTER PROJECTION diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 5b678537248..f740753382f 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -298,7 +298,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD STATISTIC' = 26, 'ALTER DROP STATISTIC' = 27, 'ALTER MATERIALIZE STATISTIC' = 28, 'ALTER STATISTIC' = 29, 'ALTER ADD PROJECTION' = 30, 'ALTER DROP PROJECTION' = 31, 'ALTER MATERIALIZE PROJECTION' = 32, 'ALTER CLEAR PROJECTION' = 33, 'ALTER PROJECTION' = 34, 'ALTER ADD CONSTRAINT' = 35, 'ALTER DROP CONSTRAINT' = 36, 'ALTER CONSTRAINT' = 37, 'ALTER TTL' = 38, 'ALTER MATERIALIZE TTL' = 39, 'ALTER SETTINGS' = 40, 'ALTER MOVE PARTITION' = 41, 'ALTER FETCH PARTITION' = 42, 'ALTER FREEZE PARTITION' = 43, 'ALTER DATABASE SETTINGS' = 44, 'ALTER NAMED COLLECTION' = 45, 'ALTER TABLE' = 46, 'ALTER DATABASE' = 47, 'ALTER VIEW REFRESH' = 48, 'ALTER VIEW MODIFY QUERY' = 49, 'ALTER VIEW' = 50, 'ALTER' = 51, 'CREATE DATABASE' = 52, 'CREATE TABLE' = 53, 'CREATE VIEW' = 54, 'CREATE DICTIONARY' = 55, 'CREATE TEMPORARY TABLE' = 56, 'CREATE ARBITRARY TEMPORARY TABLE' = 57, 'CREATE FUNCTION' = 58, 'CREATE NAMED COLLECTION' = 59, 'CREATE' = 60, 'DROP DATABASE' = 61, 'DROP TABLE' = 62, 'DROP VIEW' = 63, 'DROP DICTIONARY' = 64, 'DROP FUNCTION' = 65, 'DROP NAMED COLLECTION' = 66, 'DROP' = 67, 'UNDROP TABLE' = 68, 'TRUNCATE' = 69, 'OPTIMIZE' = 70, 'BACKUP' = 71, 'KILL QUERY' = 72, 'KILL TRANSACTION' = 73, 'MOVE PARTITION BETWEEN SHARDS' = 74, 'CREATE USER' = 75, 'ALTER USER' = 76, 'DROP USER' = 77, 'CREATE ROLE' = 78, 'ALTER ROLE' = 79, 'DROP ROLE' = 80, 'ROLE ADMIN' = 81, 'CREATE ROW POLICY' = 82, 'ALTER ROW POLICY' = 83, 'DROP ROW POLICY' = 84, 'CREATE QUOTA' = 85, 'ALTER QUOTA' = 86, 'DROP QUOTA' = 87, 'CREATE SETTINGS PROFILE' = 88, 'ALTER SETTINGS PROFILE' = 89, 'DROP SETTINGS PROFILE' = 90, 'SHOW USERS' = 91, 'SHOW ROLES' = 92, 'SHOW ROW POLICIES' = 93, 'SHOW QUOTAS' = 94, 'SHOW SETTINGS PROFILES' = 95, 'SHOW ACCESS' = 96, 'ACCESS MANAGEMENT' = 97, 'SHOW NAMED COLLECTIONS' = 98, 'SHOW NAMED COLLECTIONS SECRETS' = 99, 'NAMED COLLECTION' = 100, 'NAMED COLLECTION ADMIN' = 101, 'SYSTEM SHUTDOWN' = 102, 'SYSTEM DROP DNS CACHE' = 103, 'SYSTEM DROP MARK CACHE' = 104, 'SYSTEM DROP UNCOMPRESSED CACHE' = 105, 'SYSTEM DROP MMAP CACHE' = 106, 'SYSTEM DROP QUERY CACHE' = 107, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 108, 'SYSTEM DROP FILESYSTEM CACHE' = 109, 'SYSTEM DROP SCHEMA CACHE' = 110, 'SYSTEM DROP S3 CLIENT CACHE' = 111, 'SYSTEM DROP CACHE' = 112, 'SYSTEM RELOAD CONFIG' = 113, 'SYSTEM RELOAD USERS' = 114, 'SYSTEM RELOAD DICTIONARY' = 115, 'SYSTEM RELOAD MODEL' = 116, 'SYSTEM RELOAD FUNCTION' = 117, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 118, 'SYSTEM RELOAD' = 119, 'SYSTEM RESTART DISK' = 120, 'SYSTEM MERGES' = 121, 'SYSTEM TTL MERGES' = 122, 'SYSTEM FETCHES' = 123, 'SYSTEM MOVES' = 124, 'SYSTEM PULLING REPLICATION LOG' = 125, 'SYSTEM DISTRIBUTED SENDS' = 126, 'SYSTEM REPLICATED SENDS' = 127, 'SYSTEM SENDS' = 128, 'SYSTEM REPLICATION QUEUES' = 129, 'SYSTEM DROP REPLICA' = 130, 'SYSTEM SYNC REPLICA' = 131, 'SYSTEM RESTART REPLICA' = 132, 'SYSTEM RESTORE REPLICA' = 133, 'SYSTEM WAIT LOADING PARTS' = 134, 'SYSTEM SYNC DATABASE REPLICA' = 135, 'SYSTEM SYNC TRANSACTION LOG' = 136, 'SYSTEM SYNC FILE CACHE' = 137, 'SYSTEM FLUSH DISTRIBUTED' = 138, 'SYSTEM FLUSH LOGS' = 139, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 140, 'SYSTEM FLUSH' = 141, 'SYSTEM THREAD FUZZER' = 142, 'SYSTEM UNFREEZE' = 143, 'SYSTEM FAILPOINT' = 144, 'SYSTEM LISTEN' = 145, 'SYSTEM' = 146, 'dictGet' = 147, 'displaySecretsInShowAndSelect' = 148, 'addressToLine' = 149, 'addressToLineWithInlines' = 150, 'addressToSymbol' = 151, 'demangle' = 152, 'INTROSPECTION' = 153, 'FILE' = 154, 'URL' = 155, 'REMOTE' = 156, 'MONGO' = 157, 'REDIS' = 158, 'MEILISEARCH' = 159, 'MYSQL' = 160, 'POSTGRES' = 161, 'SQLITE' = 162, 'ODBC' = 163, 'JDBC' = 164, 'HDFS' = 165, 'S3' = 166, 'HIVE' = 167, 'AZURE' = 168, 'SOURCES' = 169, 'CLUSTER' = 170, 'ALL' = 171, 'NONE' = 172), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -586,10 +586,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD STATISTIC' = 26, 'ALTER DROP STATISTIC' = 27, 'ALTER MATERIALIZE STATISTIC' = 28, 'ALTER STATISTIC' = 29, 'ALTER ADD PROJECTION' = 30, 'ALTER DROP PROJECTION' = 31, 'ALTER MATERIALIZE PROJECTION' = 32, 'ALTER CLEAR PROJECTION' = 33, 'ALTER PROJECTION' = 34, 'ALTER ADD CONSTRAINT' = 35, 'ALTER DROP CONSTRAINT' = 36, 'ALTER CONSTRAINT' = 37, 'ALTER TTL' = 38, 'ALTER MATERIALIZE TTL' = 39, 'ALTER SETTINGS' = 40, 'ALTER MOVE PARTITION' = 41, 'ALTER FETCH PARTITION' = 42, 'ALTER FREEZE PARTITION' = 43, 'ALTER DATABASE SETTINGS' = 44, 'ALTER NAMED COLLECTION' = 45, 'ALTER TABLE' = 46, 'ALTER DATABASE' = 47, 'ALTER VIEW REFRESH' = 48, 'ALTER VIEW MODIFY QUERY' = 49, 'ALTER VIEW' = 50, 'ALTER' = 51, 'CREATE DATABASE' = 52, 'CREATE TABLE' = 53, 'CREATE VIEW' = 54, 'CREATE DICTIONARY' = 55, 'CREATE TEMPORARY TABLE' = 56, 'CREATE ARBITRARY TEMPORARY TABLE' = 57, 'CREATE FUNCTION' = 58, 'CREATE NAMED COLLECTION' = 59, 'CREATE' = 60, 'DROP DATABASE' = 61, 'DROP TABLE' = 62, 'DROP VIEW' = 63, 'DROP DICTIONARY' = 64, 'DROP FUNCTION' = 65, 'DROP NAMED COLLECTION' = 66, 'DROP' = 67, 'UNDROP TABLE' = 68, 'TRUNCATE' = 69, 'OPTIMIZE' = 70, 'BACKUP' = 71, 'KILL QUERY' = 72, 'KILL TRANSACTION' = 73, 'MOVE PARTITION BETWEEN SHARDS' = 74, 'CREATE USER' = 75, 'ALTER USER' = 76, 'DROP USER' = 77, 'CREATE ROLE' = 78, 'ALTER ROLE' = 79, 'DROP ROLE' = 80, 'ROLE ADMIN' = 81, 'CREATE ROW POLICY' = 82, 'ALTER ROW POLICY' = 83, 'DROP ROW POLICY' = 84, 'CREATE QUOTA' = 85, 'ALTER QUOTA' = 86, 'DROP QUOTA' = 87, 'CREATE SETTINGS PROFILE' = 88, 'ALTER SETTINGS PROFILE' = 89, 'DROP SETTINGS PROFILE' = 90, 'SHOW USERS' = 91, 'SHOW ROLES' = 92, 'SHOW ROW POLICIES' = 93, 'SHOW QUOTAS' = 94, 'SHOW SETTINGS PROFILES' = 95, 'SHOW ACCESS' = 96, 'ACCESS MANAGEMENT' = 97, 'SHOW NAMED COLLECTIONS' = 98, 'SHOW NAMED COLLECTIONS SECRETS' = 99, 'NAMED COLLECTION' = 100, 'NAMED COLLECTION ADMIN' = 101, 'SYSTEM SHUTDOWN' = 102, 'SYSTEM DROP DNS CACHE' = 103, 'SYSTEM DROP MARK CACHE' = 104, 'SYSTEM DROP UNCOMPRESSED CACHE' = 105, 'SYSTEM DROP MMAP CACHE' = 106, 'SYSTEM DROP QUERY CACHE' = 107, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 108, 'SYSTEM DROP FILESYSTEM CACHE' = 109, 'SYSTEM DROP SCHEMA CACHE' = 110, 'SYSTEM DROP S3 CLIENT CACHE' = 111, 'SYSTEM DROP CACHE' = 112, 'SYSTEM RELOAD CONFIG' = 113, 'SYSTEM RELOAD USERS' = 114, 'SYSTEM RELOAD DICTIONARY' = 115, 'SYSTEM RELOAD MODEL' = 116, 'SYSTEM RELOAD FUNCTION' = 117, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 118, 'SYSTEM RELOAD' = 119, 'SYSTEM RESTART DISK' = 120, 'SYSTEM MERGES' = 121, 'SYSTEM TTL MERGES' = 122, 'SYSTEM FETCHES' = 123, 'SYSTEM MOVES' = 124, 'SYSTEM PULLING REPLICATION LOG' = 125, 'SYSTEM DISTRIBUTED SENDS' = 126, 'SYSTEM REPLICATED SENDS' = 127, 'SYSTEM SENDS' = 128, 'SYSTEM REPLICATION QUEUES' = 129, 'SYSTEM DROP REPLICA' = 130, 'SYSTEM SYNC REPLICA' = 131, 'SYSTEM RESTART REPLICA' = 132, 'SYSTEM RESTORE REPLICA' = 133, 'SYSTEM WAIT LOADING PARTS' = 134, 'SYSTEM SYNC DATABASE REPLICA' = 135, 'SYSTEM SYNC TRANSACTION LOG' = 136, 'SYSTEM SYNC FILE CACHE' = 137, 'SYSTEM FLUSH DISTRIBUTED' = 138, 'SYSTEM FLUSH LOGS' = 139, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 140, 'SYSTEM FLUSH' = 141, 'SYSTEM THREAD FUZZER' = 142, 'SYSTEM UNFREEZE' = 143, 'SYSTEM FAILPOINT' = 144, 'SYSTEM LISTEN' = 145, 'SYSTEM' = 146, 'dictGet' = 147, 'displaySecretsInShowAndSelect' = 148, 'addressToLine' = 149, 'addressToLineWithInlines' = 150, 'addressToSymbol' = 151, 'demangle' = 152, 'INTROSPECTION' = 153, 'FILE' = 154, 'URL' = 155, 'REMOTE' = 156, 'MONGO' = 157, 'REDIS' = 158, 'MEILISEARCH' = 159, 'MYSQL' = 160, 'POSTGRES' = 161, 'SQLITE' = 162, 'ODBC' = 163, 'JDBC' = 164, 'HDFS' = 165, 'S3' = 166, 'HIVE' = 167, 'AZURE' = 168, 'SOURCES' = 169, 'CLUSTER' = 170, 'ALL' = 171, 'NONE' = 172), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD STATISTIC' = 26, 'ALTER DROP STATISTIC' = 27, 'ALTER MATERIALIZE STATISTIC' = 28, 'ALTER STATISTIC' = 29, 'ALTER ADD PROJECTION' = 30, 'ALTER DROP PROJECTION' = 31, 'ALTER MATERIALIZE PROJECTION' = 32, 'ALTER CLEAR PROJECTION' = 33, 'ALTER PROJECTION' = 34, 'ALTER ADD CONSTRAINT' = 35, 'ALTER DROP CONSTRAINT' = 36, 'ALTER CONSTRAINT' = 37, 'ALTER TTL' = 38, 'ALTER MATERIALIZE TTL' = 39, 'ALTER SETTINGS' = 40, 'ALTER MOVE PARTITION' = 41, 'ALTER FETCH PARTITION' = 42, 'ALTER FREEZE PARTITION' = 43, 'ALTER DATABASE SETTINGS' = 44, 'ALTER NAMED COLLECTION' = 45, 'ALTER TABLE' = 46, 'ALTER DATABASE' = 47, 'ALTER VIEW REFRESH' = 48, 'ALTER VIEW MODIFY QUERY' = 49, 'ALTER VIEW' = 50, 'ALTER' = 51, 'CREATE DATABASE' = 52, 'CREATE TABLE' = 53, 'CREATE VIEW' = 54, 'CREATE DICTIONARY' = 55, 'CREATE TEMPORARY TABLE' = 56, 'CREATE ARBITRARY TEMPORARY TABLE' = 57, 'CREATE FUNCTION' = 58, 'CREATE NAMED COLLECTION' = 59, 'CREATE' = 60, 'DROP DATABASE' = 61, 'DROP TABLE' = 62, 'DROP VIEW' = 63, 'DROP DICTIONARY' = 64, 'DROP FUNCTION' = 65, 'DROP NAMED COLLECTION' = 66, 'DROP' = 67, 'UNDROP TABLE' = 68, 'TRUNCATE' = 69, 'OPTIMIZE' = 70, 'BACKUP' = 71, 'KILL QUERY' = 72, 'KILL TRANSACTION' = 73, 'MOVE PARTITION BETWEEN SHARDS' = 74, 'CREATE USER' = 75, 'ALTER USER' = 76, 'DROP USER' = 77, 'CREATE ROLE' = 78, 'ALTER ROLE' = 79, 'DROP ROLE' = 80, 'ROLE ADMIN' = 81, 'CREATE ROW POLICY' = 82, 'ALTER ROW POLICY' = 83, 'DROP ROW POLICY' = 84, 'CREATE QUOTA' = 85, 'ALTER QUOTA' = 86, 'DROP QUOTA' = 87, 'CREATE SETTINGS PROFILE' = 88, 'ALTER SETTINGS PROFILE' = 89, 'DROP SETTINGS PROFILE' = 90, 'SHOW USERS' = 91, 'SHOW ROLES' = 92, 'SHOW ROW POLICIES' = 93, 'SHOW QUOTAS' = 94, 'SHOW SETTINGS PROFILES' = 95, 'SHOW ACCESS' = 96, 'ACCESS MANAGEMENT' = 97, 'SHOW NAMED COLLECTIONS' = 98, 'SHOW NAMED COLLECTIONS SECRETS' = 99, 'NAMED COLLECTION' = 100, 'NAMED COLLECTION ADMIN' = 101, 'SYSTEM SHUTDOWN' = 102, 'SYSTEM DROP DNS CACHE' = 103, 'SYSTEM DROP MARK CACHE' = 104, 'SYSTEM DROP UNCOMPRESSED CACHE' = 105, 'SYSTEM DROP MMAP CACHE' = 106, 'SYSTEM DROP QUERY CACHE' = 107, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 108, 'SYSTEM DROP FILESYSTEM CACHE' = 109, 'SYSTEM DROP SCHEMA CACHE' = 110, 'SYSTEM DROP S3 CLIENT CACHE' = 111, 'SYSTEM DROP CACHE' = 112, 'SYSTEM RELOAD CONFIG' = 113, 'SYSTEM RELOAD USERS' = 114, 'SYSTEM RELOAD DICTIONARY' = 115, 'SYSTEM RELOAD MODEL' = 116, 'SYSTEM RELOAD FUNCTION' = 117, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 118, 'SYSTEM RELOAD' = 119, 'SYSTEM RESTART DISK' = 120, 'SYSTEM MERGES' = 121, 'SYSTEM TTL MERGES' = 122, 'SYSTEM FETCHES' = 123, 'SYSTEM MOVES' = 124, 'SYSTEM PULLING REPLICATION LOG' = 125, 'SYSTEM DISTRIBUTED SENDS' = 126, 'SYSTEM REPLICATED SENDS' = 127, 'SYSTEM SENDS' = 128, 'SYSTEM REPLICATION QUEUES' = 129, 'SYSTEM DROP REPLICA' = 130, 'SYSTEM SYNC REPLICA' = 131, 'SYSTEM RESTART REPLICA' = 132, 'SYSTEM RESTORE REPLICA' = 133, 'SYSTEM WAIT LOADING PARTS' = 134, 'SYSTEM SYNC DATABASE REPLICA' = 135, 'SYSTEM SYNC TRANSACTION LOG' = 136, 'SYSTEM SYNC FILE CACHE' = 137, 'SYSTEM FLUSH DISTRIBUTED' = 138, 'SYSTEM FLUSH LOGS' = 139, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 140, 'SYSTEM FLUSH' = 141, 'SYSTEM THREAD FUZZER' = 142, 'SYSTEM UNFREEZE' = 143, 'SYSTEM FAILPOINT' = 144, 'SYSTEM LISTEN' = 145, 'SYSTEM' = 146, 'dictGet' = 147, 'displaySecretsInShowAndSelect' = 148, 'addressToLine' = 149, 'addressToLineWithInlines' = 150, 'addressToSymbol' = 151, 'demangle' = 152, 'INTROSPECTION' = 153, 'FILE' = 154, 'URL' = 155, 'REMOTE' = 156, 'MONGO' = 157, 'REDIS' = 158, 'MEILISEARCH' = 159, 'MYSQL' = 160, 'POSTGRES' = 161, 'SQLITE' = 162, 'ODBC' = 163, 'JDBC' = 164, 'HDFS' = 165, 'S3' = 166, 'HIVE' = 167, 'AZURE' = 168, 'SOURCES' = 169, 'CLUSTER' = 170, 'ALL' = 171, 'NONE' = 172)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From 9cbcf3a72a30b6dfb23bfb7df29b60ea5337c40e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 11 Aug 2023 21:02:44 +0200 Subject: [PATCH 0026/1097] fix build --- src/Storages/Statistic/Statistic.cpp | 35 +++++++++++++++++++++++++- src/Storages/Statistic/Statistic.h | 37 +++------------------------- 2 files changed, 38 insertions(+), 34 deletions(-) diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index eaf7d828e1f..2b2ebde070a 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -2,7 +2,7 @@ #include #include #include -#include "Storages/MergeTree/RPNBuilder.h" +#include namespace DB { @@ -94,6 +94,39 @@ std::pair ConditionEstimator::extractBinaryOp(const RPNBui return std::make_pair(function_name, value); } +Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) const +{ + auto col = extractSingleColumn(node); + if (col == std::nullopt || col == "") + { + return default_unknown_cond_factor; + } + auto it = column_estimators.find(col.value()); + ColumnEstimator estimator; + if (it != column_estimators.end()) + { + estimator = it->second; + } + auto [op, val] = extractBinaryOp(node, col.value()); + if (op == "equals") + { + if (val < - threshold || val > threshold) + return default_normal_cond_factor; + else + return default_good_cond_factor; + } + else if (op == "less" || op == "lessThan") + { + return estimator.estimateLess(val) / total_count; + } + else if (op == "greater" || op == "greaterThan") + { + return estimator.estimateLess(val) / total_count; + } + else + return default_unknown_cond_factor; +} + StatisticPtr TDigestCreator(const StatisticDescription & stat) { if (stat.column_names.size() != 1) diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index 2d16ef90ebe..460dea382fc 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -9,7 +9,6 @@ #include #include "Common/Exception.h" #include -#include "Storages/MergeTree/RPNBuilder.h" #include @@ -134,6 +133,8 @@ private: Creators creators; }; +class RPNBuilderTreeNode; + class ConditionEstimator { private: @@ -219,38 +220,8 @@ public: /// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ... /// Right now we only support simple condition like col = val / col < val - Float64 estimateSelectivity(const RPNBuilderTreeNode & node) const - { - auto col = extractSingleColumn(node); - if (col == std::nullopt || col == "") - { - return default_unknown_cond_factor; - } - auto it = column_estimators.find(col.value()); - ColumnEstimator estimator; - if (it != column_estimators.end()) - { - estimator = it->second; - } - auto [op, val] = extractBinaryOp(node, col.value()); - if (op == "equals") - { - if (val < - threshold || val > threshold) - return default_normal_cond_factor; - else - return default_good_cond_factor; - } - else if (op == "less" || op == "lessThan") - { - return estimator.estimateLess(val) / total_count; - } - else if (op == "greater" || op == "greaterThan") - { - return estimator.estimateLess(val) / total_count; - } - else - return default_unknown_cond_factor; - } + Float64 estimateSelectivity(const RPNBuilderTreeNode & node) const; + void merge(std::string part_name, StatisticPtr statistic) { column_estimators[statistic->columnName()].merge(part_name, statistic); From 82996ad33bca4e4236c2ac8f5f28ed69c0003de6 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 11 Aug 2023 23:33:06 +0200 Subject: [PATCH 0027/1097] add missing header --- src/Storages/Statistic/Statistic.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index 2b2ebde070a..a8018458e5f 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -1,4 +1,6 @@ #include +#include + #include #include #include From 838a83d1082bd86c3bd1c9eda0cdf92fc5cbbea8 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 12 Aug 2023 00:53:33 +0200 Subject: [PATCH 0028/1097] remove logs --- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 6c331a22b1d..8fffbdb5068 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -75,8 +75,6 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( if (it != column_sizes.end()) total_size_of_queried_columns += it->second; } - - LOG_DEBUG(log, "pk columns size : {}", metadata_snapshot->getPrimaryKey().column_names.size()); } void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, const ContextPtr & context) const @@ -285,7 +283,6 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree /// Find min position in PK of any column that is used in this condition. cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); } - LOG_DEBUG(log, "node {}, min pos : {}", node.getASTNode()->dumpTree(), cond.min_position_in_primary_key); res.emplace_back(std::move(cond)); } From ec07032173b7b12d01e03b4570d05e4f1669c232 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 12 Aug 2023 01:33:19 +0200 Subject: [PATCH 0029/1097] fix test --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0936d092f99..98479368fae 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -132,7 +132,7 @@ class IColumn; M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ \ - M(Bool, allow_statistic_optimize, false, "use statistic to optimize queries", 0) \ + M(Bool, allow_statistic_optimize, false, "Allows using statistic to optimize queries", 0) \ \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ From 16275168cb515cab7b734cd56629f9d8f32961a6 Mon Sep 17 00:00:00 2001 From: root Date: Wed, 23 Aug 2023 16:13:05 +0800 Subject: [PATCH 0030/1097] [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 c29261e5b45b63e0df989f522a62fba5e535afde Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 24 Aug 2023 14:09:42 +0200 Subject: [PATCH 0031/1097] support estimate by t-digest --- src/AggregateFunctions/QuantileTDigest.h | 35 +++++++++++++-- src/Interpreters/MutationsInterpreter.cpp | 3 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 1 - src/Storages/Statistic/Statistic.cpp | 5 --- src/Storages/Statistic/Statistic.h | 4 +- src/Storages/Statistic/tests/gtest_stats.cpp | 44 +++++++++++++++++++ src/Storages/StatisticsDescription.cpp | 14 ++++-- src/Storages/StatisticsDescription.h | 4 +- 8 files changed, 90 insertions(+), 20 deletions(-) create mode 100644 src/Storages/Statistic/tests/gtest_stats.cpp diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index 8706f77c12d..58d19299a8e 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -337,12 +337,39 @@ public: Float64 getCountLessThan(Float64 value) const { + bool first = true; + Count sum = 0; + Count prev_count = 0; + Float64 prev_x = 0; + Value prev_mean = 0; - ///Count sum = 0; - ///Value prev_mean = centroids.front().mean; - ///Count prev_count = centroids.front().count; + for (const auto & c : centroids) + { + std::cerr << "c "<< c.mean << " "<< c.count << std::endl; + Float64 current_x = sum + c.count * 0.5; + if (c.mean >= value) + { + /// value is smaller than any value. + if (first) + return 0; - return value; + Float64 left = prev_x + 0.5 * (prev_count == 1); + Float64 right = current_x - 0.5 * (c.count == 1); + return checkOverflow(interpolate( + static_cast(value), + prev_mean, + static_cast(left), + c.mean, + static_cast(right))); + } + sum += c.count; + prev_mean = c.mean; + prev_count = c.count; + prev_x = current_x; + first = false; + } + /// count is larger than any value. + return count; } /** Calculates the quantile q [0, 1] based on the digest. diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 22105f063fa..86082816ff1 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -716,8 +716,7 @@ void MutationsInterpreter::prepare(bool dry_run) if (it == std::cend(statistics_desc)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown statistic: {}", command.statistic_name); - for (const auto & column : it->column_names) - dependencies.emplace(column, ColumnDependency::STATISTIC); + dependencies.emplace(it->column_name, ColumnDependency::STATISTIC); materialized_statistics.emplace(command.statistic_name); } else if (command.type == MutationCommand::MATERIALIZE_PROJECTION) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 8fffbdb5068..df1844c2a87 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -10,7 +10,6 @@ #include #include #include -#include "Common/logger_useful.h" #include #include #include diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index a8018458e5f..f2358c9b221 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -131,11 +131,6 @@ Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) StatisticPtr TDigestCreator(const StatisticDescription & stat) { - if (stat.column_names.size() != 1) - { - /// throw - } - /// TODO: check column data types. return StatisticPtr(new TDigestStatistic(stat)); } diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index 460dea382fc..b4d38a76a05 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -45,7 +45,7 @@ public: const String & columnName() const { - return statistics.column_names[0]; + return statistics.column_name; } /// const String& type() const = 0; /// virtual StatisticType statisticType() const = 0; @@ -93,7 +93,7 @@ public: void update(const Block & block) override { - const auto & column_with_type = block.getByName(statistics.column_names[0]); + const auto & column_with_type = block.getByName(statistics.column_name); size_t size = block.rows(); for (size_t i = 0; i < size; ++i) diff --git a/src/Storages/Statistic/tests/gtest_stats.cpp b/src/Storages/Statistic/tests/gtest_stats.cpp new file mode 100644 index 00000000000..48c2d2e1f5e --- /dev/null +++ b/src/Storages/Statistic/tests/gtest_stats.cpp @@ -0,0 +1,44 @@ +#include + +#include + +TEST(Statistic, TDigestLessThan) +{ + /// this is the simplest data which is continuous integeters. + /// so the estimated errors should be low. + + std::vector data; + data.reserve(100000); + for (int i = 0; i < 100000; i++) + data.push_back(i); + + auto test_less_than = [](const std::vector & data1, + const std::vector & v, + const std::vector & answers, + const std::vector & eps) + { + + DB::QuantileTDigest t_digest; + + for (int i = 0; i < data1.size(); i++) + t_digest.add(data1[i]); + t_digest.compress(); + + for (int i = 0; i < v.size(); i ++) + { + auto value = v[i]; + auto result = t_digest.getCountLessThan(value); + auto answer = answers[i]; + auto error = eps[i]; + ASSERT_LE(result, answer * (1 + error)); + ASSERT_GE(result, answer * (1 - error)); + } + }; + test_less_than(data, {-1, 1e9, 50000.0, 3000.0, 30.0}, {0, 100000, 50000, 3000, 30}, {0, 0, 0.001, 0.001, 0.001}); + + /// If we reversely construct the digest, the error is as bad as 5%. + std::reverse(data.begin(), data.end()); + test_less_than(data, {-1, 1e9, 50000.0, 3000.0, 30.0}, {0, 100000, 50000, 3000, 30}, {0, 0, 0.001, 0.001, 0.001}); + + +} diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 50c75995de1..5a3ba84d2cc 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -41,14 +41,18 @@ StatisticDescription StatisticDescription::getStatisticFromAST(const ASTPtr & de stat.type = Poco::toLower(stat_definition->type->name); ASTPtr expr_list = extractKeyExpressionList(stat_definition->columns->clone()); + if (expr_list->children.size() != 1) + { + throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistic must contain exactly one column"); + } for (const auto & ast : expr_list->children) { ASTIdentifier* ident = ast->as(); if (!ident || !columns.hasPhysical(ident->getColumnName())) throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column"); const auto & column = columns.get(ident->getColumnName()); - stat.column_names.push_back(column.name); - stat.data_types.push_back(column.type); + stat.column_name = column.name; + stat.data_type = column.type; } UNUSED(context); @@ -60,7 +64,8 @@ StatisticDescription::StatisticDescription(const StatisticDescription & other) : definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr) , name(other.name) , type(other.type) - , column_names(other.column_names) + , column_name(other.column_name) + , data_type(other.data_type) { } @@ -76,7 +81,8 @@ StatisticDescription & StatisticDescription::operator=(const StatisticDescriptio name = other.name; type = other.type; - column_names = other.column_names; + column_name = other.column_name; + data_type = other.data_type; return *this; } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 2cbce381990..531326eadaf 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -18,10 +18,10 @@ struct StatisticDescription String type; /// Names of statistic columns - Names column_names; + String column_name; /// Data types of statistic columns - DataTypes data_types; + DataTypePtr data_type; static StatisticDescription getStatisticFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context); From d80fd7c74cea624f71db788e08fe9171c1f80601 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 24 Aug 2023 14:20:34 +0200 Subject: [PATCH 0032/1097] small improvement --- src/Storages/Statistic/Statistic.cpp | 2 +- src/Storages/Statistic/Statistic.h | 7 +------ src/Storages/Statistic/tests/gtest_stats.cpp | 2 +- 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index f2358c9b221..b117ff6e603 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -123,7 +123,7 @@ Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) } else if (op == "greater" || op == "greaterThan") { - return estimator.estimateLess(val) / total_count; + return estimator.estimateGreater(val) / total_count; } else return default_unknown_cond_factor; diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index b4d38a76a05..7857c43bd16 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -47,6 +47,7 @@ public: { return statistics.column_name; } + /// const String& type() const = 0; /// virtual StatisticType statisticType() const = 0; @@ -69,12 +70,6 @@ public: { } - struct Range - { - Float64 left, right; - }; - - /// FIXME: implement correct count estimate method. Float64 estimateLess(Float64 val) const { return data.getCountLessThan(val); diff --git a/src/Storages/Statistic/tests/gtest_stats.cpp b/src/Storages/Statistic/tests/gtest_stats.cpp index 48c2d2e1f5e..b3070040c5c 100644 --- a/src/Storages/Statistic/tests/gtest_stats.cpp +++ b/src/Storages/Statistic/tests/gtest_stats.cpp @@ -22,6 +22,7 @@ TEST(Statistic, TDigestLessThan) for (int i = 0; i < data1.size(); i++) t_digest.add(data1[i]); + t_digest.compress(); for (int i = 0; i < v.size(); i ++) @@ -36,7 +37,6 @@ TEST(Statistic, TDigestLessThan) }; test_less_than(data, {-1, 1e9, 50000.0, 3000.0, 30.0}, {0, 100000, 50000, 3000, 30}, {0, 0, 0.001, 0.001, 0.001}); - /// If we reversely construct the digest, the error is as bad as 5%. std::reverse(data.begin(), data.end()); test_less_than(data, {-1, 1e9, 50000.0, 3000.0, 30.0}, {0, 100000, 50000, 3000, 30}, {0, 0, 0.001, 0.001, 0.001}); From 2330a28f7bd057b1f4fad13021d093a9e17de036 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 4 Sep 2023 15:51:00 +0200 Subject: [PATCH 0033/1097] support estimate by stat --- src/AggregateFunctions/QuantileTDigest.h | 5 +- src/Core/Settings.h | 1 + src/Interpreters/MutationsInterpreter.cpp | 8 +-- src/Parsers/ASTAlterQuery.cpp | 12 +---- src/Parsers/ASTAlterQuery.h | 1 - src/Parsers/ASTStatisticDeclaration.cpp | 16 +++--- src/Parsers/ASTStatisticDeclaration.h | 6 +-- src/Parsers/ParserAlterQuery.cpp | 14 ++---- src/Parsers/ParserCreateQuery.cpp | 15 ++---- src/Storages/AlterCommands.cpp | 49 +++++++------------ src/Storages/AlterCommands.h | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 26 ++++++++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 46 +++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 2 + .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 + src/Storages/MergeTree/MutateTask.cpp | 6 +-- src/Storages/MutationCommands.cpp | 3 +- src/Storages/MutationCommands.h | 2 +- src/Storages/Statistic/Statistic.cpp | 2 +- src/Storages/Statistic/Statistic.h | 41 ++++++++-------- src/Storages/StatisticsDescription.cpp | 41 +++++----------- src/Storages/StatisticsDescription.h | 4 +- .../0_stateless/02864_statistic_operate.sql | 47 ++++++++++++++++++ 24 files changed, 214 insertions(+), 141 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistic_operate.sql diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index 58d19299a8e..979c3f2af15 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -345,7 +345,7 @@ public: for (const auto & c : centroids) { - std::cerr << "c "<< c.mean << " "<< c.count << std::endl; + /// std::cerr << "c "<< c.mean << " "<< c.count << std::endl; Float64 current_x = sum + c.count * 0.5; if (c.mean >= value) { @@ -355,12 +355,13 @@ public: Float64 left = prev_x + 0.5 * (prev_count == 1); Float64 right = current_x - 0.5 * (c.count == 1); - return checkOverflow(interpolate( + Float64 result = checkOverflow(interpolate( static_cast(value), prev_mean, static_cast(left), c.mean, static_cast(right))); + return result; } sum += c.count; prev_mean = c.mean; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d27091b191e..f9312889265 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -134,6 +134,7 @@ class IColumn; M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ \ M(Bool, allow_statistic_optimize, false, "Allows using statistic to optimize queries", 0) \ + M(Bool, allow_experimental_statistic, false, "Allows using statistic", 0) \ \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index dc65f8a88d9..b9eb6ee9a96 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -728,13 +728,13 @@ void MutationsInterpreter::prepare(bool dry_run) std::cbegin(statistics_desc), std::end(statistics_desc), [&](const StatisticDescription & statistic) { - return statistic.name == command.statistic_name; + return statistic.column_name == command.statistic_column_name; }); if (it == std::cend(statistics_desc)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown statistic: {}", command.statistic_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown statistic column: {}", command.statistic_column_name); dependencies.emplace(it->column_name, ColumnDependency::STATISTIC); - materialized_statistics.emplace(command.statistic_name); + materialized_statistics.emplace(command.statistic_column_name); } else if (command.type == MutationCommand::MATERIALIZE_PROJECTION) { @@ -755,7 +755,7 @@ void MutationsInterpreter::prepare(bool dry_run) else if (command.type == MutationCommand::DROP_STATISTIC) { mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); - materialized_statistics.erase(command.statistic_name); + materialized_statistics.erase(command.statistic_column_name); } else if (command.type == MutationCommand::DROP_PROJECTION) { diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index e981392f97b..2db7bb93e8b 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -206,20 +206,12 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD STATISTIC " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : ""); statistic_decl->formatImpl(settings, state, frame); - - if (first) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : ""); - else if (statistic) /// AFTER - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : ""); - statistic->formatImpl(settings, state, frame); - } } else if (type == ASTAlterCommand::DROP_STATISTIC) { settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_statistic ? "CLEAR " : "DROP ") << "INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); - statistic->formatImpl(settings, state, frame); + statistic_decl->formatImpl(settings, state, frame); if (partition) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); @@ -229,7 +221,7 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & else if (type == ASTAlterCommand::MATERIALIZE_STATISTIC) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE STATISTIC " << (settings.hilite ? hilite_none : ""); - statistic->formatImpl(settings, state, frame); + statistic_decl->formatImpl(settings, state, frame); if (partition) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index d58797b13c8..6be9bcf34a3 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -134,7 +134,6 @@ public: ASTPtr projection; ASTPtr statistic_decl; - ASTPtr statistic; /** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE queries. * The value or ID of the partition is stored here. diff --git a/src/Parsers/ASTStatisticDeclaration.cpp b/src/Parsers/ASTStatisticDeclaration.cpp index 53b20b167b7..196eb994fed 100644 --- a/src/Parsers/ASTStatisticDeclaration.cpp +++ b/src/Parsers/ASTStatisticDeclaration.cpp @@ -12,23 +12,19 @@ ASTPtr ASTStatisticDeclaration::clone() const { auto res = std::make_shared(); - res->name = name; + res->column_name = column_name; + res->type = type; - if (columns) - res->set(res->columns, columns->clone()); - if (type) - res->set(res->type, type->clone()); - return std::move(res); + return res; } -void ASTStatisticDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const +void ASTStatisticDeclaration::formatImpl(const FormatSettings & s, FormatState &, FormatStateStacked) const { - s.ostr << backQuoteIfNeed(name); + s.ostr << backQuoteIfNeed(column_name); s.ostr << " "; - columns->formatImpl(s, state, frame); s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); - type->formatImpl(s, state, frame); + s.ostr << backQuoteIfNeed(type); } } diff --git a/src/Parsers/ASTStatisticDeclaration.h b/src/Parsers/ASTStatisticDeclaration.h index 0d5ab7723e9..7fba8872a94 100644 --- a/src/Parsers/ASTStatisticDeclaration.h +++ b/src/Parsers/ASTStatisticDeclaration.h @@ -12,9 +12,9 @@ class ASTFunction; class ASTStatisticDeclaration : public IAST { public: - String name; - IAST * columns; - ASTFunction * type; + String column_name; + /// We do not support to set bucket number for tdigest + String type; /** Get the text that identifies this element. */ String getID(char) const override { return "Stat"; } diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index bb94d98d587..0051136fa1f 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -342,21 +342,13 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->type = ASTAlterCommand::ADD_STATISTIC; - - if (s_first.ignore(pos, expected)) - command->first = true; - else if (s_after.ignore(pos, expected)) - { - if (!parser_name.parse(pos, command->statistic, expected)) - return false; - } } else if (s_drop_statistic.ignore(pos, expected)) { if (s_if_exists.ignore(pos, expected)) command->if_exists = true; - if (!parser_name.parse(pos, command->statistic, expected)) + if (!parser_stat_decl.parse(pos, command->statistic_decl, expected)) return false; command->type = ASTAlterCommand::DROP_STATISTIC; @@ -367,7 +359,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (s_if_exists.ignore(pos, expected)) command->if_exists = true; - if (!parser_name.parse(pos, command->statistic, expected)) + if (!parser_stat_decl.parse(pos, command->statistic_decl, expected)) return false; command->type = ASTAlterCommand::DROP_STATISTIC; @@ -385,7 +377,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (s_if_exists.ignore(pos, expected)) command->if_exists = true; - if (!parser_name.parse(pos, command->statistic, expected)) + if (!parser_stat_decl.parse(pos, command->statistic_decl, expected)) return false; command->type = ASTAlterCommand::MATERIALIZE_STATISTIC; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index a39190f274b..065ba99b4c8 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -165,29 +165,24 @@ bool ParserStatisticDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserKeyword s_type("TYPE"); ParserIdentifier name_p; - ParserDataType data_type_p; - ParserExpression expression_p; + ParserIdentifier type_p; ASTPtr name; - ASTPtr columns; + ASTPtr column; ASTPtr type; if (!name_p.parse(pos, name, expected)) return false; - if (!expression_p.parse(pos, columns, expected)) - return false; - if (!s_type.ignore(pos, expected)) return false; - if (!data_type_p.parse(pos, type, expected)) + if (!type_p.parse(pos, type, expected)) return false; auto stat = std::make_shared(); - stat->name = name->as().name(); - stat->set(stat->columns, columns); - stat->set(stat->type, type); + stat->column_name = name->as().name(); + stat->type = type->as().name(); node = stat; return true; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 56de8048339..003e39a738a 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -242,13 +242,10 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ const auto & ast_stat_decl = command_ast->statistic_decl->as(); - command.statistic_name = ast_stat_decl.name; - - if (command_ast->statistic) - command.after_statistic_name = command_ast->statistic->as().name(); + command.statistic_column_name = ast_stat_decl.column_name; + command.statistic_type = ast_stat_decl.type; command.if_not_exists = command_ast->if_not_exists; - command.first = command_ast->first; return command; } @@ -316,7 +313,10 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ AlterCommand command; command.ast = command_ast->clone(); command.type = AlterCommand::DROP_STATISTIC; - command.statistic_name = command_ast->statistic->as().name(); + const auto & ast_stat_decl = command_ast->statistic_decl->as(); + + command.statistic_column_name = ast_stat_decl.column_name; + command.statistic_type = ast_stat_decl.type; command.if_exists = command_ast->if_exists; command.clear = command_ast->clear_statistic; @@ -589,18 +589,15 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } else if (type == ADD_STATISTIC) { - if (std::any_of( + if (!if_not_exists && std::any_of( metadata.statistics.cbegin(), metadata.statistics.cend(), [this](const auto & statistic) { - return statistic.name == statistic_name; + return statistic.column_name == statistic_column_name && statistic.type == statistic_type; })) { - if (if_not_exists) - return; - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add statistic {} : statistic with this name already exists", statistic_name); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add statistic {} with type {}: statistic on this column with this type already exists", statistic_column_name, statistic_type); } auto insert_it = metadata.statistics.end(); @@ -609,22 +606,6 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) if (first) insert_it = metadata.statistics.begin(); - if (!after_statistic_name.empty()) - { - insert_it = std::find_if( - metadata.statistics.begin(), - metadata.statistics.end(), - [this](const auto & statistic) - { - return statistic.name == after_statistic_name; - }); - - if (insert_it == metadata.statistics.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong statistic name. Cannot find statistic {} to insert after", backQuote(after_statistic_name)); - - ++insert_it; - } - metadata.statistics.emplace(insert_it, StatisticDescription::getStatisticFromAST(statistic_decl, metadata.columns, context)); } else if (type == DROP_STATISTIC) @@ -636,14 +617,14 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) metadata.statistics.end(), [this](const auto & statistic) { - return statistic.name == statistic_name; + return statistic.column_name == statistic_column_name && statistic.type == statistic_type; }); if (erase_it == metadata.statistics.end()) { if (if_exists) return; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong statistic name. Cannot find statistic {} to drop", backQuote(statistic_name)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong statistic name. Cannot find statistic {} with type {} to drop", backQuote(statistic_column_name), statistic_type); } metadata.statistics.erase(erase_it); @@ -976,7 +957,15 @@ std::optional AlterCommand::tryConvertToMutationCommand(Storage } else if (type == DROP_STATISTIC) { + result.type = MutationCommand::Type::DROP_STATISTIC; + result.column_name = statistic_column_name; + if (clear) + result.clear = true; + if (partition) + result.partition = partition; + + result.predicate = nullptr; } else if (type == DROP_PROJECTION) { diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index eae538815f0..03a6fcefe22 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -121,8 +121,8 @@ struct AlterCommand String projection_name; ASTPtr statistic_decl = nullptr; - String after_statistic_name; - String statistic_name; + String statistic_column_name; + String statistic_type; /// For MODIFY TTL ASTPtr ttl = nullptr; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 6564fc67767..f6d7faa7d73 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -635,6 +636,31 @@ String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(bool with_subc return *minimum_size_column; } +Statistics IMergeTreeDataPart::loadStatistics() const +{ + const auto & metadata_snaphost = storage.getInMemoryMetadata(); + + auto total_statistics = MergeTreeStatisticFactory::instance().getMany(metadata_snaphost.getStatistics()); + + Statistics result; + for (auto & stat : total_statistics) + { + String file_name = stat->getFileName() + STAT_FILE_SUFFIX; + String file_path = fs::path(getDataPartStorage().getRelativePath()) / file_name; + + if (!metadata_manager->exists(file_name)) + { + LOG_INFO(storage.log, "Cannot find stats file {}", file_path); + continue; + } + auto stat_file = metadata_manager->read(file_name); + CompressedReadBuffer compressed_buffer(*stat_file); + stat->deserialize(compressed_buffer); + result.push_back(stat); + } + return result; +} + void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency) { assertOnDisk(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 58609c77c41..2037cc23105 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -169,6 +169,8 @@ public: void remove(); + Statistics loadStatistics() const; + /// Initialize columns (from columns.txt if exists, or create from column files if not). /// Load various metadata into memory: checksums from checksums.txt, index if required, etc. void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 810e21562b1..bcb42791da0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -422,6 +422,52 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const return storage_policy; } +ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const +{ + auto parts = getDataPartsVectorForInternalUsage(); + + auto metadata_snapshot = getInMemoryMetadataPtr(); + if (parts.empty()) + { + return {}; + } + + ASTPtr expression_ast; + Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, true /* one_part */); + // + // Generate valid expressions for filtering + bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, local_context, virtual_columns_block, expression_ast); + + ConditionEstimator result; + PartitionPruner partition_pruner(metadata_snapshot, query_info, local_context, true /* strict */); + + if (partition_pruner.isUseless() && !valid) + { + /// Read all partitions. + for (const auto & part : parts) + { + auto stats = part->loadStatistics(); + /// TODO: We only have one stats file for every part. + for (const auto & stat : stats) + result.merge(part->info.getPartNameV1(), part->rows_count, stat); + } + } + else + { + for (const auto & part : parts) + { + if (!partition_pruner.canBePruned(*part)) + { + auto stats = part->loadStatistics(); + for (const auto & stat : stats) + result.merge(part->info.getPartNameV1(), part->rows_count, stat); + } + } + } + + return result; +} + bool MergeTreeData::supportsFinal() const { return merging_params.mode == MergingParams::Collapsing diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e4801cffa36..8e42ecaf28d 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -422,6 +422,8 @@ public: bool supportsPrewhere() const override { return true; } + ConditionEstimator getConditionEstimatorByPredicate(const SelectQueryInfo &, ContextPtr) const override; + bool supportsFinal() const override; bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index ad5eca02692..f5f07bdb598 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -273,6 +273,8 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree cond.good = cond.viable; cond.selectivity = estimator.estimateSelectivity(node); + + LOG_DEBUG(log, "Condition {} has selectivity {}", node.getASTNode()->dumpTree(), cond.selectivity); } if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index d6ba6c56349..103ac4b3501 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -459,7 +459,7 @@ static std::set getStatisticsToRecalculate(const StorageMetadataPt const auto & stats = metadata_snapshot->getStatistics(); for (const auto & stat_desc : stats) { - if (materialized_stats.contains(stat_desc.name)) + if (materialized_stats.contains(stat_desc.column_name)) { stats_to_recalc.insert(stats_factory.get(stat_desc)); } @@ -1358,13 +1358,13 @@ private: const auto & statistics = ctx->metadata_snapshot->getStatistics(); for (const auto & stat : statistics) { - if (ctx->materialized_statistics.contains(stat.name)) + if (ctx->materialized_statistics.contains(stat.column_name)) { stats.push_back(MergeTreeStatisticFactory::instance().get(stat)); } else { - auto prefix = fmt::format("{}{}.", STAT_FILE_PREFIX, stat.name); + auto prefix = fmt::format("{}{}.", STAT_FILE_PREFIX, stat.column_name); auto it = ctx->source_part->checksums.files.upper_bound(prefix); while (it != ctx->source_part->checksums.files.end()) { diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index b00dca95c56..4c0f0d80a69 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -76,7 +77,7 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.type = MATERIALIZE_STATISTIC; res.partition = command->partition; res.predicate = nullptr; - res.statistic_name = command->statistic->as().name(); + res.statistic_column_name = command->statistic_decl->as().column_name; return res; } else if (command->type == ASTAlterCommand::MATERIALIZE_PROJECTION) diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index c9fa59bc309..ab925850ddb 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -53,7 +53,7 @@ struct MutationCommand /// For MATERIALIZE INDEX and PROJECTION and STATISTIC String index_name; String projection_name; - String statistic_name; + String statistic_column_name; /// For MATERIALIZE INDEX, UPDATE and DELETE. ASTPtr partition; diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index b117ff6e603..4ea2951b63d 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -143,7 +143,7 @@ void MergeTreeStatisticFactory::registerCreator(const std::string & stat_type, C MergeTreeStatisticFactory::MergeTreeStatisticFactory() { - registerCreator("t_digest", TDigestCreator); + registerCreator("tdigest", TDigestCreator); ///registerCreator("cm_sketch", CMSketchCreator); } diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index 7857c43bd16..7db4594eb5f 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -13,7 +13,6 @@ #include /// this is for user-defined statistic. -/// For auto collected statisic, we can use 'auto_statistic_' constexpr auto STAT_FILE_PREFIX = "statistic_"; constexpr auto STAT_FILE_SUFFIX = ".stat"; @@ -28,37 +27,38 @@ class IStatistic { public: explicit IStatistic(const StatisticDescription & stat_) - : statistics(stat_) + : stat(stat_) { } virtual ~IStatistic() = default; + /// statistic_[col_name]_[type] String getFileName() const { - return STAT_FILE_PREFIX + name(); - } - - const String & name() const - { - return statistics.name; + return STAT_FILE_PREFIX + columnName(); } const String & columnName() const { - return statistics.column_name; + return stat.column_name; } - /// const String& type() const = 0; - /// virtual StatisticType statisticType() const = 0; + const String & type() const + { + return stat.type; + } virtual void serialize(WriteBuffer & buf) = 0; + virtual void deserialize(ReadBuffer & buf) = 0; + virtual void update(const Block & block) = 0; + virtual UInt64 count() = 0; protected: - const StatisticDescription & statistics; + const StatisticDescription & stat; }; @@ -66,7 +66,7 @@ class TDigestStatistic : public IStatistic { QuantileTDigest data; public: - explicit TDigestStatistic(const StatisticDescription & stat) : IStatistic(stat) + explicit TDigestStatistic(const StatisticDescription & stat_) : IStatistic(stat_) { } @@ -88,7 +88,7 @@ public: void update(const Block & block) override { - const auto & column_with_type = block.getByName(statistics.column_name); + const auto & column_with_type = block.getByName(columnName()); size_t size = block.rows(); for (size_t i = 0; i < size; ++i) @@ -141,11 +141,11 @@ private: /// This is used to assume that condition is likely to have good selectivity. static constexpr auto threshold = 2; - UInt64 total_count; + UInt64 total_count = 0; struct PartColumnEstimator { - UInt64 part_count; + UInt64 part_count = 0; std::shared_ptr t_digest; @@ -184,6 +184,7 @@ private: { estimators[part_name].merge(statistic); } + Float64 estimateLess(Float64 val) const { if (estimators.empty()) @@ -210,18 +211,18 @@ private: std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const; public: - ConditionEstimator() = default; /// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ... /// Right now we only support simple condition like col = val / col < val Float64 estimateSelectivity(const RPNBuilderTreeNode & node) const; - void merge(std::string part_name, StatisticPtr statistic) + void merge(std::string part_name, UInt64 part_count, StatisticPtr statistic) { - column_estimators[statistic->columnName()].merge(part_name, statistic); + total_count += part_count; + if (statistic != nullptr) + column_estimators[statistic->columnName()].merge(part_name, statistic); } - }; diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 5a3ba84d2cc..2dd8d7ab8e6 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -25,35 +25,20 @@ StatisticDescription StatisticDescription::getStatisticFromAST(const ASTPtr & de if (!stat_definition) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create statistic from non ASTStatisticDeclaration AST"); - if (stat_definition->name.empty()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistic must have name in definition."); - - // type == nullptr => auto - if (!stat_definition->type) - throw Exception(ErrorCodes::INCORRECT_QUERY, "TYPE is required for statistics"); - - if (stat_definition->type->parameters && !stat_definition->type->parameters->children.empty()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistics type cannot have parameters"); - StatisticDescription stat; stat.definition_ast = definition_ast->clone(); - stat.name = stat_definition->name; - stat.type = Poco::toLower(stat_definition->type->name); + stat.type = Poco::toLower(stat_definition->type); + if (stat.type != "tdigest") + throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect type name {}", stat.type); + String column_name = stat_definition->column_name; - ASTPtr expr_list = extractKeyExpressionList(stat_definition->columns->clone()); - if (expr_list->children.size() != 1) - { - throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistic must contain exactly one column"); - } - for (const auto & ast : expr_list->children) - { - ASTIdentifier* ident = ast->as(); - if (!ident || !columns.hasPhysical(ident->getColumnName())) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column"); - const auto & column = columns.get(ident->getColumnName()); - stat.column_name = column.name; - stat.data_type = column.type; - } + if (!columns.hasPhysical(column_name)) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", column_name); + + const auto & column = columns.getPhysical(column_name); + stat.column_name = column.name; + /// TODO: check if it is numeric. + stat.data_type = column.type; UNUSED(context); @@ -62,7 +47,6 @@ StatisticDescription StatisticDescription::getStatisticFromAST(const ASTPtr & de StatisticDescription::StatisticDescription(const StatisticDescription & other) : definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr) - , name(other.name) , type(other.type) , column_name(other.column_name) , data_type(other.data_type) @@ -79,7 +63,6 @@ StatisticDescription & StatisticDescription::operator=(const StatisticDescriptio else definition_ast.reset(); - name = other.name; type = other.type; column_name = other.column_name; data_type = other.data_type; @@ -91,7 +74,7 @@ StatisticDescription & StatisticDescription::operator=(const StatisticDescriptio bool StatisticsDescriptions::has(const String & name) const { for (const auto & statistic : *this) - if (statistic.name == name) + if (statistic.column_name == name) return true; return false; } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 531326eadaf..ea05260b4d8 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -12,9 +12,7 @@ struct StatisticDescription /// Definition AST of statistic ASTPtr definition_ast; - /// Statistic name - String name; - + /// the type of statistic, right now it's only tdigest. String type; /// Names of statistic columns diff --git a/tests/queries/0_stateless/02864_statistic_operate.sql b/tests/queries/0_stateless/02864_statistic_operate.sql new file mode 100644 index 00000000000..5358fd7dde9 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistic_operate.sql @@ -0,0 +1,47 @@ +DROP TABLE IF EXISTS t1; + +SET allow_experimental_statistic = 1; +SET allow_statistic_optimize = 1; + +CREATE TABLE t1 +( + a Int64, + b Float64, + pk String, + STATISTIC a TYPE tdigest, + STATISTIC b TYPE tdigest +) Engine = MergeTree() ORDER BY pk; + +SHOW CREATE TABLE t1; + +INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'After insert'; +EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; +SELECT count(*) FROM t1 WHERE b < 10 and a < 10; + +ALTER TABLE t1 DROP STATISTIC a TYPE tdigest; +ALTER TABLE t1 DROP STATISTIC b TYPE tdigest; + +SELECT 'After drop statistic'; +EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; +SELECT count(*) FROM t1 WHERE b < 10 and a < 10; + +ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; +ALTER TABLE t1 ADD STATISTIC b TYPE tdigest; + +ALTER TABLE t1 MATERIALIZE STATISTIC a TYPE tdigest; +ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; +INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'After materialize statistic'; +EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; +SELECT count(*) FROM t1 WHERE b < 10 and a < 10; + +OPTIMIZE TABLE t1 FINAL; + +SELECT 'After merge'; +EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; +SELECT count(*) FROM t1 WHERE b < 10 and a < 10; + +DROP TABLE IF EXISTS t1; From e1bc6cb0a702cf8f7aed19fa3a78d221ae6c6702 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 5 Sep 2023 15:24:21 -0300 Subject: [PATCH 0034/1097] 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 ddcb64f39f17c74885e0a00a4a5f732b6af6c7b7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 8 Sep 2023 02:27:17 +0200 Subject: [PATCH 0035/1097] update docs and refine statements --- .../mergetree-family/mergetree.md | 24 ++++- src/Interpreters/InterpreterCreateQuery.cpp | 10 +- src/Interpreters/MutationsInterpreter.cpp | 26 +++-- src/Parsers/ASTStatisticDeclaration.cpp | 21 +++- src/Parsers/ASTStatisticDeclaration.h | 6 +- src/Parsers/ParserCreateQuery.cpp | 9 +- src/Storages/AlterCommands.cpp | 67 ++++++------ src/Storages/AlterCommands.h | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 8 +- src/Storages/MutationCommands.cpp | 6 +- src/Storages/MutationCommands.h | 2 +- src/Storages/Statistic/Statistic.cpp | 15 +-- src/Storages/Statistic/Statistic.h | 12 +-- src/Storages/StatisticsDescription.cpp | 100 ++++++++---------- src/Storages/StatisticsDescription.h | 22 ++-- .../02864_statistic_operate.reference | 22 ++++ .../0_stateless/02864_statistic_operate.sql | 18 ++-- 17 files changed, 209 insertions(+), 161 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistic_operate.reference diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 4f506126682..afccce2ed5a 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -44,7 +44,10 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2], ... PROJECTION projection_name_1 (SELECT [GROUP BY] [ORDER BY]), - PROJECTION projection_name_2 (SELECT [GROUP BY] [ORDER BY]) + PROJECTION projection_name_2 (SELECT [GROUP BY] [ORDER BY]), + ... + STATISTIC TYPE type1, + STATISTIC TYPE type2 ) ENGINE = MergeTree() ORDER BY expr [PARTITION BY expr] @@ -1353,3 +1356,22 @@ In this sample configuration: - `_part_uuid` — Unique part identifier (if enabled MergeTree setting `assign_part_uuids`). - `_partition_value` — Values (a tuple) of a `partition by` expression. - `_sample_factor` — Sample factor (from the query). + +## Column Statistics (Experimental) {#column-statistics} + +The statistic declaration is in the columns section of the `CREATE` query. + +``` sql +STATISTIC TYPE type +``` + +For tables from the `*MergeTree` family, statistics can be specified. + +These lightweight statistics aggregate information about distribution of values in columns. +They can be used for query optimization (At current time they are used for moving expressions to PREWHERE). + +#### Available Types of Column Statistics {#available-types-of-column-statistics} + +- `tdigest` + + Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 9ddb8d83963..cf67b6c9231 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -459,8 +459,8 @@ ASTPtr InterpreterCreateQuery::formatStatistics(const StatisticsDescriptions & s { auto res = std::make_shared(); - for (const auto & statistic : statistics) - res->children.push_back(statistic.definition_ast->clone()); + for (const auto & definition_ast : statistics.definition_asts) + res->children.push_back(definition_ast->clone()); return res; } @@ -721,8 +721,10 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } if (create.columns_list->stats) for (const auto & statistic : create.columns_list->stats->children) - properties.stats.push_back( - StatisticDescription::getStatisticFromAST(statistic->clone(), properties.columns, getContext())); + { + auto stats = StatisticsDescriptions::getStatisticsFromAST(statistic->clone(), properties.columns, getContext()); + properties.stats.merge(stats); + } if (create.columns_list->projections) for (const auto & projection_ast : create.columns_list->projections->children) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index b9eb6ee9a96..961a4f7ac72 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -724,17 +724,20 @@ void MutationsInterpreter::prepare(bool dry_run) else if (command.type == MutationCommand::MATERIALIZE_STATISTIC) { mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); - auto it = std::find_if( - std::cbegin(statistics_desc), std::end(statistics_desc), - [&](const StatisticDescription & statistic) - { - return statistic.column_name == command.statistic_column_name; - }); - if (it == std::cend(statistics_desc)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown statistic column: {}", command.statistic_column_name); + for (const auto & stat_column_name: command.statistic_columns) + { + auto it = std::find_if( + std::cbegin(statistics_desc), std::end(statistics_desc), + [&](const StatisticDescription & statistic) + { + return statistic.column_name == stat_column_name; + }); + if (it == std::cend(statistics_desc)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown statistic column: {}", stat_column_name); - dependencies.emplace(it->column_name, ColumnDependency::STATISTIC); - materialized_statistics.emplace(command.statistic_column_name); + dependencies.emplace(it->column_name, ColumnDependency::STATISTIC); + materialized_statistics.emplace(stat_column_name); + } } else if (command.type == MutationCommand::MATERIALIZE_PROJECTION) { @@ -755,7 +758,8 @@ void MutationsInterpreter::prepare(bool dry_run) else if (command.type == MutationCommand::DROP_STATISTIC) { mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); - materialized_statistics.erase(command.statistic_column_name); + for (const auto & stat_column_name: command.statistic_columns) + materialized_statistics.erase(stat_column_name); } else if (command.type == MutationCommand::DROP_PROJECTION) { diff --git a/src/Parsers/ASTStatisticDeclaration.cpp b/src/Parsers/ASTStatisticDeclaration.cpp index 196eb994fed..0e20b020ab3 100644 --- a/src/Parsers/ASTStatisticDeclaration.cpp +++ b/src/Parsers/ASTStatisticDeclaration.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -12,17 +13,27 @@ ASTPtr ASTStatisticDeclaration::clone() const { auto res = std::make_shared(); - res->column_name = column_name; + res->set(res->columns, columns->clone()); res->type = type; return res; } - -void ASTStatisticDeclaration::formatImpl(const FormatSettings & s, FormatState &, FormatStateStacked) const +std::vector ASTStatisticDeclaration::getColumnNames() const { - s.ostr << backQuoteIfNeed(column_name); - s.ostr << " "; + std::vector result; + result.reserve(columns->children.size()); + for (const ASTPtr & column_ast : columns->children) + { + result.push_back(column_ast->as().name()); + } + return result; + +} + +void ASTStatisticDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const +{ + columns->formatImpl(s, state, frame); s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); s.ostr << backQuoteIfNeed(type); } diff --git a/src/Parsers/ASTStatisticDeclaration.h b/src/Parsers/ASTStatisticDeclaration.h index 7fba8872a94..f936c93f2ba 100644 --- a/src/Parsers/ASTStatisticDeclaration.h +++ b/src/Parsers/ASTStatisticDeclaration.h @@ -12,13 +12,15 @@ class ASTFunction; class ASTStatisticDeclaration : public IAST { public: - String column_name; - /// We do not support to set bucket number for tdigest + IAST * columns; + /// TODO type should be a list of ASTFunction, for example, 'tdigest(256), hyperloglog(128)', etc. String type; /** Get the text that identifies this element. */ String getID(char) const override { return "Stat"; } + std::vector getColumnNames() const; + ASTPtr clone() const override; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; }; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 3ac12212054..344d00beb4e 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -165,14 +165,13 @@ bool ParserStatisticDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & { ParserKeyword s_type("TYPE"); - ParserIdentifier name_p; + ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma), false); ParserIdentifier type_p; - ASTPtr name; - ASTPtr column; + ASTPtr columns; ASTPtr type; - if (!name_p.parse(pos, name, expected)) + if (!columns_p.parse(pos, columns, expected)) return false; if (!s_type.ignore(pos, expected)) @@ -182,7 +181,7 @@ bool ParserStatisticDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; auto stat = std::make_shared(); - stat->column_name = name->as().name(); + stat->set(stat->columns, columns); stat->type = type->as().name(); node = stat; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 003e39a738a..c1f7711fce7 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -242,9 +242,8 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ const auto & ast_stat_decl = command_ast->statistic_decl->as(); - command.statistic_column_name = ast_stat_decl.column_name; + command.statistic_columns = ast_stat_decl.getColumnNames(); command.statistic_type = ast_stat_decl.type; - command.if_not_exists = command_ast->if_not_exists; return command; @@ -315,7 +314,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.type = AlterCommand::DROP_STATISTIC; const auto & ast_stat_decl = command_ast->statistic_decl->as(); - command.statistic_column_name = ast_stat_decl.column_name; + command.statistic_columns = ast_stat_decl.getColumnNames(); command.statistic_type = ast_stat_decl.type; command.if_exists = command_ast->if_exists; command.clear = command_ast->clear_statistic; @@ -589,45 +588,47 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } else if (type == ADD_STATISTIC) { - if (!if_not_exists && std::any_of( - metadata.statistics.cbegin(), - metadata.statistics.cend(), - [this](const auto & statistic) - { - return statistic.column_name == statistic_column_name && statistic.type == statistic_type; - })) + /// TODO: Right now we assume there is only one type of statistics for simple implement. + for (const auto & statistic_column_name : statistic_columns) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add statistic {} with type {}: statistic on this column with this type already exists", statistic_column_name, statistic_type); + if (!if_not_exists && std::any_of( + metadata.statistics.cbegin(), + metadata.statistics.cend(), + [&](const auto & statistic) + { + return statistic.column_name == statistic_column_name; + })) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add statistic {} with type {}: statistic on this column with this type already exists", statistic_column_name, statistic_type); + } } - auto insert_it = metadata.statistics.end(); - - /// insert the index in the beginning of the indices list - if (first) - insert_it = metadata.statistics.begin(); - - metadata.statistics.emplace(insert_it, StatisticDescription::getStatisticFromAST(statistic_decl, metadata.columns, context)); + auto stats = StatisticsDescriptions::getStatisticsFromAST(statistic_decl, metadata.columns, context); + metadata.statistics.merge(stats); } else if (type == DROP_STATISTIC) { if (!partition && !clear) { - auto erase_it = std::find_if( - metadata.statistics.begin(), - metadata.statistics.end(), - [this](const auto & statistic) - { - return statistic.column_name == statistic_column_name && statistic.type == statistic_type; - }); - - if (erase_it == metadata.statistics.end()) + for (const auto & stat_column_name : statistic_columns) { - if (if_exists) - return; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong statistic name. Cannot find statistic {} with type {} to drop", backQuote(statistic_column_name), statistic_type); - } + auto erase_it = std::find_if( + metadata.statistics.begin(), + metadata.statistics.end(), + [stat_column_name](const auto & statistic) + { + return statistic.column_name == stat_column_name; + }); - metadata.statistics.erase(erase_it); + if (erase_it == metadata.statistics.end()) + { + if (if_exists) + return; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong statistic name. Cannot find statistic {} with type {} to drop", backQuote(stat_column_name), statistic_type); + } + LOG_INFO(&Poco::Logger::get("drop_stat"), "dropping statistic {}", erase_it->column_name); + metadata.statistics.erase(erase_it); + } } } else if (type == ADD_CONSTRAINT) @@ -958,7 +959,7 @@ std::optional AlterCommand::tryConvertToMutationCommand(Storage else if (type == DROP_STATISTIC) { result.type = MutationCommand::Type::DROP_STATISTIC; - result.column_name = statistic_column_name; + result.statistic_columns = statistic_columns; if (clear) result.clear = true; diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 03a6fcefe22..f5b7c1c0063 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -121,7 +121,7 @@ struct AlterCommand String projection_name; ASTPtr statistic_decl = nullptr; - String statistic_column_name; + std::vector statistic_columns; String statistic_type; /// For MODIFY TTL diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index b93604bcac6..2093f667fcb 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -580,9 +580,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, columns, context)); if (args.query.columns_list && args.query.columns_list->stats) - for (const auto & stat : args.query.columns_list->stats->children) - metadata.statistics.push_back( - StatisticDescription::getStatisticFromAST(stat, columns, args.getContext())); + for (const auto & stat_ast : args.query.columns_list->stats->children) + { + auto stats = StatisticsDescriptions::getStatisticsFromAST(stat_ast, columns, args.getContext()); + metadata.statistics.merge(stats); + } if (args.query.columns_list && args.query.columns_list->projections) for (auto & projection_ast : args.query.columns_list->projections->children) diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 4c0f0d80a69..46322f3ef05 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -77,7 +77,11 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.type = MATERIALIZE_STATISTIC; res.partition = command->partition; res.predicate = nullptr; - res.statistic_column_name = command->statistic_decl->as().column_name; + for (const ASTPtr & column_ast : command->statistic_decl->as().columns->children) + { + const auto & column = column_ast->as().getColumnName(); + res.statistic_columns.push_back(column); + } return res; } else if (command->type == ASTAlterCommand::MATERIALIZE_PROJECTION) diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index 2600112f744..014a227dff3 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -53,7 +53,7 @@ struct MutationCommand /// For MATERIALIZE INDEX and PROJECTION and STATISTIC String index_name = {}; String projection_name = {}; - String statistic_column_name = {}; + std::vector statistic_columns = {}; /// For MATERIALIZE INDEX, UPDATE and DELETE. ASTPtr partition = {}; diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index 4ea2951b63d..17aa6a76f1c 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -135,7 +135,7 @@ StatisticPtr TDigestCreator(const StatisticDescription & stat) return StatisticPtr(new TDigestStatistic(stat)); } -void MergeTreeStatisticFactory::registerCreator(const std::string & stat_type, Creator creator) +void MergeTreeStatisticFactory::registerCreator(StatisticType stat_type, Creator creator) { if (!creators.emplace(stat_type, std::move(creator)).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticFactory: the statistic creator type {} is not unique", stat_type); @@ -143,7 +143,7 @@ void MergeTreeStatisticFactory::registerCreator(const std::string & stat_type, C MergeTreeStatisticFactory::MergeTreeStatisticFactory() { - registerCreator("tdigest", TDigestCreator); + registerCreator(TDigest, TDigestCreator); ///registerCreator("cm_sketch", CMSketchCreator); } @@ -160,16 +160,7 @@ StatisticPtr MergeTreeStatisticFactory::get(const StatisticDescription & stat) c if (it == creators.end()) { throw Exception(ErrorCodes::INCORRECT_QUERY, - "Unknown Statistic type '{}'. Available types: {}", stat.type, - std::accumulate(creators.cbegin(), creators.cend(), std::string{}, - [] (auto && left, const auto & right) -> std::string - { - if (left.empty()) - return right.first; - else - return left + ", " + right.first; - }) - ); + "Unknown Statistic type '{}'. Available types: tdigest", stat.type); } return std::make_shared(stat); } diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index 7db4594eb5f..eb05649f0a6 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -43,10 +43,10 @@ public: return stat.column_name; } - const String & type() const - { - return stat.type; - } + //const String & type() const + //{ + // return stat.type; + //} virtual void serialize(WriteBuffer & buf) = 0; @@ -118,13 +118,13 @@ public: Statistics getMany(const std::vector & stats) const; - void registerCreator(const std::string & type, Creator creator); + void registerCreator(StatisticType type, Creator creator); protected: MergeTreeStatisticFactory(); private: - using Creators = std::unordered_map; + using Creators = std::unordered_map; Creators creators; }; diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 2dd8d7ab8e6..0fff9581d57 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -10,6 +10,8 @@ #include #include +#include + namespace DB { @@ -19,58 +21,51 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; }; -StatisticDescription StatisticDescription::getStatisticFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context) +StatisticType StatisticDescription::stringToType(String type) +{ + if (type.empty()) + return TDigest; + if (type == "tdigest") + return TDigest; + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}", type); +} + +StatisticsDescriptions StatisticsDescriptions::getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context) { const auto * stat_definition = definition_ast->as(); if (!stat_definition) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create statistic from non ASTStatisticDeclaration AST"); - StatisticDescription stat; - stat.definition_ast = definition_ast->clone(); - stat.type = Poco::toLower(stat_definition->type); - if (stat.type != "tdigest") - throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect type name {}", stat.type); - String column_name = stat_definition->column_name; + LOG_INFO(&Poco::Logger::get("stats_desc"), "stat_def is like {}", stat_definition->dumpTree()); - if (!columns.hasPhysical(column_name)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", column_name); + StatisticsDescriptions stats; + for (const auto & column_ast : stat_definition->columns->children) + { + StatisticDescription stat; + stat.type = StatisticDescription::stringToType(Poco::toLower(stat_definition->type)); + String column_name = column_ast->as().name(); - const auto & column = columns.getPhysical(column_name); - stat.column_name = column.name; - /// TODO: check if it is numeric. - stat.data_type = column.type; + if (!columns.hasPhysical(column_name)) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", column_name); + + const auto & column = columns.getPhysical(column_name); + stat.column_name = column.name; + /// TODO: check if it is numeric. + stat.data_type = column.type; + stats.push_back(stat); + } + stats.definition_asts.push_back(definition_ast); + + if (stats.empty()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistic column list"); + + LOG_INFO(&Poco::Logger::get("stats_desc"), "there are {} stats", stats.size()); UNUSED(context); - return stat; + return stats; } -StatisticDescription::StatisticDescription(const StatisticDescription & other) - : definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr) - , type(other.type) - , column_name(other.column_name) - , data_type(other.data_type) -{ -} - -StatisticDescription & StatisticDescription::operator=(const StatisticDescription & other) -{ - if (&other == this) - return *this; - - if (other.definition_ast) - definition_ast = other.definition_ast->clone(); - else - definition_ast.reset(); - - type = other.type; - column_name = other.column_name; - data_type = other.data_type; - - return *this; -} - - bool StatisticsDescriptions::has(const String & name) const { for (const auto & statistic : *this) @@ -79,31 +74,22 @@ bool StatisticsDescriptions::has(const String & name) const return false; } +void StatisticsDescriptions::merge(const StatisticsDescriptions & other) +{ + insert(end(), other.begin(), other.end()); + definition_asts.insert(definition_asts.end(), other.definition_asts.begin(), other.definition_asts.end()); +} + String StatisticsDescriptions::toString() const { if (empty()) return {}; ASTExpressionList list; - for (const auto & statistic : *this) - list.children.push_back(statistic.definition_ast); + for (const auto & ast : definition_asts) + list.children.push_back(ast); return serializeAST(list); } -StatisticsDescriptions StatisticsDescriptions::parse(const String & str, const ColumnsDescription & columns, ContextPtr context) -{ - StatisticsDescriptions result; - if (str.empty()) - return result; - - ParserStatisticDeclaration parser; - ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - - for (const auto & index : list->children) - result.emplace_back(StatisticDescription::getStatisticFromAST(index, columns, context)); - - return result; -} - } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index ea05260b4d8..b018ce26665 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -7,13 +7,15 @@ namespace DB { +enum StatisticType +{ + TDigest = 0, +}; + struct StatisticDescription { - /// Definition AST of statistic - ASTPtr definition_ast; - /// the type of statistic, right now it's only tdigest. - String type; + StatisticType type; /// Names of statistic columns String column_name; @@ -21,24 +23,22 @@ struct StatisticDescription /// Data types of statistic columns DataTypePtr data_type; - static StatisticDescription getStatisticFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context); - StatisticDescription() = default; - /// We need custom copy constructors because we don't want - /// unintentionaly share AST variables and modify them. - StatisticDescription(const StatisticDescription & other); - StatisticDescription & operator=(const StatisticDescription & other); + static StatisticType stringToType(String type); }; struct StatisticsDescriptions : public std::vector { + std::vector definition_asts; /// Stat with name exists bool has(const String & name) const; + /// merge with other Statistics + void merge(const StatisticsDescriptions & other); /// Convert description to string String toString() const; /// Parse description from string - static StatisticsDescriptions parse(const String & str, const ColumnsDescription & columns, ContextPtr context); + static StatisticsDescriptions getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context); }; } diff --git a/tests/queries/0_stateless/02864_statistic_operate.reference b/tests/queries/0_stateless/02864_statistic_operate.reference new file mode 100644 index 00000000000..424b16cb8a5 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistic_operate.reference @@ -0,0 +1,22 @@ +CREATE TABLE default.t1\n(\n `a` Int64,\n `b` Float64,\n `pk` String,\n STATISTIC a, b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +After insert +SELECT count() +FROM t1 +PREWHERE (a < 10) AND (b < 10) +10 +After drop statistic +SELECT count() +FROM t1 +PREWHERE (b < 10) AND (a < 10) +10 +After add statistic +After materialize statistic +SELECT count() +FROM t1 +PREWHERE (a < 10) AND (b < 10) +20 +After merge +SELECT count() +FROM t1 +PREWHERE (a < 10) AND (b < 10) +20 diff --git a/tests/queries/0_stateless/02864_statistic_operate.sql b/tests/queries/0_stateless/02864_statistic_operate.sql index 5358fd7dde9..6b74cc37e2a 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.sql +++ b/tests/queries/0_stateless/02864_statistic_operate.sql @@ -8,8 +8,7 @@ CREATE TABLE t1 a Int64, b Float64, pk String, - STATISTIC a TYPE tdigest, - STATISTIC b TYPE tdigest + STATISTIC a, b TYPE tdigest, ) Engine = MergeTree() ORDER BY pk; SHOW CREATE TABLE t1; @@ -20,18 +19,21 @@ SELECT 'After insert'; EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; SELECT count(*) FROM t1 WHERE b < 10 and a < 10; -ALTER TABLE t1 DROP STATISTIC a TYPE tdigest; -ALTER TABLE t1 DROP STATISTIC b TYPE tdigest; +ALTER TABLE t1 DROP STATISTIC a, b TYPE tdigest; SELECT 'After drop statistic'; EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; SELECT count(*) FROM t1 WHERE b < 10 and a < 10; -ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -ALTER TABLE t1 ADD STATISTIC b TYPE tdigest; +--SHOW CREATE TABLE t1; -ALTER TABLE t1 MATERIALIZE STATISTIC a TYPE tdigest; -ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; +ALTER TABLE t1 ADD STATISTIC a, b TYPE tdigest; + +SELECT 'After add statistic'; + +--SHOW CREATE TABLE t1; + +ALTER TABLE t1 MATERIALIZE STATISTIC a, b TYPE tdigest; INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; SELECT 'After materialize statistic'; From f9abf164414ff2c69a135d7ee18b4f8f9a6b0bdc Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 9 Sep 2023 04:04:18 +0200 Subject: [PATCH 0036/1097] fix fast tests --- src/Interpreters/InterpreterCreateQuery.cpp | 7 +-- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++ src/Storages/StatisticsDescription.cpp | 43 ++++++++++++++++--- src/Storages/StatisticsDescription.h | 2 + .../02864_statistic_operate.reference | 4 +- .../0_stateless/02864_statistic_operate.sql | 4 +- .../aspell-ignore/en/aspell-dict.txt | 1 + 7 files changed, 48 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index cf67b6c9231..349ac683db9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -457,12 +457,7 @@ ASTPtr InterpreterCreateQuery::formatIndices(const IndicesDescription & indices) ASTPtr InterpreterCreateQuery::formatStatistics(const StatisticsDescriptions & statistics) { - auto res = std::make_shared(); - - for (const auto & definition_ast : statistics.definition_asts) - res->children.push_back(definition_ast->clone()); - - return res; + return statistics.getAST(); } ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription & constraints) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 467a68b7190..c2512144fea 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -438,6 +438,9 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const { + if (!local_context->getSettings().allow_statistic_optimize) + return {}; + auto parts = getDataPartsVectorForInternalUsage(); auto metadata_snapshot = getInMemoryMetadataPtr(); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 0fff9581d57..097c2cd1ad5 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -30,6 +30,18 @@ StatisticType StatisticDescription::stringToType(String type) throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}", type); } +namespace +{ + +String typeToString(StatisticType type) +{ + if (type == TDigest) + return "tdigest"; + return "unknown"; +} + +} + StatisticsDescriptions StatisticsDescriptions::getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context) { const auto * stat_definition = definition_ast->as(); @@ -54,7 +66,7 @@ StatisticsDescriptions StatisticsDescriptions::getStatisticsFromAST(const ASTPtr stat.data_type = column.type; stats.push_back(stat); } - stats.definition_asts.push_back(definition_ast); + /// stats.definition_asts.push_back(definition_ast); if (stats.empty()) throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistic column list"); @@ -77,7 +89,28 @@ bool StatisticsDescriptions::has(const String & name) const void StatisticsDescriptions::merge(const StatisticsDescriptions & other) { insert(end(), other.begin(), other.end()); - definition_asts.insert(definition_asts.end(), other.definition_asts.begin(), other.definition_asts.end()); + /// definition_asts.insert(definition_asts.end(), other.definition_asts.begin(), other.definition_asts.end()); +} + +ASTPtr StatisticsDescriptions::getAST() const +{ + + auto list = std::make_shared(); + /// for (const auto & ast : definition_asts) + /// list.children.push_back(ast); + + for (const auto & stat : *this) + { + auto stat_ast = std::make_shared(); + auto cols_ast = std::make_shared(); + auto col_ast = std::make_shared(stat.column_name); + cols_ast->children.push_back(col_ast); + stat_ast->set(stat_ast->columns, cols_ast); + stat_ast->type = typeToString(stat.type); + + list->children.push_back(stat_ast); + } + return list; } String StatisticsDescriptions::toString() const @@ -85,11 +118,7 @@ String StatisticsDescriptions::toString() const if (empty()) return {}; - ASTExpressionList list; - for (const auto & ast : definition_asts) - list.children.push_back(ast); - - return serializeAST(list); + return serializeAST(*getAST()); } } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index b018ce26665..70a6c6bb34f 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -35,6 +35,8 @@ struct StatisticsDescriptions : public std::vector bool has(const String & name) const; /// merge with other Statistics void merge(const StatisticsDescriptions & other); + + ASTPtr getAST() const; /// Convert description to string String toString() const; /// Parse description from string diff --git a/tests/queries/0_stateless/02864_statistic_operate.reference b/tests/queries/0_stateless/02864_statistic_operate.reference index 424b16cb8a5..26be9b47b8e 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.reference +++ b/tests/queries/0_stateless/02864_statistic_operate.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.t1\n(\n `a` Int64,\n `b` Float64,\n `pk` String,\n STATISTIC a, b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Int64,\n `b` Float64,\n `pk` String,\n STATISTIC a TYPE tdigest,\n STATISTIC b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 After insert SELECT count() FROM t1 @@ -9,7 +9,9 @@ SELECT count() FROM t1 PREWHERE (b < 10) AND (a < 10) 10 +CREATE TABLE default.t1\n(\n `a` Int64,\n `b` Float64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 After add statistic +CREATE TABLE default.t1\n(\n `a` Int64,\n `b` Float64,\n `pk` String,\n STATISTIC a TYPE tdigest,\n STATISTIC b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 After materialize statistic SELECT count() FROM t1 diff --git a/tests/queries/0_stateless/02864_statistic_operate.sql b/tests/queries/0_stateless/02864_statistic_operate.sql index 6b74cc37e2a..c0c97e60dd2 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.sql +++ b/tests/queries/0_stateless/02864_statistic_operate.sql @@ -25,13 +25,13 @@ SELECT 'After drop statistic'; EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; SELECT count(*) FROM t1 WHERE b < 10 and a < 10; ---SHOW CREATE TABLE t1; +SHOW CREATE TABLE t1; ALTER TABLE t1 ADD STATISTIC a, b TYPE tdigest; SELECT 'After add statistic'; ---SHOW CREATE TABLE t1; +SHOW CREATE TABLE t1; ALTER TABLE t1 MATERIALIZE STATISTIC a, b TYPE tdigest; INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2c29fd9369e..79a75665552 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -853,6 +853,7 @@ SystemReplicasThreadsActive TABLUM TCPConnection TCPThreads +TDigest TINYINT TKSV TLSv From 57b5f3ca7842520910b47953e4ea6f740a82ce94 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 11 Sep 2023 07:24:36 +0200 Subject: [PATCH 0037/1097] fix --- src/Storages/Statistic/Statistic.cpp | 2 +- src/Storages/Statistic/Statistic.h | 4 ++-- src/Storages/StatisticsDescription.cpp | 3 --- src/Storages/StatisticsDescription.h | 1 - 4 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index 17aa6a76f1c..9b7ca3ab92d 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -165,7 +165,7 @@ StatisticPtr MergeTreeStatisticFactory::get(const StatisticDescription & stat) c return std::make_shared(stat); } -Statistics MergeTreeStatisticFactory::getMany(const std::vector & stats) const +Statistics MergeTreeStatisticFactory::getMany(const StatisticsDescriptions & stats) const { Statistics result; for (const auto & stat : stats) diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index eb05649f0a6..6653e073df4 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -58,7 +58,7 @@ public: protected: - const StatisticDescription & stat; + StatisticDescription stat; }; @@ -116,7 +116,7 @@ public: StatisticPtr get(const StatisticDescription & stat) const; - Statistics getMany(const std::vector & stats) const; + Statistics getMany(const StatisticsDescriptions & stats) const; void registerCreator(StatisticType type, Creator creator); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 097c2cd1ad5..3aee41f17e9 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -66,7 +66,6 @@ StatisticsDescriptions StatisticsDescriptions::getStatisticsFromAST(const ASTPtr stat.data_type = column.type; stats.push_back(stat); } - /// stats.definition_asts.push_back(definition_ast); if (stats.empty()) throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistic column list"); @@ -96,8 +95,6 @@ ASTPtr StatisticsDescriptions::getAST() const { auto list = std::make_shared(); - /// for (const auto & ast : definition_asts) - /// list.children.push_back(ast); for (const auto & stat : *this) { diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 70a6c6bb34f..2949cac3245 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -30,7 +30,6 @@ struct StatisticDescription struct StatisticsDescriptions : public std::vector { - std::vector definition_asts; /// Stat with name exists bool has(const String & name) const; /// merge with other Statistics From 430a4fda9cf03a304bace0c7f50dcdfa6afed52a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 12 Sep 2023 18:30:55 +0200 Subject: [PATCH 0038/1097] fix some tests --- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 3 ++- src/Storages/Statistic/Statistic.cpp | 14 ++++++++++++-- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index f5f07bdb598..59902a6a115 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -274,7 +274,8 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree cond.selectivity = estimator.estimateSelectivity(node); - LOG_DEBUG(log, "Condition {} has selectivity {}", node.getASTNode()->dumpTree(), cond.selectivity); + if (node.getASTNode() != nullptr) + LOG_DEBUG(log, "Condition {} has selectivity {}", node.getASTNode()->dumpTree(), cond.selectivity); } if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index 9b7ca3ab92d..7a909f599a9 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -104,11 +104,19 @@ Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) return default_unknown_cond_factor; } auto it = column_estimators.find(col.value()); + + /// If there the estimator of the column is not found or there are no data at all, + /// we use dummy estimation. + bool dummy = total_count == 0; ColumnEstimator estimator; if (it != column_estimators.end()) { estimator = it->second; } + else + { + dummy = true; + } auto [op, val] = extractBinaryOp(node, col.value()); if (op == "equals") { @@ -119,10 +127,14 @@ Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) } else if (op == "less" || op == "lessThan") { + if (dummy) + return default_normal_cond_factor; return estimator.estimateLess(val) / total_count; } else if (op == "greater" || op == "greaterThan") { + if (dummy) + return default_normal_cond_factor; return estimator.estimateGreater(val) / total_count; } else @@ -144,8 +156,6 @@ void MergeTreeStatisticFactory::registerCreator(StatisticType stat_type, Creator MergeTreeStatisticFactory::MergeTreeStatisticFactory() { registerCreator(TDigest, TDigestCreator); - - ///registerCreator("cm_sketch", CMSketchCreator); } MergeTreeStatisticFactory & MergeTreeStatisticFactory::instance() From 6970411c2e3f3d1e5899247cad39e3f7a64aee85 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 13 Sep 2023 08:26:25 +0200 Subject: [PATCH 0039/1097] fix wide part and fuzzer --- src/Storages/Statistic/Statistic.cpp | 2 ++ .../queries/0_stateless/02864_statistic_operate.reference | 7 ++++--- tests/queries/0_stateless/02864_statistic_operate.sql | 8 +++++--- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index 7a909f599a9..1bd593be5a5 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -93,6 +93,8 @@ std::pair ConditionEstimator::extractBinaryOp(const RPNBui value = output_value.get(); else if (type == Field::Types::Float64) value = output_value.get(); + else + return {}; return std::make_pair(function_name, value); } diff --git a/tests/queries/0_stateless/02864_statistic_operate.reference b/tests/queries/0_stateless/02864_statistic_operate.reference index 26be9b47b8e..67f9eb6de15 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.reference +++ b/tests/queries/0_stateless/02864_statistic_operate.reference @@ -1,17 +1,18 @@ -CREATE TABLE default.t1\n(\n `a` Int64,\n `b` Float64,\n `pk` String,\n STATISTIC a TYPE tdigest,\n STATISTIC b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String,\n STATISTIC a TYPE tdigest,\n STATISTIC b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After insert SELECT count() FROM t1 PREWHERE (a < 10) AND (b < 10) 10 +0 After drop statistic SELECT count() FROM t1 PREWHERE (b < 10) AND (a < 10) 10 -CREATE TABLE default.t1\n(\n `a` Int64,\n `b` Float64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After add statistic -CREATE TABLE default.t1\n(\n `a` Int64,\n `b` Float64,\n `pk` String,\n STATISTIC a TYPE tdigest,\n STATISTIC b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String,\n STATISTIC a TYPE tdigest,\n STATISTIC b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After materialize statistic SELECT count() FROM t1 diff --git a/tests/queries/0_stateless/02864_statistic_operate.sql b/tests/queries/0_stateless/02864_statistic_operate.sql index c0c97e60dd2..66fc4a9ec34 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.sql +++ b/tests/queries/0_stateless/02864_statistic_operate.sql @@ -5,11 +5,12 @@ SET allow_statistic_optimize = 1; CREATE TABLE t1 ( - a Int64, - b Float64, + a Float64, + b Int64, pk String, STATISTIC a, b TYPE tdigest, -) Engine = MergeTree() ORDER BY pk; +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; SHOW CREATE TABLE t1; @@ -18,6 +19,7 @@ INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMI SELECT 'After insert'; EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; SELECT count(*) FROM t1 WHERE b < 10 and a < 10; +SELECT count(*) FROM t1 WHERE b < NULL and a < '10'; ALTER TABLE t1 DROP STATISTIC a, b TYPE tdigest; From 83d1b44a3e71ab125ab7aa41cff3c0eac4c1391d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 19 Sep 2023 17:08:15 +0200 Subject: [PATCH 0040/1097] validate --- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/InterpreterAlterQuery.cpp | 5 +++ src/Interpreters/InterpreterCreateQuery.cpp | 6 +++ src/Storages/MergeTree/MergeTreeData.cpp | 11 +++++ src/Storages/Statistic/Statistic.cpp | 27 ++++++++++++ src/Storages/Statistic/Statistic.h | 15 ++++--- src/Storages/StatisticsDescription.cpp | 9 ++-- .../02864_statistic_exception.reference | 0 .../0_stateless/02864_statistic_exception.sql | 42 +++++++++++++++++++ 9 files changed, 108 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistic_exception.reference create mode 100644 tests/queries/0_stateless/02864_statistic_exception.sql diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f23685c37d1..4a808fd4c7f 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -585,6 +585,7 @@ M(700, USER_SESSION_LIMIT_EXCEEDED) \ M(701, CLUSTER_DOESNT_EXIST) \ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ + M(703, ILLEGAL_STATISTIC) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c0cde4566c1..6911b1c9cc8 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -151,6 +151,11 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong parameter type in ALTER query"); + if (getContext()->getSettings().allow_experimental_statistic || + command_ast->type == ASTAlterCommand::ADD_STATISTIC || + command_ast->type == ASTAlterCommand::DROP_STATISTIC || + command_ast->type == ASTAlterCommand::MATERIALIZE_STATISTIC) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Alter table with statistic is now disabled. Turn on allow_experimental_statistic"); } if (typeid_cast(database.get())) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 349ac683db9..5dcac7e98b2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -714,12 +714,18 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.indices.push_back(index_desc); } + if (create.columns_list->stats) + { + const auto & settings = getContext()->getSettingsRef(); + if (!settings.allow_experimental_statistic) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistic is now disabled. Turn on allow_experimental_statistic"); for (const auto & statistic : create.columns_list->stats->children) { auto stats = StatisticsDescriptions::getStatisticsFromAST(statistic->clone(), properties.columns, getContext()); properties.stats.merge(stats); } + } if (create.columns_list->projections) for (const auto & projection_ast : create.columns_list->projections->children) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c2512144fea..3de698cd81d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -651,6 +651,17 @@ void MergeTreeData::checkProperties( } } + if (!new_metadata.statistics.empty()) + { + for (const auto & stat : new_metadata.statistics) + { + auto column = all_columns.tryGetByName(stat.column_name); + if (!column.has_value()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "column {} is not found", stat.column_name); + MergeTreeStatisticFactory::instance().validate(stat, column->type); + } + } + checkKeyExpression(*new_sorting_key.expression, new_sorting_key.sample_block, "Sorting", allow_nullable_key); } diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index 1bd593be5a5..b72a1780772 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -13,6 +14,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int INCORRECT_QUERY; + extern const int ILLEGAL_STATISTIC; } @@ -149,15 +151,30 @@ StatisticPtr TDigestCreator(const StatisticDescription & stat) return StatisticPtr(new TDigestStatistic(stat)); } +void TDigestValidator(const StatisticDescription &, DataTypePtr data_type) +{ + data_type = removeNullable(data_type); + if (!data_type->isValueRepresentedByNumber()) + throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "TDigest does not support type {}", data_type->getName()); +} + void MergeTreeStatisticFactory::registerCreator(StatisticType stat_type, Creator creator) { if (!creators.emplace(stat_type, std::move(creator)).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticFactory: the statistic creator type {} is not unique", stat_type); } +void MergeTreeStatisticFactory::registerValidator(StatisticType stat_type, Validator validator) +{ + if (!validators.emplace(stat_type, std::move(validator)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticFactory: the statistic validator type {} is not unique", stat_type); + +} + MergeTreeStatisticFactory::MergeTreeStatisticFactory() { registerCreator(TDigest, TDigestCreator); + registerValidator(TDigest, TDigestValidator); } MergeTreeStatisticFactory & MergeTreeStatisticFactory::instance() @@ -166,6 +183,16 @@ MergeTreeStatisticFactory & MergeTreeStatisticFactory::instance() return instance; } +void MergeTreeStatisticFactory::validate(const StatisticDescription & stat, DataTypePtr data_type) const +{ + auto it = validators.find(stat.type); + if (it == validators.end()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown Statistic type '{}'", stat.type); + } + it->second(stat, data_type); +} + StatisticPtr MergeTreeStatisticFactory::get(const StatisticDescription & stat) const { auto it = creators.find(stat.type); diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index 6653e073df4..ec912dcc39c 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -23,6 +23,9 @@ class IStatistic; using StatisticPtr = std::shared_ptr; using Statistics = std::vector; +/// Statistic for a column +/// right now we support +/// - tdigest class IStatistic { public: @@ -43,11 +46,6 @@ public: return stat.column_name; } - //const String & type() const - //{ - // return stat.type; - //} - virtual void serialize(WriteBuffer & buf) = 0; virtual void deserialize(ReadBuffer & buf) = 0; @@ -112,20 +110,27 @@ class MergeTreeStatisticFactory : private boost::noncopyable public: static MergeTreeStatisticFactory & instance(); + void validate(const StatisticDescription & stat, DataTypePtr data_type) const; + using Creator = std::function; + using Validator = std::function; + StatisticPtr get(const StatisticDescription & stat) const; Statistics getMany(const StatisticsDescriptions & stats) const; void registerCreator(StatisticType type, Creator creator); + void registerValidator(StatisticType type, Validator validator); protected: MergeTreeStatisticFactory(); private: using Creators = std::unordered_map; + using Validators = std::unordered_map; Creators creators; + Validators validators; }; class RPNBuilderTreeNode; diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 3aee41f17e9..5d9ea6d670b 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -18,13 +18,12 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_QUERY; + extern const int ILLEGAL_STATISTIC; extern const int LOGICAL_ERROR; }; StatisticType StatisticDescription::stringToType(String type) { - if (type.empty()) - return TDigest; if (type == "tdigest") return TDigest; throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}", type); @@ -87,8 +86,12 @@ bool StatisticsDescriptions::has(const String & name) const void StatisticsDescriptions::merge(const StatisticsDescriptions & other) { + /// Check duplicate + for (const auto & old_stat : * this) + for (const auto & new_stat : other) + if (old_stat.column_name == new_stat.column_name) + throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Statistic column {} has existed", old_stat.column_name); insert(end(), other.begin(), other.end()); - /// definition_asts.insert(definition_asts.end(), other.definition_asts.begin(), other.definition_asts.end()); } ASTPtr StatisticsDescriptions::getAST() const diff --git a/tests/queries/0_stateless/02864_statistic_exception.reference b/tests/queries/0_stateless/02864_statistic_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02864_statistic_exception.sql b/tests/queries/0_stateless/02864_statistic_exception.sql new file mode 100644 index 00000000000..58b43768679 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistic_exception.sql @@ -0,0 +1,42 @@ +DROP TABLE IF EXISTS t1; + +CREATE TABLE t1 +( + a Float64, + b Int64, + pk String, + STATISTIC a, b TYPE tdigest, +) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } + +SET allow_experimental_statistic = 1; + +CREATE TABLE t1 +( + a Float64, + b Int64, + pk String, + STATISTIC a, a TYPE tdigest, +) Engine = MergeTree() ORDER BY pk; -- { serverError ILLEGAL_STATISTIC } + +CREATE TABLE t1 +( + a Float64, + b Int64, + pk String, + STATISTIC a, pk TYPE tdigest, +) Engine = MergeTree() ORDER BY pk; -- { serverError ILLEGAL_STATISTIC } + +CREATE TABLE t1 +( + a Float64, + b Int64, + pk String, +) Engine = MergeTree() ORDER BY pk; + +ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; +ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE t1 ADD STATISTIC pk tdigest; -- { serverError ILLEGAL_STATISTIC } +ALTER TABLE t1 DROP STATISTIC b TYPE tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; -- { serverError INCORRECT_QUERY } + +DROP TABLE t1; From 7883ae0d8cf60657a7a225f5d00bbe3655ca8a70 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 22 Sep 2023 13:22:35 +0800 Subject: [PATCH 0041/1097] 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 0042/1097] 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 0043/1097] 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 0044/1097] Fix queries in the test --- .../0_stateless/01568_window_functions_distributed.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.sql b/tests/queries/0_stateless/01568_window_functions_distributed.sql index 140b02bfa6d..ef8e9541b05 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.sql +++ b/tests/queries/0_stateless/01568_window_functions_distributed.sql @@ -13,11 +13,11 @@ create table t_01568 engine Memory as select intDiv(number, 3) p, modulo(number, 3) o, number from numbers(9); -select sum(number) over w, max(number) over w from t_01568 window w as (partition by p); +select sum(number) over w, max(number) over w from t_01568 window w as (partition by p) order by p; -select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by p; -select distinct sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select distinct sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by p; -- window functions + aggregation w/shards select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3); From b7cfc4d82d8949a174dc3c08390f65d0531c748a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 25 Sep 2023 15:34:00 +0000 Subject: [PATCH 0045/1097] 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 0046/1097] 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 0047/1097] 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 8cb9fe0154ddeaa206b6dd593e67b0c90fe1db75 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 26 Sep 2023 19:16:01 +0200 Subject: [PATCH 0048/1097] refine code --- src/Databases/DatabasesCommon.cpp | 2 - src/Interpreters/InterpreterAlterQuery.cpp | 4 +- src/Interpreters/InterpreterCreateQuery.cpp | 32 +++---- src/Interpreters/InterpreterCreateQuery.h | 2 - src/Interpreters/MutationsInterpreter.cpp | 15 +--- src/Parsers/ASTAlterQuery.cpp | 2 +- src/Parsers/ASTColumnDeclaration.cpp | 12 +++ src/Parsers/ASTColumnDeclaration.h | 1 + src/Parsers/ASTCreateQuery.cpp | 12 --- src/Parsers/ASTCreateQuery.h | 3 +- src/Parsers/ExpressionElementParsers.cpp | 27 ++++++ src/Parsers/ExpressionElementParsers.h | 8 ++ src/Parsers/ParserCreateQuery.cpp | 12 --- src/Parsers/ParserCreateQuery.h | 17 +++- src/Storages/AlterCommands.cpp | 52 +++++------ src/Storages/ColumnsDescription.cpp | 6 ++ src/Storages/ColumnsDescription.h | 2 + src/Storages/IStorage.cpp | 6 ++ src/Storages/IStorage.h | 5 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 11 +-- .../MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 54 +++++++---- .../MergeTree/registerStorageMergeTree.cpp | 12 +-- src/Storages/MutationCommands.cpp | 18 ++-- src/Storages/Statistic/Statistic.cpp | 8 +- src/Storages/Statistic/Statistic.h | 6 +- src/Storages/StatisticsDescription.cpp | 89 ++++++------------- src/Storages/StatisticsDescription.h | 25 ++---- src/Storages/StorageInMemoryMetadata.cpp | 13 --- src/Storages/StorageInMemoryMetadata.h | 8 -- .../test_manipulate_statistic/__init__.py | 0 .../config/config.xml | 7 ++ .../test_manipulate_statistic/test.py | 78 ++++++++++++++++ .../0_stateless/02864_statistic_exception.sql | 27 +++--- .../02864_statistic_operate.reference | 4 +- .../0_stateless/02864_statistic_operate.sql | 5 +- 39 files changed, 334 insertions(+), 259 deletions(-) create mode 100644 tests/integration/test_manipulate_statistic/__init__.py create mode 100644 tests/integration/test_manipulate_statistic/config/config.xml create mode 100644 tests/integration/test_manipulate_statistic/test.py diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 86419fcb207..4ba793d858d 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -46,13 +46,11 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo { ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); - ASTPtr new_statistics = InterpreterCreateQuery::formatStatistics(metadata.statistics); ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints); ASTPtr new_projections = InterpreterCreateQuery::formatProjections(metadata.projections); ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices); - ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->stats, new_statistics); ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints); ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->projections, new_projections); } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 8053aa1af53..8adfcc5421c 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -151,10 +151,10 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong parameter type in ALTER query"); - if (getContext()->getSettings().allow_experimental_statistic || + if (!getContext()->getSettings().allow_experimental_statistic && ( command_ast->type == ASTAlterCommand::ADD_STATISTIC || command_ast->type == ASTAlterCommand::DROP_STATISTIC || - command_ast->type == ASTAlterCommand::MATERIALIZE_STATISTIC) + command_ast->type == ASTAlterCommand::MATERIALIZE_STATISTIC)) throw Exception(ErrorCodes::INCORRECT_QUERY, "Alter table with statistic is now disabled. Turn on allow_experimental_statistic"); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b8c29b81c53..c9aff97cfe2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -435,6 +435,12 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->children.push_back(column_declaration->codec); } + if (column.stat) + { + column_declaration->stat_type = column.stat->ast; + column_declaration->children.push_back(column_declaration->stat_type); + } + if (column.ttl) { column_declaration->ttl = column.ttl; @@ -457,11 +463,6 @@ ASTPtr InterpreterCreateQuery::formatIndices(const IndicesDescription & indices) return res; } -ASTPtr InterpreterCreateQuery::formatStatistics(const StatisticsDescriptions & statistics) -{ - return statistics.getAST(); -} - ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription & constraints) { auto res = std::make_shared(); @@ -642,6 +643,13 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec); } + if (col_decl.stat_type) + { + if (!context_->getSettingsRef().allow_experimental_statistic) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistic is now disabled. Turn on allow_experimental_statistic"); + column.stat = StatisticDescription::getStatisticFromColumnDeclaration(col_decl); + } + if (col_decl.ttl) column.ttl = col_decl.ttl; @@ -717,18 +725,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.indices.push_back(index_desc); } - if (create.columns_list->stats) - { - const auto & settings = getContext()->getSettingsRef(); - if (!settings.allow_experimental_statistic) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistic is now disabled. Turn on allow_experimental_statistic"); - for (const auto & statistic : create.columns_list->stats->children) - { - auto stats = StatisticsDescriptions::getStatisticsFromAST(statistic->clone(), properties.columns, getContext()); - properties.stats.merge(stats); - } - } - if (create.columns_list->projections) for (const auto & projection_ast : create.columns_list->projections->children) { @@ -814,13 +810,11 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti ASTPtr new_columns = formatColumns(properties.columns); ASTPtr new_indices = formatIndices(properties.indices); - ASTPtr new_statistics = formatStatistics(properties.stats); ASTPtr new_constraints = formatConstraints(properties.constraints); ASTPtr new_projections = formatProjections(properties.projections); create.columns_list->setOrReplace(create.columns_list->columns, new_columns); create.columns_list->setOrReplace(create.columns_list->indices, new_indices); - create.columns_list->setOrReplace(create.columns_list->stats, new_statistics); create.columns_list->setOrReplace(create.columns_list->constraints, new_constraints); create.columns_list->setOrReplace(create.columns_list->projections, new_projections); diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 66ac0ebf89c..67339dea928 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -38,7 +38,6 @@ public: static ASTPtr formatColumns(const NamesAndTypesList & columns, const NamesAndAliases & alias_columns); static ASTPtr formatColumns(const ColumnsDescription & columns); static ASTPtr formatIndices(const IndicesDescription & indices); - static ASTPtr formatStatistics(const StatisticsDescriptions & statistics); static ASTPtr formatConstraints(const ConstraintsDescription & constraints); static ASTPtr formatProjections(const ProjectionsDescription & projections); @@ -81,7 +80,6 @@ private: { ColumnsDescription columns; IndicesDescription indices; - StatisticsDescriptions stats; ConstraintsDescription constraints; ProjectionsDescription projections; }; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 961a4f7ac72..87924aa3825 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -54,6 +54,7 @@ namespace ErrorCodes extern const int CANNOT_UPDATE_COLUMN; extern const int UNEXPECTED_EXPRESSION; extern const int THERE_IS_NO_COLUMN; + extern const int ILLEGAL_STATISTIC; } namespace @@ -485,7 +486,6 @@ void MutationsInterpreter::prepare(bool dry_run) /// TODO Should we get columns, indices and projections from the part itself? Table metadata may be different const ColumnsDescription & columns_desc = metadata_snapshot->getColumns(); const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices(); - const StatisticsDescriptions & statistics_desc = metadata_snapshot->getStatistics(); const ProjectionsDescription & projections_desc = metadata_snapshot->getProjections(); auto storage_snapshot = std::make_shared(*source.getStorage(), metadata_snapshot); @@ -726,16 +726,9 @@ void MutationsInterpreter::prepare(bool dry_run) mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); for (const auto & stat_column_name: command.statistic_columns) { - auto it = std::find_if( - std::cbegin(statistics_desc), std::end(statistics_desc), - [&](const StatisticDescription & statistic) - { - return statistic.column_name == stat_column_name; - }); - if (it == std::cend(statistics_desc)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown statistic column: {}", stat_column_name); - - dependencies.emplace(it->column_name, ColumnDependency::STATISTIC); + if (!columns_desc.has(stat_column_name) || !columns_desc.get(stat_column_name).stat) + throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Unknown statistic column: {}", stat_column_name); + dependencies.emplace(stat_column_name, ColumnDependency::STATISTIC); materialized_statistics.emplace(stat_column_name); } } diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 2db7bb93e8b..57f82c05099 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -209,7 +209,7 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & } else if (type == ASTAlterCommand::DROP_STATISTIC) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_statistic ? "CLEAR " : "DROP ") << "INDEX " + settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_statistic ? "CLEAR " : "DROP ") << "STATISTIC " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); statistic_decl->formatImpl(settings, state, frame); if (partition) diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index e9b490a1be3..b0d812eec6b 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -39,6 +39,12 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->codec); } + if (stat_type) + { + res->stat_type = stat_type->clone(); + res->children.push_back(res->stat_type); + } + if (ttl) { res->ttl = ttl->clone(); @@ -99,6 +105,12 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta codec->formatImpl(settings, state, frame); } + if (stat_type) + { + settings.ostr << ' '; + stat_type->formatImpl(settings, state, frame); + } + if (ttl) { settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "TTL" << (settings.hilite ? hilite_none : "") << ' '; diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 9d486667911..a54abae97ea 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -19,6 +19,7 @@ public: bool ephemeral_default = false; ASTPtr comment; ASTPtr codec; + ASTPtr stat_type; ASTPtr ttl; ASTPtr collation; bool primary_key_specifier = false; diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index d2bab2366aa..1562586bd93 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -134,8 +134,6 @@ ASTPtr ASTColumns::clone() const res->set(res->columns, columns->clone()); if (indices) res->set(res->indices, indices->clone()); - if (stats) - res->set(res->stats, stats->clone()); if (constraints) res->set(res->constraints, constraints->clone()); if (projections) @@ -170,16 +168,6 @@ void ASTColumns::formatImpl(const FormatSettings & s, FormatState & state, Forma list.children.push_back(elem); } } - if (stats) - { - for (const auto & stat : stats->children) - { - auto elem = std::make_shared(); - elem->prefix = "STATISTIC"; - elem->set(elem->elem, stat->clone()); - list.children.push_back(elem); - } - } if (constraints) { for (const auto & constraint : constraints->children) diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 0ef3975335b..28f5e05802b 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -53,7 +53,6 @@ class ASTColumns : public IAST public: ASTExpressionList * columns = nullptr; ASTExpressionList * indices = nullptr; - ASTExpressionList * stats = nullptr; ASTExpressionList * constraints = nullptr; ASTExpressionList * projections = nullptr; IAST * primary_key = nullptr; @@ -68,7 +67,7 @@ public: bool empty() const { return (!columns || columns->children.empty()) && (!indices || indices->children.empty()) && (!constraints || constraints->children.empty()) - && (!projections || projections->children.empty()) && (!stats || stats->children.empty()); + && (!projections || projections->children.empty()); } void forEachPointerToChild(std::function f) override diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index f25e7f3c7e2..c85ceec4120 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -685,6 +685,33 @@ bool ParserCodec::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; } +bool ParserStatisticType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserList stat_type_parser(std::make_unique(), + std::make_unique(TokenType::Comma), false); + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ASTPtr stat_type; + + ++pos; + + if (!stat_type_parser.parse(pos, stat_type, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto function_node = std::make_shared(); + function_node->name = "STATISTIC"; + function_node->arguments = stat_type; + function_node->children.push_back(function_node->arguments); + + node = function_node; + return true; +} + bool ParserCollation::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr collation; diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index a7980c08671..2104a71cd0d 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -197,6 +197,14 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/// STATISTIC(tdigest(200)) +class ParserStatisticType : public IParserBase +{ +protected: + const char * getName() const override { return "statistic"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + /** Parse collation * COLLATE utf8_unicode_ci NOT NULL */ diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index f9b4d90e5a8..a674cb30f0a 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -307,14 +307,12 @@ bool ParserForeignKeyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_index("INDEX"); - ParserKeyword s_stat("STATISTIC"); ParserKeyword s_constraint("CONSTRAINT"); ParserKeyword s_projection("PROJECTION"); ParserKeyword s_foreign_key("FOREIGN KEY"); ParserKeyword s_primary_key("PRIMARY KEY"); ParserIndexDeclaration index_p; - ParserStatisticDeclaration stat_p; ParserConstraintDeclaration constraint_p; ParserProjectionDeclaration projection_p; ParserForeignKeyDeclaration foreign_key_p; @@ -333,11 +331,6 @@ bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte if (!constraint_p.parse(pos, new_node, expected)) return false; } - else if (s_stat.ignore(pos, expected)) - { - if (!stat_p.parse(pos, new_node, expected)) - return false; - } else if (s_projection.ignore(pos, expected)) { if (!projection_p.parse(pos, new_node, expected)) @@ -392,7 +385,6 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr columns = std::make_shared(); ASTPtr indices = std::make_shared(); - ASTPtr stats = std::make_shared(); ASTPtr constraints = std::make_shared(); ASTPtr projections = std::make_shared(); ASTPtr primary_key; @@ -413,8 +405,6 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E } else if (elem->as()) indices->children.push_back(elem); - else if (elem->as()) - stats->children.push_back(elem); else if (elem->as()) constraints->children.push_back(elem); else if (elem->as()) @@ -443,8 +433,6 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E res->set(res->columns, columns); if (!indices->children.empty()) res->set(res->indices, indices); - if (!stats->children.empty()) - res->set(res->stats, stats); if (!constraints->children.empty()) res->set(res->constraints, constraints); if (!projections->children.empty()) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 33d7a95c9ca..910ee048442 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -131,6 +131,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_auto_increment{"AUTO_INCREMENT"}; ParserKeyword s_comment{"COMMENT"}; ParserKeyword s_codec{"CODEC"}; + ParserKeyword s_stat{"STATISTIC"}; ParserKeyword s_ttl{"TTL"}; ParserKeyword s_remove{"REMOVE"}; ParserKeyword s_type{"TYPE"}; @@ -141,6 +142,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserLiteral literal_parser; ParserCodec codec_parser; ParserCollation collation_parser; + ParserStatisticType stat_type_parser; ParserExpression expression_parser; /// mandatory column name @@ -176,6 +178,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr default_expression; ASTPtr comment_expression; ASTPtr codec_expression; + ASTPtr stat_type_expression; ASTPtr ttl_expression; ASTPtr collation_expression; bool primary_key_specifier = false; @@ -301,6 +304,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E return false; } + if (s_stat.ignore(pos, expected)) + { + if (!stat_type_parser.parse(pos, stat_type_expression, expected)) + return false; + } + if (s_ttl.ignore(pos, expected)) { if (!expression_parser.parse(pos, ttl_expression, expected)) @@ -342,6 +351,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(codec_expression)); } + if (stat_type_expression) + { + column_declaration->stat_type = stat_type_expression; + column_declaration->children.push_back(std::move(stat_type_expression)); + } + if (ttl_expression) { column_declaration->ttl = ttl_expression; @@ -389,8 +404,6 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -/** name BY columns TYPE typename(arg1, arg2, ...) */ -/** name BY columns */ class ParserStatisticDeclaration : public IParserBase { public: diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 84d0ff74c6d..e18999e1e71 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -38,6 +38,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_STATISTIC; extern const int BAD_ARGUMENTS; extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int LOGICAL_ERROR; @@ -588,46 +589,37 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } else if (type == ADD_STATISTIC) { - /// TODO: Right now we assume there is only one type of statistics for simple implement. for (const auto & statistic_column_name : statistic_columns) { - if (!if_not_exists && std::any_of( - metadata.statistics.cbegin(), - metadata.statistics.cend(), - [&](const auto & statistic) - { - return statistic.column_name == statistic_column_name; - })) + if (!metadata.columns.has(statistic_column_name)) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add statistic {} with type {}: statistic on this column with this type already exists", statistic_column_name, statistic_type); + throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Cannot add statistic {} with type {}: this column is not found", statistic_column_name, statistic_type); } + if (metadata.columns.get(statistic_column_name).stat) + throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Cannot add statistic {} with type {}: statistic on this column with this type already exists", statistic_column_name, statistic_type); } - auto stats = StatisticsDescriptions::getStatisticsFromAST(statistic_decl, metadata.columns, context); - metadata.statistics.merge(stats); + auto stats = StatisticDescription::getStatisticsFromAST(statistic_decl, metadata.columns); + for (auto && stat : stats) + { + metadata.columns.modify(stat.column_name, + [&](ColumnDescription & column) { column.stat = std::move(stat); }); + } } else if (type == DROP_STATISTIC) { - if (!partition && !clear) + for (const auto & stat_column_name : statistic_columns) { - for (const auto & stat_column_name : statistic_columns) + if (!metadata.columns.has(stat_column_name) || !metadata.columns.get(stat_column_name).stat) { - auto erase_it = std::find_if( - metadata.statistics.begin(), - metadata.statistics.end(), - [stat_column_name](const auto & statistic) - { - return statistic.column_name == stat_column_name; - }); - - if (erase_it == metadata.statistics.end()) - { - if (if_exists) - return; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong statistic name. Cannot find statistic {} with type {} to drop", backQuote(stat_column_name), statistic_type); - } - LOG_INFO(&Poco::Logger::get("drop_stat"), "dropping statistic {}", erase_it->column_name); - metadata.statistics.erase(erase_it); + if (if_exists) + return; + throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Wrong statistic name. Cannot find statistic {} with type {} to drop", backQuote(stat_column_name), statistic_type); + } + if (!partition && !clear) + { + metadata.columns.modify(stat_column_name, + [&](ColumnDescription & column) { column.stat = std::nullopt; }); } } } @@ -854,7 +846,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada if (isRemovingProperty() || type == REMOVE_TTL || type == REMOVE_SAMPLE_BY) return false; - if (type == DROP_INDEX || type == DROP_PROJECTION || type == RENAME_COLUMN) + if (type == DROP_INDEX || type == DROP_PROJECTION || type == RENAME_COLUMN || type == DROP_STATISTIC) return true; /// Drop alias is metadata alter, in other case mutation is required. diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0c918bda5fd..500e7ae6d82 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -90,6 +90,12 @@ void ColumnDescription::writeText(WriteBuffer & buf) const writeEscapedString(queryToString(codec), buf); } + if (stat) + { + writeChar('\t', buf); + writeEscapedString(queryToString(stat->ast), buf); + } + if (ttl) { writeChar('\t', buf); diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index ee0bb5efb66..9d92984d5d8 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -83,6 +84,7 @@ struct ColumnDescription String comment; ASTPtr codec; ASTPtr ttl; + std::optional stat; ColumnDescription() = default; ColumnDescription(ColumnDescription &&) = default; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index ae7659e074f..47b0cb0df2a 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -222,6 +223,11 @@ StorageID IStorage::getStorageID() const return storage_id; } +ConditionEstimator IStorage::getConditionEstimatorByPredicate(const SelectQueryInfo &, ContextPtr) const +{ + return {}; +} + void IStorage::renameInMemory(const StorageID & new_table_id) { std::lock_guard lock(id_mutex); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 9e95f4edee1..40fdbe4a68c 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -69,6 +68,8 @@ using DatabaseAndTableName = std::pair; class BackupEntriesCollector; class RestorerFromBackup; +class ConditionEstimator; + struct ColumnSize { size_t marks = 0; @@ -136,7 +137,7 @@ public: /// Returns true if the storage supports queries with the PREWHERE section. virtual bool supportsPrewhere() const { return false; } - virtual ConditionEstimator getConditionEstimatorByPredicate(const SelectQueryInfo &, ContextPtr) const { return {}; } + virtual ConditionEstimator getConditionEstimatorByPredicate(const SelectQueryInfo &, ContextPtr) const; /// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported. /// This is needed for engines whose aggregates data from multiple tables, like Merge. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c412744b18b..440e1e1b06d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -638,7 +638,7 @@ Statistics IMergeTreeDataPart::loadStatistics() const { const auto & metadata_snaphost = storage.getInMemoryMetadata(); - auto total_statistics = MergeTreeStatisticFactory::instance().getMany(metadata_snaphost.getStatistics()); + auto total_statistics = MergeTreeStatisticFactory::instance().getMany(metadata_snaphost.getColumns()); Statistics result; for (auto & stat : total_statistics) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 30887ae1b7c..1c98c15107d 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -366,7 +366,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->metadata_snapshot, global_ctx->merging_columns, MergeTreeIndexFactory::instance().getMany(global_ctx->metadata_snapshot->getSecondaryIndices()), - MergeTreeStatisticFactory::instance().getMany(global_ctx->metadata_snapshot->getStatistics()), + MergeTreeStatisticFactory::instance().getMany(global_ctx->metadata_snapshot->getColumns()), ctx->compression_codec, global_ctx->txn, /*reset_columns=*/ true, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9d43caa4765..3042eac075d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -651,15 +651,10 @@ void MergeTreeData::checkProperties( } } - if (!new_metadata.statistics.empty()) + for (const auto & col : new_metadata.columns) { - for (const auto & stat : new_metadata.statistics) - { - auto column = all_columns.tryGetByName(stat.column_name); - if (!column.has_value()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "column {} is not found", stat.column_name); - MergeTreeStatisticFactory::instance().validate(stat, column->type); - } + if (col.stat) + MergeTreeStatisticFactory::instance().validate(*col.stat, col.type); } checkKeyExpression(*new_sorting_key.expression, new_sorting_key.sample_block, "Sorting", allow_nullable_key); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 170ea797e70..42c36f2e510 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -94,7 +94,7 @@ MutableDataPartStoragePtr MergeTreeDataPartInMemory::flushToDisk(const String & auto compression_codec = storage.getContext()->chooseCompressionCodec(0, 0); auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); - auto stats = MergeTreeStatisticFactory::instance().getMany(metadata_snapshot->getStatistics()); + auto stats = MergeTreeStatisticFactory::instance().getMany(metadata_snapshot->getColumns()); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, stats, compression_codec, NO_TRANSACTION_PTR); out.write(block); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index faa533bbc3d..5803cd6cc33 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -569,7 +569,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( metadata_snapshot, columns, indices, - MergeTreeStatisticFactory::instance().getMany(metadata_snapshot->getStatistics()), + MergeTreeStatisticFactory::instance().getMany(metadata_snapshot->getColumns()), compression_codec, context->getCurrentTransaction(), false, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 91a4ea3f702..445317defc0 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -88,7 +88,9 @@ static void splitAndModifyMutationCommands( if (command.type == MutationCommand::Type::MATERIALIZE_COLUMN) mutated_columns.emplace(command.column_name); } - else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION) + else if (command.type == MutationCommand::Type::DROP_INDEX + || command.type == MutationCommand::Type::DROP_PROJECTION + || command.type == MutationCommand::Type::DROP_STATISTIC) { for_file_renames.push_back(command); } @@ -204,7 +206,9 @@ static void splitAndModifyMutationCommands( { for_interpreter.push_back(command); } - else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION) + else if (command.type == MutationCommand::Type::DROP_INDEX + || command.type == MutationCommand::Type::DROP_PROJECTION + || command.type == MutationCommand::Type::DROP_STATISTIC) { for_file_renames.push_back(command); } @@ -456,12 +460,12 @@ static std::set getStatisticsToRecalculate(const StorageMetadataPt { const auto & stats_factory = MergeTreeStatisticFactory::instance(); std::set stats_to_recalc; - const auto & stats = metadata_snapshot->getStatistics(); - for (const auto & stat_desc : stats) + const auto & columns = metadata_snapshot->getColumns(); + for (const auto & col_desc : columns) { - if (materialized_stats.contains(stat_desc.column_name)) + if (col_desc.stat && materialized_stats.contains(col_desc.name)) { - stats_to_recalc.insert(stats_factory.get(stat_desc)); + stats_to_recalc.insert(stats_factory.get(*col_desc.stat)); } } return stats_to_recalc; @@ -667,11 +671,11 @@ static NameToNameVector collectFilesForRenames( if (source_part->checksums.has(command.column_name + ".proj")) add_rename(command.column_name + ".proj", ""); } - //else if (command.type == MutationCommand::Type::DROP_STATISTICS) - //{ - // if (source_part->checksums.has(command.column_name + ".stat")) - // add_rename(command.column_name + ".stat", ""); - //} + else if (command.type == MutationCommand::Type::DROP_STATISTIC) + { + if (source_part->checksums.has(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX)) + add_rename(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX, ""); + } else if (isWidePart(source_part)) { if (command.type == MutationCommand::Type::DROP_COLUMN) @@ -1315,10 +1319,14 @@ private: NameSet entries_to_hardlink; NameSet removed_indices; + NameSet removed_stats; for (const auto & command : ctx->for_file_renames) { if (command.type == MutationCommand::DROP_INDEX) removed_indices.insert(command.column_name); + else if (command.type == MutationCommand::DROP_STATISTIC) + for (const auto & column_name : command.statistic_columns) + removed_stats.insert(column_name); } bool is_full_part_storage = isFullPartStorage(ctx->new_data_part->getDataPartStorage()); @@ -1354,17 +1362,23 @@ private: } } + for (auto stat_name : removed_stats) + LOG_INFO(ctx->log, "removed stat {}", stat_name); + Statistics stats; - const auto & statistics = ctx->metadata_snapshot->getStatistics(); - for (const auto & stat : statistics) + const auto & columns = ctx->metadata_snapshot->getColumns(); + for (const auto & col : columns) { - if (ctx->materialized_statistics.contains(stat.column_name)) + if (!col.stat || removed_stats.contains(col.name)) + continue; + + if (ctx->materialized_statistics.contains(col.name)) { - stats.push_back(MergeTreeStatisticFactory::instance().get(stat)); + stats.push_back(MergeTreeStatisticFactory::instance().get(*col.stat)); } else { - auto prefix = fmt::format("{}{}.", STAT_FILE_PREFIX, stat.column_name); + auto prefix = fmt::format("{}{}.", STAT_FILE_PREFIX, col.name); auto it = ctx->source_part->checksums.files.upper_bound(prefix); while (it != ctx->source_part->checksums.files.end()) { @@ -1378,6 +1392,9 @@ private: } } + for (auto hardlink : entries_to_hardlink) + LOG_INFO(ctx->log, "hard link {}", hardlink); + NameSet removed_projections; for (const auto & command : ctx->for_file_renames) { @@ -1639,6 +1656,10 @@ private: } } + LOG_INFO(ctx->log, "hard link size : {}", hardlinked_files.size()); + for (auto name : hardlinked_files) + LOG_INFO(ctx->log, "hard link file : {}", name); + /// Tracking of hardlinked files required for zero-copy replication. /// We don't remove them when we delete last copy of source part because /// new part can use them. @@ -1941,7 +1962,6 @@ bool MutateTask::prepare() ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); ctx->materialized_statistics = ctx->interpreter->grabMaterializedStatistics(); - LOG_INFO(ctx->log, "stats number {}", ctx->materialized_statistics.size()); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index b299a7aa873..7ec88eb2e7b 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -580,12 +580,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) for (auto & index : args.query.columns_list->indices->children) metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, columns, context)); - if (args.query.columns_list && args.query.columns_list->stats) - for (const auto & stat_ast : args.query.columns_list->stats->children) - { - auto stats = StatisticsDescriptions::getStatisticsFromAST(stat_ast, columns, args.getContext()); - metadata.statistics.merge(stats); - } + ///if (args.query.columns_list && args.query.columns_list->stats) + /// for (const auto & stat_ast : args.query.columns_list->stats->children) + /// { + /// auto stats = StatisticsDescriptions::getStatisticsFromAST(stat_ast, columns, args.getContext()); + /// metadata.statistics.merge(stats); + /// } if (args.query.columns_list && args.query.columns_list->projections) for (auto & projection_ast : args.query.columns_list->projections->children) diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 46322f3ef05..9a2e66d9948 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -77,11 +77,7 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.type = MATERIALIZE_STATISTIC; res.partition = command->partition; res.predicate = nullptr; - for (const ASTPtr & column_ast : command->statistic_decl->as().columns->children) - { - const auto & column = column_ast->as().getColumnName(); - res.statistic_columns.push_back(column); - } + res.statistic_columns = command->statistic_decl->as().getColumnNames(); return res; } else if (command->type == ASTAlterCommand::MATERIALIZE_PROJECTION) @@ -138,6 +134,18 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.clear = true; return res; } + else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_STATISTIC) + { + MutationCommand res; + res.ast = command->ptr(); + res.type = MutationCommand::Type::DROP_STATISTIC; + if (command->partition) + res.partition = command->partition; + if (command->clear_index) + res.clear = true; + res.statistic_columns = command->statistic_decl->as().getColumnNames(); + return res; + } else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_PROJECTION) { MutationCommand res; diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index b72a1780772..661312d4865 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -204,11 +205,12 @@ StatisticPtr MergeTreeStatisticFactory::get(const StatisticDescription & stat) c return std::make_shared(stat); } -Statistics MergeTreeStatisticFactory::getMany(const StatisticsDescriptions & stats) const +Statistics MergeTreeStatisticFactory::getMany(const ColumnsDescription & columns) const { Statistics result; - for (const auto & stat : stats) - result.push_back(get(stat)); + for (const auto & col : columns) + if (col.stat) + result.push_back(get(*col.stat)); return result; } diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index ec912dcc39c..7c95fe48c11 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -4,10 +4,10 @@ #include #include #include +#include #include #include #include -#include "Common/Exception.h" #include #include @@ -105,6 +105,8 @@ public: } }; +class ColumnsDescription; + class MergeTreeStatisticFactory : private boost::noncopyable { public: @@ -118,7 +120,7 @@ public: StatisticPtr get(const StatisticDescription & stat) const; - Statistics getMany(const StatisticsDescriptions & stats) const; + Statistics getMany(const ColumnsDescription & columns) const; void registerCreator(StatisticType type, Creator creator); void registerValidator(StatisticType type, Validator validator); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 5d9ea6d670b..9fdfaf9b9b8 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -18,42 +19,35 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_QUERY; - extern const int ILLEGAL_STATISTIC; extern const int LOGICAL_ERROR; }; -StatisticType StatisticDescription::stringToType(String type) +StatisticType stringToType(String type) { if (type == "tdigest") return TDigest; throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}", type); } -namespace -{ - -String typeToString(StatisticType type) +String StatisticDescription::getTypeName() const { if (type == TDigest) return "tdigest"; - return "unknown"; + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}", type); } -} - -StatisticsDescriptions StatisticsDescriptions::getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context) +std::vector StatisticDescription::getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns) { const auto * stat_definition = definition_ast->as(); if (!stat_definition) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create statistic from non ASTStatisticDeclaration AST"); - LOG_INFO(&Poco::Logger::get("stats_desc"), "stat_def is like {}", stat_definition->dumpTree()); - - StatisticsDescriptions stats; + std::vector stats; + stats.reserve(stat_definition->columns->children.size()); for (const auto & column_ast : stat_definition->columns->children) { StatisticDescription stat; - stat.type = StatisticDescription::stringToType(Poco::toLower(stat_definition->type)); + stat.type = stringToType(Poco::toLower(stat_definition->type)); String column_name = column_ast->as().name(); if (!columns.hasPhysical(column_name)) @@ -61,64 +55,39 @@ StatisticsDescriptions StatisticsDescriptions::getStatisticsFromAST(const ASTPtr const auto & column = columns.getPhysical(column_name); stat.column_name = column.name; - /// TODO: check if it is numeric. - stat.data_type = column.type; + + auto function_node = std::make_shared(); + function_node->name = "STATISTIC"; + function_node->arguments = std::make_shared(); + function_node->arguments->children.push_back(std::make_shared(stat_definition->type)); + function_node->children.push_back(function_node->arguments); + + stat.ast = function_node; + stats.push_back(stat); } if (stats.empty()) throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistic column list"); - LOG_INFO(&Poco::Logger::get("stats_desc"), "there are {} stats", stats.size()); - - UNUSED(context); - return stats; } -bool StatisticsDescriptions::has(const String & name) const +String queryToString(const IAST & query); + +StatisticDescription StatisticDescription::getStatisticFromColumnDeclaration(const ASTColumnDeclaration & column) { - for (const auto & statistic : *this) - if (statistic.column_name == name) - return true; - return false; -} + const auto & stat_type_list_ast = column.stat_type->as().arguments; + if (stat_type_list_ast->children.size() != 1) + throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect only one statistic type for column {}", queryToString(column)); + const auto & stat_type = stat_type_list_ast->children[0]->as().name; -void StatisticsDescriptions::merge(const StatisticsDescriptions & other) -{ - /// Check duplicate - for (const auto & old_stat : * this) - for (const auto & new_stat : other) - if (old_stat.column_name == new_stat.column_name) - throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Statistic column {} has existed", old_stat.column_name); - insert(end(), other.begin(), other.end()); -} + StatisticDescription stat; + stat.type = stringToType(Poco::toLower(stat_type)); + stat.column_name = column.name; + stat.ast = column.stat_type; -ASTPtr StatisticsDescriptions::getAST() const -{ - - auto list = std::make_shared(); - - for (const auto & stat : *this) - { - auto stat_ast = std::make_shared(); - auto cols_ast = std::make_shared(); - auto col_ast = std::make_shared(stat.column_name); - cols_ast->children.push_back(col_ast); - stat_ast->set(stat_ast->columns, cols_ast); - stat_ast->type = typeToString(stat.type); - - list->children.push_back(stat_ast); - } - return list; -} - -String StatisticsDescriptions::toString() const -{ - if (empty()) - return {}; - - return serializeAST(*getAST()); + return stat; } } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 2949cac3245..0918433e553 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -1,8 +1,8 @@ #pragma once #include +#include #include -#include namespace DB { @@ -12,6 +12,8 @@ enum StatisticType TDigest = 0, }; +class ColumnsDescription; + struct StatisticDescription { /// the type of statistic, right now it's only tdigest. @@ -20,26 +22,15 @@ struct StatisticDescription /// Names of statistic columns String column_name; - /// Data types of statistic columns - DataTypePtr data_type; + ASTPtr ast; + + String getTypeName() const; StatisticDescription() = default; - static StatisticType stringToType(String type); -}; + static StatisticDescription getStatisticFromColumnDeclaration(const ASTColumnDeclaration & column); -struct StatisticsDescriptions : public std::vector -{ - /// Stat with name exists - bool has(const String & name) const; - /// merge with other Statistics - void merge(const StatisticsDescriptions & other); - - ASTPtr getAST() const; - /// Convert description to string - String toString() const; - /// Parse description from string - static StatisticsDescriptions getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context); + static std::vector getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); }; } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index fdaeb87f3e0..af285a953dc 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -4,7 +4,6 @@ #include #include #include -#include "Storages/StatisticsDescription.h" #include #include #include @@ -29,7 +28,6 @@ namespace ErrorCodes StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & other) : columns(other.columns) , secondary_indices(other.secondary_indices) - , statistics(other.statistics) , constraints(other.constraints) , projections(other.projections.clone()) , minmax_count_projection( @@ -54,7 +52,6 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo columns = other.columns; secondary_indices = other.secondary_indices; - statistics = other.statistics; constraints = other.constraints; projections = other.projections.clone(); if (other.minmax_count_projection) @@ -94,11 +91,6 @@ void StorageInMemoryMetadata::setSecondaryIndices(IndicesDescription secondary_i secondary_indices = std::move(secondary_indices_); } -void StorageInMemoryMetadata::setStatistics(StatisticsDescriptions statistics_) -{ - statistics = std::move(statistics_); -} - void StorageInMemoryMetadata::setConstraints(ConstraintsDescription constraints_) { constraints = std::move(constraints_); @@ -154,11 +146,6 @@ const IndicesDescription & StorageInMemoryMetadata::getSecondaryIndices() const return secondary_indices; } -const StatisticsDescriptions & StorageInMemoryMetadata::getStatistics() const -{ - return statistics; -} - bool StorageInMemoryMetadata::hasSecondaryIndices() const { return !secondary_indices.empty(); diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 1244737ccc6..30b2b303492 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -25,8 +24,6 @@ struct StorageInMemoryMetadata ColumnsDescription columns; /// Table indices. Currently supported for MergeTree only. IndicesDescription secondary_indices; - - StatisticsDescriptions statistics; /// Table constraints. Currently supported for MergeTree only. ConstraintsDescription constraints; /// Table projections. Currently supported for MergeTree only. @@ -78,9 +75,6 @@ struct StorageInMemoryMetadata /// Sets secondary indices void setSecondaryIndices(IndicesDescription secondary_indices_); - /// Sets statistics - void setStatistics(StatisticsDescriptions statistics_); - /// Sets constraints void setConstraints(ConstraintsDescription constraints_); @@ -111,8 +105,6 @@ struct StorageInMemoryMetadata /// Returns secondary indices const IndicesDescription & getSecondaryIndices() const; - const StatisticsDescriptions & getStatistics() const; - /// Has at least one non primary index bool hasSecondaryIndices() const; diff --git a/tests/integration/test_manipulate_statistic/__init__.py b/tests/integration/test_manipulate_statistic/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_manipulate_statistic/config/config.xml b/tests/integration/test_manipulate_statistic/config/config.xml new file mode 100644 index 00000000000..b47f8123499 --- /dev/null +++ b/tests/integration/test_manipulate_statistic/config/config.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_manipulate_statistic/test.py b/tests/integration/test_manipulate_statistic/test.py new file mode 100644 index 00000000000..c48bfac19c8 --- /dev/null +++ b/tests/integration/test_manipulate_statistic/test.py @@ -0,0 +1,78 @@ +import pytest +import logging + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance("node1", user_configs=["config/config.xml"], with_zookeeper=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def check_stat_file_on_disk(node, table, part_name, column_name, exist): + part_path = node.query( + "SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format( + table, part_name + ) + ).strip() + output = node.exec_in_container( + [ + "bash", + "-c", + "find {p} -type f -name statistic_{col}.stat".format( + p=part_path, col=column_name + ), + ], + privileged=True, + ) + logging.debug(f"stat file ls in {part_path} for column {column_name}, shows {output}") + if exist: + assert len(output) != 0 + else: + assert len(output) == 0 + +def test_single_node(started_cluster): + node1.query("DROP TABLE IF EXISTS test_stat") + + node1.query(""" + CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) + ENGINE = MergeTree() ORDER BY a; + """) + + node1.query("INSERT INTO test_stat VALUES (1,2,3), (4,5,6)") + + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "a", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "b", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "c", True) + + node1.query("ALTER TABLE test_stat DROP STATISTIC a type tdigest") + + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "a", False) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "b", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "c", True) + + node1.query("ALTER TABLE test_stat CLEAR STATISTIC b, c type tdigest") + + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "a", False) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "b", False) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "c", False) + + node1.query("ALTER TABLE test_stat MATERIALIZE STATISTIC b, c type tdigest") + + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "a", False) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "b", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "c", True) + + node1.query("ALTER TABLE test_stat ADD STATISTIC a type tdigest") + + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "a", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "b", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "c", True) diff --git a/tests/queries/0_stateless/02864_statistic_exception.sql b/tests/queries/0_stateless/02864_statistic_exception.sql index 58b43768679..32ac2021d29 100644 --- a/tests/queries/0_stateless/02864_statistic_exception.sql +++ b/tests/queries/0_stateless/02864_statistic_exception.sql @@ -2,29 +2,25 @@ DROP TABLE IF EXISTS t1; CREATE TABLE t1 ( - a Float64, - b Int64, + a Float64 STATISTIC(tdigest), + b Int64 STATISTIC(tdigest), pk String, - STATISTIC a, b TYPE tdigest, ) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } SET allow_experimental_statistic = 1; CREATE TABLE t1 ( - a Float64, + a Float64 STATISTIC(tdigest), b Int64, - pk String, - STATISTIC a, a TYPE tdigest, + pk String STATISTIC(tdigest), ) Engine = MergeTree() ORDER BY pk; -- { serverError ILLEGAL_STATISTIC } CREATE TABLE t1 ( - a Float64, + a Float64 STATISTIC(tdigest, tdigest(10)), b Int64, - pk String, - STATISTIC a, pk TYPE tdigest, -) Engine = MergeTree() ORDER BY pk; -- { serverError ILLEGAL_STATISTIC } +) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } CREATE TABLE t1 ( @@ -34,9 +30,12 @@ CREATE TABLE t1 ) Engine = MergeTree() ORDER BY pk; ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -- { serverError INCORRECT_QUERY } -ALTER TABLE t1 ADD STATISTIC pk tdigest; -- { serverError ILLEGAL_STATISTIC } -ALTER TABLE t1 DROP STATISTIC b TYPE tdigest; -- { serverError INCORRECT_QUERY } -ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } +ALTER TABLE t1 ADD STATISTIC pk TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } +ALTER TABLE t1 DROP STATISTIC b TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } +ALTER TABLE t1 DROP STATISTIC a TYPE tdigest; +ALTER TABLE t1 DROP STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } +ALTER TABLE t1 CLEAR STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } +ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } DROP TABLE t1; diff --git a/tests/queries/0_stateless/02864_statistic_operate.reference b/tests/queries/0_stateless/02864_statistic_operate.reference index 67f9eb6de15..2726064be00 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.reference +++ b/tests/queries/0_stateless/02864_statistic_operate.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String,\n STATISTIC a TYPE tdigest,\n STATISTIC b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `b` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After insert SELECT count() FROM t1 @@ -12,7 +12,7 @@ PREWHERE (b < 10) AND (a < 10) 10 CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After add statistic -CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String,\n STATISTIC a TYPE tdigest,\n STATISTIC b TYPE tdigest\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `b` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After materialize statistic SELECT count() FROM t1 diff --git a/tests/queries/0_stateless/02864_statistic_operate.sql b/tests/queries/0_stateless/02864_statistic_operate.sql index 66fc4a9ec34..e0a13afab84 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.sql +++ b/tests/queries/0_stateless/02864_statistic_operate.sql @@ -5,10 +5,9 @@ SET allow_statistic_optimize = 1; CREATE TABLE t1 ( - a Float64, - b Int64, + a Float64 STATISTIC(tdigest), + b Int64 STATISTIC(tdigest), pk String, - STATISTIC a, b TYPE tdigest, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; From 6c4474d3d123979723a81314b201087443050680 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 26 Sep 2023 22:22:11 +0200 Subject: [PATCH 0049/1097] refine code structure --- .../sql-reference/statements/alter/index.md | 1 + .../statements/alter/statistic.md | 25 ++++ src/Storages/IStorage.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 1 + .../MergeTree/MergeTreeWhereOptimizer.h | 2 +- src/Storages/Statistic/Estimator.cpp | 135 ++++++++++++++++++ src/Storages/Statistic/Estimator.h | 106 ++++++++++++++ src/Storages/Statistic/Statistic.cpp | 129 ----------------- src/Storages/Statistic/Statistic.h | 101 ------------- 9 files changed, 270 insertions(+), 232 deletions(-) create mode 100644 docs/en/sql-reference/statements/alter/statistic.md create mode 100644 src/Storages/Statistic/Estimator.cpp create mode 100644 src/Storages/Statistic/Estimator.h diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/index.md index dca34d16f25..e76e42a424c 100644 --- a/docs/en/sql-reference/statements/alter/index.md +++ b/docs/en/sql-reference/statements/alter/index.md @@ -16,6 +16,7 @@ Most `ALTER TABLE` queries modify table settings or data: - [INDEX](/docs/en/sql-reference/statements/alter/skipping-index.md) - [CONSTRAINT](/docs/en/sql-reference/statements/alter/constraint.md) - [TTL](/docs/en/sql-reference/statements/alter/ttl.md) +- [STATISTIC](../../../sql-reference/statements/alter/statistic.md) :::note Most `ALTER TABLE` queries are supported only for [\*MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables, as well as [Merge](/docs/en/engines/table-engines/special/merge.md) and [Distributed](/docs/en/engines/table-engines/special/distributed.md). diff --git a/docs/en/sql-reference/statements/alter/statistic.md b/docs/en/sql-reference/statements/alter/statistic.md new file mode 100644 index 00000000000..a4719325997 --- /dev/null +++ b/docs/en/sql-reference/statements/alter/statistic.md @@ -0,0 +1,25 @@ +--- +toc_hidden_folder: true +sidebar_position: 42 +sidebar_label: STATISTIC +--- + +# Manipulating Column Statistics + +The following operations are available: + +- `ALTER TABLE [db].table ADD STATISTIC (columns list) TYPE type` - Adds statistic description to tables metadata. + +- `ALTER TABLE [db].table DROP STATISTIC (columns list) TYPE type` - Removes statistic description from tables metadata and deletes statistic files from disk. + +- `ALTER TABLE [db].table CLEAR STATISTIC (columns list) TYPE type` - Deletes statistic files from disk. + +- `ALTER TABLE [db.]table MATERIALIZE STATISTIC (columns list) TYPE type` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). + +The first two commands are lightweight in a sense that they only change metadata or remove files. + +Also, they are replicated, syncing statistics metadata via ZooKeeper. + +:::note +Statistic manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). +::: diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 47b0cb0df2a..2c9e339cb48 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 290f1d1d987..d909d5adef5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -73,6 +73,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index b08fd627a29..0e0205b2b1e 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -4,9 +4,9 @@ #include #include #include +#include #include -#include "Storages/Statistic/Statistic.h" #include #include diff --git a/src/Storages/Statistic/Estimator.cpp b/src/Storages/Statistic/Estimator.cpp new file mode 100644 index 00000000000..031a38a4171 --- /dev/null +++ b/src/Storages/Statistic/Estimator.cpp @@ -0,0 +1,135 @@ +#include +#include + +namespace DB +{ + +std::optional ConditionEstimator::extractSingleColumn(const RPNBuilderTreeNode & node) const +{ + if (node.isConstant()) + { + return std::nullopt; + } + + if (!node.isFunction()) + { + auto column_name = node.getColumnName(); + return {column_name}; + } + + auto function_node = node.toFunctionNode(); + size_t arguments_size = function_node.getArgumentsSize(); + std::optional result; + for (size_t i = 0; i < arguments_size; ++i) + { + auto function_argument = function_node.getArgumentAt(i); + auto subresult = extractSingleColumn(function_argument); + if (subresult == std::nullopt) + continue; + else if (subresult == "") + return ""; + else if (result == std::nullopt) + result = subresult; + else if (result.value() != subresult.value()) + return ""; + } + return result; +} + +std::pair ConditionEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const +{ + if (!node.isFunction()) + return {}; + + auto function_node = node.toFunctionNode(); + if (function_node.getArgumentsSize() != 2) + return {}; + + std::string function_name = function_node.getFunctionName(); + + auto lhs_argument = function_node.getArgumentAt(0); + auto rhs_argument = function_node.getArgumentAt(1); + + auto lhs_argument_column_name = lhs_argument.getColumnName(); + auto rhs_argument_column_name = rhs_argument.getColumnName(); + + bool lhs_argument_is_column = column_name == (lhs_argument_column_name); + bool rhs_argument_is_column = column_name == (rhs_argument_column_name); + + bool lhs_argument_is_constant = lhs_argument.isConstant(); + bool rhs_argument_is_constant = rhs_argument.isConstant(); + + RPNBuilderTreeNode * constant_node = nullptr; + + if (lhs_argument_is_column && rhs_argument_is_constant) + constant_node = &rhs_argument; + else if (lhs_argument_is_constant && rhs_argument_is_column) + constant_node = &lhs_argument; + else + return {}; + + Field output_value; + DataTypePtr output_type; + if (!constant_node->tryGetConstant(output_value, output_type)) + return {}; + + const auto type = output_value.getType(); + Float64 value; + if (type == Field::Types::Int64) + value = output_value.get(); + else if (type == Field::Types::UInt64) + value = output_value.get(); + else if (type == Field::Types::Float64) + value = output_value.get(); + else + return {}; + return std::make_pair(function_name, value); +} + +Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) const +{ + auto col = extractSingleColumn(node); + if (col == std::nullopt || col == "") + { + return default_unknown_cond_factor; + } + auto it = column_estimators.find(col.value()); + + /// If there the estimator of the column is not found or there are no data at all, + /// we use dummy estimation. + bool dummy = total_count == 0; + ColumnEstimator estimator; + if (it != column_estimators.end()) + { + estimator = it->second; + } + else + { + dummy = true; + } + auto [op, val] = extractBinaryOp(node, col.value()); + if (op == "equals") + { + if (val < - threshold || val > threshold) + return default_normal_cond_factor; + else + return default_good_cond_factor; + } + else if (op == "less" || op == "lessThan") + { + if (dummy) + return default_normal_cond_factor; + return estimator.estimateLess(val) / total_count; + } + else if (op == "greater" || op == "greaterThan") + { + if (dummy) + return default_normal_cond_factor; + return estimator.estimateGreater(val) / total_count; + } + else + return default_unknown_cond_factor; +} + + +} diff --git a/src/Storages/Statistic/Estimator.h b/src/Storages/Statistic/Estimator.h new file mode 100644 index 00000000000..a52351f4879 --- /dev/null +++ b/src/Storages/Statistic/Estimator.h @@ -0,0 +1,106 @@ +#pragma once + +#include + +namespace DB +{ + +class RPNBuilderTreeNode; + +class ConditionEstimator +{ +private: + + static constexpr auto default_good_cond_factor = 0.1; + static constexpr auto default_normal_cond_factor = 0.5; + static constexpr auto default_unknown_cond_factor = 1.0; + /// Conditions like "x = N" are considered good if abs(N) > threshold. + /// This is used to assume that condition is likely to have good selectivity. + static constexpr auto threshold = 2; + + UInt64 total_count = 0; + + struct PartColumnEstimator + { + UInt64 part_count = 0; + + std::shared_ptr t_digest; + + void merge(StatisticPtr statistic) + { + UInt64 cur_part_count = statistic->count(); + if (part_count == 0) + part_count = cur_part_count; + + if (typeid_cast(statistic.get())) + { + t_digest = std::static_pointer_cast(statistic); + } + } + + Float64 estimateLess(Float64 val) const + { + if (t_digest != nullptr) + return t_digest -> estimateLess(val); + return part_count * default_normal_cond_factor; + } + + Float64 estimateGreator(Float64 val) const + { + if (t_digest != nullptr) + return part_count - t_digest -> estimateLess(val); + return part_count * default_normal_cond_factor; + } + }; + + struct ColumnEstimator + { + std::map estimators; + + void merge(std::string part_name, StatisticPtr statistic) + { + estimators[part_name].merge(statistic); + } + + Float64 estimateLess(Float64 val) const + { + if (estimators.empty()) + return default_normal_cond_factor; + Float64 result = 0; + for (const auto & [key, estimator] : estimators) + result += estimator.estimateLess(val); + return result; + } + + Float64 estimateGreater(Float64 val) const + { + if (estimators.empty()) + return default_normal_cond_factor; + Float64 result = 0; + for (const auto & [key, estimator] : estimators) + result += estimator.estimateGreator(val); + return result; + } + }; + + std::map column_estimators; + std::optional extractSingleColumn(const RPNBuilderTreeNode & node) const; + std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const; + +public: + ConditionEstimator() = default; + + /// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ... + /// Right now we only support simple condition like col = val / col < val + Float64 estimateSelectivity(const RPNBuilderTreeNode & node) const; + + void merge(std::string part_name, UInt64 part_count, StatisticPtr statistic) + { + total_count += part_count; + if (statistic != nullptr) + column_estimators[statistic->columnName()].merge(part_name, statistic); + } +}; + + +} diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index 661312d4865..311172bffc0 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB { @@ -18,134 +17,6 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTIC; } - -std::optional ConditionEstimator::extractSingleColumn(const RPNBuilderTreeNode & node) const -{ - if (node.isConstant()) - { - return std::nullopt; - } - - if (!node.isFunction()) - { - auto column_name = node.getColumnName(); - return {column_name}; - } - - auto function_node = node.toFunctionNode(); - size_t arguments_size = function_node.getArgumentsSize(); - std::optional result; - for (size_t i = 0; i < arguments_size; ++i) - { - auto function_argument = function_node.getArgumentAt(i); - auto subresult = extractSingleColumn(function_argument); - if (subresult == std::nullopt) - continue; - else if (subresult == "") - return ""; - else if (result == std::nullopt) - result = subresult; - else if (result.value() != subresult.value()) - return ""; - } - return result; -} - -std::pair ConditionEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const -{ - if (!node.isFunction()) - return {}; - - auto function_node = node.toFunctionNode(); - if (function_node.getArgumentsSize() != 2) - return {}; - - std::string function_name = function_node.getFunctionName(); - - auto lhs_argument = function_node.getArgumentAt(0); - auto rhs_argument = function_node.getArgumentAt(1); - - auto lhs_argument_column_name = lhs_argument.getColumnName(); - auto rhs_argument_column_name = rhs_argument.getColumnName(); - - bool lhs_argument_is_column = column_name == (lhs_argument_column_name); - bool rhs_argument_is_column = column_name == (rhs_argument_column_name); - - bool lhs_argument_is_constant = lhs_argument.isConstant(); - bool rhs_argument_is_constant = rhs_argument.isConstant(); - - RPNBuilderTreeNode * constant_node = nullptr; - - if (lhs_argument_is_column && rhs_argument_is_constant) - constant_node = &rhs_argument; - else if (lhs_argument_is_constant && rhs_argument_is_column) - constant_node = &lhs_argument; - else - return {}; - - Field output_value; - DataTypePtr output_type; - if (!constant_node->tryGetConstant(output_value, output_type)) - return {}; - - const auto type = output_value.getType(); - Float64 value; - if (type == Field::Types::Int64) - value = output_value.get(); - else if (type == Field::Types::UInt64) - value = output_value.get(); - else if (type == Field::Types::Float64) - value = output_value.get(); - else - return {}; - return std::make_pair(function_name, value); -} - -Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) const -{ - auto col = extractSingleColumn(node); - if (col == std::nullopt || col == "") - { - return default_unknown_cond_factor; - } - auto it = column_estimators.find(col.value()); - - /// If there the estimator of the column is not found or there are no data at all, - /// we use dummy estimation. - bool dummy = total_count == 0; - ColumnEstimator estimator; - if (it != column_estimators.end()) - { - estimator = it->second; - } - else - { - dummy = true; - } - auto [op, val] = extractBinaryOp(node, col.value()); - if (op == "equals") - { - if (val < - threshold || val > threshold) - return default_normal_cond_factor; - else - return default_good_cond_factor; - } - else if (op == "less" || op == "lessThan") - { - if (dummy) - return default_normal_cond_factor; - return estimator.estimateLess(val) / total_count; - } - else if (op == "greater" || op == "greaterThan") - { - if (dummy) - return default_normal_cond_factor; - return estimator.estimateGreater(val) / total_count; - } - else - return default_unknown_cond_factor; -} - StatisticPtr TDigestCreator(const StatisticDescription & stat) { /// TODO: check column data types. diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index 7c95fe48c11..0bb416b9feb 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -76,7 +76,6 @@ public: void serialize(WriteBuffer & buf) override { data.serialize(buf); - LOG_DEBUG(&Poco::Logger::get("t-digest"), "serialize into {} data", buf.offset()); } void deserialize(ReadBuffer & buf) override @@ -95,8 +94,6 @@ public: Float64 value = column_with_type.column->getFloat64(i); data.add(value, 1); } - - LOG_DEBUG(&Poco::Logger::get("t-digest"), "write into {} data", size); } UInt64 count() override @@ -135,102 +132,4 @@ private: Validators validators; }; -class RPNBuilderTreeNode; - -class ConditionEstimator -{ -private: - - static constexpr auto default_good_cond_factor = 0.1; - static constexpr auto default_normal_cond_factor = 0.5; - static constexpr auto default_unknown_cond_factor = 1.0; - /// Conditions like "x = N" are considered good if abs(N) > threshold. - /// This is used to assume that condition is likely to have good selectivity. - static constexpr auto threshold = 2; - - UInt64 total_count = 0; - - struct PartColumnEstimator - { - UInt64 part_count = 0; - - std::shared_ptr t_digest; - - void merge(StatisticPtr statistic) - { - UInt64 cur_part_count = statistic->count(); - if (part_count == 0) - part_count = cur_part_count; - - if (typeid_cast(statistic.get())) - { - t_digest = std::static_pointer_cast(statistic); - } - } - - Float64 estimateLess(Float64 val) const - { - if (t_digest != nullptr) - return t_digest -> estimateLess(val); - return part_count * default_normal_cond_factor; - } - - Float64 estimateGreator(Float64 val) const - { - if (t_digest != nullptr) - return part_count - t_digest -> estimateLess(val); - return part_count * default_normal_cond_factor; - } - }; - - struct ColumnEstimator - { - std::map estimators; - - void merge(std::string part_name, StatisticPtr statistic) - { - estimators[part_name].merge(statistic); - } - - Float64 estimateLess(Float64 val) const - { - if (estimators.empty()) - return default_normal_cond_factor; - Float64 result = 0; - for (const auto & [key, estimator] : estimators) - result += estimator.estimateLess(val); - return result; - } - - Float64 estimateGreater(Float64 val) const - { - if (estimators.empty()) - return default_normal_cond_factor; - Float64 result = 0; - for (const auto & [key, estimator] : estimators) - result += estimator.estimateGreator(val); - return result; - } - }; - - std::map column_estimators; - std::optional extractSingleColumn(const RPNBuilderTreeNode & node) const; - std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const; - -public: - ConditionEstimator() = default; - - /// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ... - /// Right now we only support simple condition like col = val / col < val - Float64 estimateSelectivity(const RPNBuilderTreeNode & node) const; - - void merge(std::string part_name, UInt64 part_count, StatisticPtr statistic) - { - total_count += part_count; - if (statistic != nullptr) - column_estimators[statistic->columnName()].merge(part_name, statistic); - } -}; - - } From 76b9d43dc44e33271500767ebf7d8dc09e8b650b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 27 Sep 2023 00:48:47 +0200 Subject: [PATCH 0050/1097] make code beautiful --- .../table-engines/mergetree-family/mergetree.md | 2 +- docs/en/sql-reference/statements/alter/index.md | 2 +- .../sql-reference/statements/alter/statistic.md | 8 ++++---- src/Access/Common/AccessType.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 11 ----------- .../MergeTree/registerStorageMergeTree.cpp | 7 ------- src/Storages/MutationCommands.cpp | 1 - .../test_manipulate_statistic/test.py | 17 +++++++++++++---- 8 files changed, 20 insertions(+), 30 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index bf68153e2a2..16a9e977c6b 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -44,7 +44,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2], ... PROJECTION projection_name_1 (SELECT [GROUP BY] [ORDER BY]), - PROJECTION projection_name_2 (SELECT [GROUP BY] [ORDER BY]), + PROJECTION projection_name_2 (SELECT [GROUP BY] [ORDER BY]) ) ENGINE = MergeTree() ORDER BY expr [PARTITION BY expr] diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/index.md index e76e42a424c..d28542e0a43 100644 --- a/docs/en/sql-reference/statements/alter/index.md +++ b/docs/en/sql-reference/statements/alter/index.md @@ -16,7 +16,7 @@ Most `ALTER TABLE` queries modify table settings or data: - [INDEX](/docs/en/sql-reference/statements/alter/skipping-index.md) - [CONSTRAINT](/docs/en/sql-reference/statements/alter/constraint.md) - [TTL](/docs/en/sql-reference/statements/alter/ttl.md) -- [STATISTIC](../../../sql-reference/statements/alter/statistic.md) +- [STATISTIC](/docs/en/sql-reference/statements/alter/statistic.md) :::note Most `ALTER TABLE` queries are supported only for [\*MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables, as well as [Merge](/docs/en/engines/table-engines/special/merge.md) and [Distributed](/docs/en/engines/table-engines/special/distributed.md). diff --git a/docs/en/sql-reference/statements/alter/statistic.md b/docs/en/sql-reference/statements/alter/statistic.md index a4719325997..1c2e45b23fd 100644 --- a/docs/en/sql-reference/statements/alter/statistic.md +++ b/docs/en/sql-reference/statements/alter/statistic.md @@ -1,6 +1,6 @@ --- -toc_hidden_folder: true -sidebar_position: 42 +slug: /en/sql-reference/statements/alter/statistic +sidebar_position: 45 sidebar_label: STATISTIC --- @@ -14,12 +14,12 @@ The following operations are available: - `ALTER TABLE [db].table CLEAR STATISTIC (columns list) TYPE type` - Deletes statistic files from disk. -- `ALTER TABLE [db.]table MATERIALIZE STATISTIC (columns list) TYPE type` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE STATISTIC (columns list) TYPE type` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing statistics metadata via ZooKeeper. :::note -Statistic manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). +Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 4b0ec66043d..2590a50b0dd 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -54,7 +54,7 @@ enum class AccessType M(ALTER_ADD_STATISTIC, "ALTER ADD STATISTIC", TABLE, ALTER_STATISTIC) \ M(ALTER_DROP_STATISTIC, "ALTER DROP STATISTIC", TABLE, ALTER_STATISTIC) \ M(ALTER_MATERIALIZE_STATISTIC, "ALTER MATERIALIZE STATISTIC", TABLE, ALTER_STATISTIC) \ - M(ALTER_STATISTIC, "STATISTIC", GROUP, ALTER_TABLE) /* allows to execute ALTER ORDER BY or ALTER {ADD|DROP...} INDEX */\ + M(ALTER_STATISTIC, "STATISTIC", GROUP, ALTER_TABLE) /* allows to execute ALTER STATISTIC */\ \ M(ALTER_ADD_PROJECTION, "ADD PROJECTION", TABLE, ALTER_PROJECTION) \ M(ALTER_DROP_PROJECTION, "DROP PROJECTION", TABLE, ALTER_PROJECTION) \ diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index f9099cab08b..ebe34aa0702 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1377,9 +1377,6 @@ private: } } - for (auto stat_name : removed_stats) - LOG_INFO(ctx->log, "removed stat {}", stat_name); - Statistics stats; const auto & columns = ctx->metadata_snapshot->getColumns(); for (const auto & col : columns) @@ -1407,9 +1404,6 @@ private: } } - for (auto hardlink : entries_to_hardlink) - LOG_INFO(ctx->log, "hard link {}", hardlink); - NameSet removed_projections; for (const auto & command : ctx->for_file_renames) { @@ -1498,7 +1492,6 @@ private: ctx->minmax_idx = std::make_shared(); - LOG_TRACE(ctx->log, "going to write {} stats", stats.size()); ctx->out = std::make_shared( ctx->new_data_part, ctx->metadata_snapshot, @@ -1671,10 +1664,6 @@ private: } } - LOG_INFO(ctx->log, "hard link size : {}", hardlinked_files.size()); - for (auto name : hardlinked_files) - LOG_INFO(ctx->log, "hard link file : {}", name); - /// Tracking of hardlinked files required for zero-copy replication. /// We don't remove them when we delete last copy of source part because /// new part can use them. diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 7ec88eb2e7b..0a182789311 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -580,13 +580,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) for (auto & index : args.query.columns_list->indices->children) metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, columns, context)); - ///if (args.query.columns_list && args.query.columns_list->stats) - /// for (const auto & stat_ast : args.query.columns_list->stats->children) - /// { - /// auto stats = StatisticsDescriptions::getStatisticsFromAST(stat_ast, columns, args.getContext()); - /// metadata.statistics.merge(stats); - /// } - if (args.query.columns_list && args.query.columns_list->projections) for (auto & projection_ast : args.query.columns_list->projections->children) { diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 9a2e66d9948..03200d0d9fa 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -11,7 +11,6 @@ #include #include #include -#include "Parsers/ASTAlterQuery.h" #include #include diff --git a/tests/integration/test_manipulate_statistic/test.py b/tests/integration/test_manipulate_statistic/test.py index c48bfac19c8..f3f059ce4a2 100644 --- a/tests/integration/test_manipulate_statistic/test.py +++ b/tests/integration/test_manipulate_statistic/test.py @@ -5,7 +5,10 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", user_configs=["config/config.xml"], with_zookeeper=True) +node1 = cluster.add_instance( + "node1", user_configs=["config/config.xml"], with_zookeeper=True +) + @pytest.fixture(scope="module") def started_cluster(): @@ -17,6 +20,7 @@ def started_cluster(): finally: cluster.shutdown() + def check_stat_file_on_disk(node, table, part_name, column_name, exist): part_path = node.query( "SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format( @@ -33,19 +37,24 @@ def check_stat_file_on_disk(node, table, part_name, column_name, exist): ], privileged=True, ) - logging.debug(f"stat file ls in {part_path} for column {column_name}, shows {output}") + logging.debug( + f"stat file ls in {part_path} for column {column_name}, shows {output}" + ) if exist: assert len(output) != 0 else: assert len(output) == 0 + def test_single_node(started_cluster): node1.query("DROP TABLE IF EXISTS test_stat") - node1.query(""" + node1.query( + """ CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) ENGINE = MergeTree() ORDER BY a; - """) + """ + ) node1.query("INSERT INTO test_stat VALUES (1,2,3), (4,5,6)") From 15fe392fed647ece1d18fea8d2a6b8197c1a9ae6 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 27 Sep 2023 10:13:23 +0000 Subject: [PATCH 0051/1097] 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 0052/1097] 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 6ce77b07f72961c0f5b47c7a2113134c00db102d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 27 Sep 2023 17:59:53 +0200 Subject: [PATCH 0053/1097] refine exception messages --- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- src/Storages/Statistic/tests/gtest_stats.cpp | 4 ++-- src/Storages/StatisticsDescription.cpp | 4 ++-- .../00002_log_and_exception_messages_formatting.sql | 1 + tests/queries/0_stateless/02864_statistic_exception.sql | 7 +++++++ 5 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 59902a6a115..0cac051bb2c 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -275,7 +275,7 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree cond.selectivity = estimator.estimateSelectivity(node); if (node.getASTNode() != nullptr) - LOG_DEBUG(log, "Condition {} has selectivity {}", node.getASTNode()->dumpTree(), cond.selectivity); + LOG_TEST(log, "Condition {} has selectivity {}", node.getASTNode()->dumpTree(), cond.selectivity); } if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) diff --git a/src/Storages/Statistic/tests/gtest_stats.cpp b/src/Storages/Statistic/tests/gtest_stats.cpp index b3070040c5c..52e60a7b030 100644 --- a/src/Storages/Statistic/tests/gtest_stats.cpp +++ b/src/Storages/Statistic/tests/gtest_stats.cpp @@ -20,8 +20,8 @@ TEST(Statistic, TDigestLessThan) DB::QuantileTDigest t_digest; - for (int i = 0; i < data1.size(); i++) - t_digest.add(data1[i]); + for (Int64 i : data1) + t_digest.add(i); t_digest.compress(); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 9fdfaf9b9b8..a427fb6a7cd 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -26,14 +26,14 @@ StatisticType stringToType(String type) { if (type == "tdigest") return TDigest; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}. We only support statistic type `tdigest` right now.", type); } String StatisticDescription::getTypeName() const { if (type == TDigest) return "tdigest"; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}. We only support statistic type `tdigest` right now.", type); } std::vector StatisticDescription::getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns) 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..aa7603f8c35 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 @@ -46,6 +46,7 @@ create temporary table known_short_messages (s String) as select * from (select 'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}', 'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64' +'Unknown statistic column: {}' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. diff --git a/tests/queries/0_stateless/02864_statistic_exception.sql b/tests/queries/0_stateless/02864_statistic_exception.sql index 32ac2021d29..95b94a9bca3 100644 --- a/tests/queries/0_stateless/02864_statistic_exception.sql +++ b/tests/queries/0_stateless/02864_statistic_exception.sql @@ -22,6 +22,12 @@ CREATE TABLE t1 b Int64, ) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } +CREATE TABLE t1 +( + a Float64 STATISTIC(xyz), + b Int64, +) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } + CREATE TABLE t1 ( a Float64, @@ -29,6 +35,7 @@ CREATE TABLE t1 pk String, ) Engine = MergeTree() ORDER BY pk; +ALTER TABLE t1 ADD STATISTIC a TYPE xyz; -- { serverError INCORRECT_QUERY } ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } ALTER TABLE t1 ADD STATISTIC pk TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } From 18f6fd1e5f8f676cc6d8850b28bdde35f3702289 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 27 Sep 2023 21:46:20 +0200 Subject: [PATCH 0054/1097] more comments --- src/Storages/Statistic/Estimator.h | 17 +++++++++++------ src/Storages/Statistic/Statistic.h | 3 ++- ...02_log_and_exception_messages_formatting.sql | 2 +- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Storages/Statistic/Estimator.h b/src/Storages/Statistic/Estimator.h index a52351f4879..53ea46cbfd5 100644 --- a/src/Storages/Statistic/Estimator.h +++ b/src/Storages/Statistic/Estimator.h @@ -7,6 +7,7 @@ namespace DB class RPNBuilderTreeNode; +/// It estimates the selectivity of a condition. class ConditionEstimator { private: @@ -20,11 +21,13 @@ private: UInt64 total_count = 0; + /// Minimum estimator for values in a part. It can contains multiple types of statistics. + /// But right now we only have tdigest; struct PartColumnEstimator { UInt64 part_count = 0; - std::shared_ptr t_digest; + std::shared_ptr tdigest; void merge(StatisticPtr statistic) { @@ -34,25 +37,27 @@ private: if (typeid_cast(statistic.get())) { - t_digest = std::static_pointer_cast(statistic); + tdigest = std::static_pointer_cast(statistic); } } Float64 estimateLess(Float64 val) const { - if (t_digest != nullptr) - return t_digest -> estimateLess(val); + if (tdigest != nullptr) + return tdigest -> estimateLess(val); return part_count * default_normal_cond_factor; } Float64 estimateGreator(Float64 val) const { - if (t_digest != nullptr) - return part_count - t_digest -> estimateLess(val); + if (tdigest != nullptr) + return part_count - tdigest -> estimateLess(val); return part_count * default_normal_cond_factor; } }; + /// An estimator for a column consists of several PartColumnEstimator. + /// We simply get selectivity for every part estimator and combine the result. struct ColumnEstimator { std::map estimators; diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index 0bb416b9feb..f474fffafa5 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -23,7 +23,7 @@ class IStatistic; using StatisticPtr = std::shared_ptr; using Statistics = std::vector; -/// Statistic for a column +/// Statistic contains the distribution of values in a column. /// right now we support /// - tdigest class IStatistic @@ -60,6 +60,7 @@ protected: }; +/// TDigestStatistic is a kind of histogram. class TDigestStatistic : public IStatistic { QuantileTDigest 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 aa7603f8c35..38626fc2928 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 @@ -45,7 +45,7 @@ create temporary table known_short_messages (s String) as select * from (select 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', 'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}', -'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64' +'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64', 'Unknown statistic column: {}' ] as arr) array join arr; From d6c1c0e805f024a5f4c7e00ef4d65c6bedd2bfa0 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 28 Sep 2023 18:18:14 +0200 Subject: [PATCH 0055/1097] refine tests and support drop columns with statistic --- src/Storages/MergeTree/MutateTask.cpp | 24 ++++---- .../test_manipulate_statistic/test.py | 59 +++++++++++++------ 2 files changed, 56 insertions(+), 27 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ebe34aa0702..ed6bf8f5c7e 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -675,8 +675,9 @@ static NameToNameVector collectFilesForRenames( } else if (command.type == MutationCommand::Type::DROP_STATISTIC) { - if (source_part->checksums.has(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX)) - add_rename(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX, ""); + for (const auto & statistic_column_name : command.statistic_columns) + if (source_part->checksums.has(STAT_FILE_PREFIX + statistic_column_name + STAT_FILE_SUFFIX)) + add_rename(STAT_FILE_PREFIX + statistic_column_name + STAT_FILE_SUFFIX, ""); } else if (isWidePart(source_part)) { @@ -696,6 +697,10 @@ static NameToNameVector collectFilesForRenames( if (auto serialization = source_part->tryGetSerialization(command.column_name)) serialization->enumerateStreams(callback); + + /// if we drop a column with statistic, we should also drop the stat file. + if (source_part->checksums.has(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX)) + add_rename(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX, ""); } else if (command.type == MutationCommand::Type::RENAME_COLUMN) { @@ -1377,7 +1382,7 @@ private: } } - Statistics stats; + Statistics stats_to_rewrite; const auto & columns = ctx->metadata_snapshot->getColumns(); for (const auto & col : columns) { @@ -1386,20 +1391,19 @@ private: if (ctx->materialized_statistics.contains(col.name)) { - stats.push_back(MergeTreeStatisticFactory::instance().get(*col.stat)); + stats_to_rewrite.push_back(MergeTreeStatisticFactory::instance().get(*col.stat)); } else { + /// We only hard-link statistics which + /// 1. not in `DROP STATISTIC` statement. It is filtered by `removed_stats` + /// 2. not in column list anymore, including `DROP COLUMN`. It is not touched by this loop. auto prefix = fmt::format("{}{}.", STAT_FILE_PREFIX, col.name); auto it = ctx->source_part->checksums.files.upper_bound(prefix); - while (it != ctx->source_part->checksums.files.end()) + if (it != ctx->source_part->checksums.files.end() && startsWith(it->first, prefix)) { - if (!startsWith(it->first, prefix)) - break; - entries_to_hardlink.insert(it->first); ctx->existing_indices_checksums.addFile(it->first, it->second.file_size, it->second.file_hash); - ++it; } } } @@ -1497,7 +1501,7 @@ private: ctx->metadata_snapshot, ctx->new_data_part->getColumns(), skip_indices, - stats, + stats_to_rewrite, ctx->compression_codec, ctx->txn, /*reset_columns=*/ true, diff --git a/tests/integration/test_manipulate_statistic/test.py b/tests/integration/test_manipulate_statistic/test.py index f3f059ce4a2..8bb4d19a23a 100644 --- a/tests/integration/test_manipulate_statistic/test.py +++ b/tests/integration/test_manipulate_statistic/test.py @@ -27,6 +27,9 @@ def check_stat_file_on_disk(node, table, part_name, column_name, exist): table, part_name ) ).strip() + + assert len(part_path) != 0 + output = node.exec_in_container( [ "bash", @@ -38,7 +41,7 @@ def check_stat_file_on_disk(node, table, part_name, column_name, exist): privileged=True, ) logging.debug( - f"stat file ls in {part_path} for column {column_name}, shows {output}" + f"Checking stat file in {part_path} for column {column_name}, got {output}" ) if exist: assert len(output) != 0 @@ -46,16 +49,7 @@ def check_stat_file_on_disk(node, table, part_name, column_name, exist): assert len(output) == 0 -def test_single_node(started_cluster): - node1.query("DROP TABLE IF EXISTS test_stat") - - node1.query( - """ - CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) - ENGINE = MergeTree() ORDER BY a; - """ - ) - +def run_test_single_node(started_cluster): node1.query("INSERT INTO test_stat VALUES (1,2,3), (4,5,6)") check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "a", True) @@ -76,12 +70,43 @@ def test_single_node(started_cluster): node1.query("ALTER TABLE test_stat MATERIALIZE STATISTIC b, c type tdigest") - check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "a", False) - check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "b", True) - check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "c", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "a", False) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "b", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "c", True) node1.query("ALTER TABLE test_stat ADD STATISTIC a type tdigest") + node1.query("ALTER TABLE test_stat MATERIALIZE STATISTIC a type tdigest") - check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "a", True) - check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "b", True) - check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "c", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "a", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "b", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "c", True) + + node1.query("ALTER TABLE test_stat DROP COLUMN c") + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_6", "a", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_6", "b", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_6", "c", False) + + +def test_single_node_wide(started_cluster): + node1.query("DROP TABLE IF EXISTS test_stat") + + node1.query( + """ + CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) + ENGINE = MergeTree() ORDER BY a + SETTINGS min_bytes_for_wide_part = 0; + """ + ) + run_test_single_node(started_cluster) + + +def test_single_node_normal(started_cluster): + node1.query("DROP TABLE IF EXISTS test_stat") + + node1.query( + """ + CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) + ENGINE = MergeTree() ORDER BY a; + """ + ) + run_test_single_node(started_cluster) From a72541d93f1cf46aa0fb577bcb59e5f2845c7a81 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Sep 2023 17:39:38 +0200 Subject: [PATCH 0056/1097] 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 ba878fb43a6f830361800e8496e16625230031b7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 4 Oct 2023 00:58:26 +0200 Subject: [PATCH 0057/1097] support rename --- src/Storages/AlterCommands.cpp | 2 + .../MergeTreeDataPartWriterOnDisk.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 44 +++++++++++++------ src/Storages/Statistic/Statistic.cpp | 13 +++++- src/Storages/Statistic/Statistic.h | 16 +------ .../test_manipulate_statistic/test.py | 12 +++++ .../02864_statistic_operate.reference | 6 +++ .../0_stateless/02864_statistic_operate.sql | 7 +++ 9 files changed, 74 insertions(+), 30 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index febeab5ad93..6e9e034c2a9 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -727,6 +727,8 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) rename_visitor.visit(column_to_modify.default_desc.expression); if (column_to_modify.ttl) rename_visitor.visit(column_to_modify.ttl); + if (column_to_modify.name == column_name && column_to_modify.stat) + column_to_modify.stat->column_name = rename_to; }); } if (metadata.table_ttl.definition_ast) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index a5a86256c9d..6e544b4a35a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -331,7 +331,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeStatistics(const Block { for (const auto & stat_ptr : stats) { - stat_ptr->update(block); + stat_ptr->update(block.getByName(stat_ptr->columnName()).column); } } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 0cac051bb2c..2322fcc9867 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -275,7 +275,7 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree cond.selectivity = estimator.estimateSelectivity(node); if (node.getASTNode() != nullptr) - LOG_TEST(log, "Condition {} has selectivity {}", node.getASTNode()->dumpTree(), cond.selectivity); + LOG_TRACE(log, "Condition {} has selectivity {}", node.getASTNode()->dumpTree(), cond.selectivity); } if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ed6bf8f5c7e..58f73033a3b 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -733,6 +733,10 @@ static NameToNameVector collectFilesForRenames( if (auto serialization = source_part->tryGetSerialization(command.column_name)) serialization->enumerateStreams(callback); + + /// if we rename a column with statistic, we should also rename the stat file. + if (source_part->checksums.has(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX)) + add_rename(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX, STAT_FILE_PREFIX + command.rename_to + STAT_FILE_SUFFIX); } else if (command.type == MutationCommand::Type::READ_COLUMN) { @@ -919,7 +923,7 @@ struct MutationContext std::set indices_to_recalc; std::set stats_to_recalc; std::set projections_to_recalc; - MergeTreeData::DataPart::Checksums existing_indices_checksums; + MergeTreeData::DataPart::Checksums existing_indices_stats_checksums; NameSet files_to_skip; NameToNameVector files_to_rename; @@ -1340,6 +1344,8 @@ private: NameSet removed_indices; NameSet removed_stats; + /// A stat file need to be renamed iff the column is renamed. + NameToNameMap renamed_stats; for (const auto & command : ctx->for_file_renames) { if (command.type == MutationCommand::DROP_INDEX) @@ -1347,6 +1353,9 @@ private: else if (command.type == MutationCommand::DROP_STATISTIC) for (const auto & column_name : command.statistic_columns) removed_stats.insert(column_name); + else if (command.type == MutationCommand::RENAME_COLUMN + && ctx->source_part->checksums.files.contains(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX)) + renamed_stats[STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX] = STAT_FILE_PREFIX + command.rename_to + STAT_FILE_SUFFIX; } bool is_full_part_storage = isFullPartStorage(ctx->new_data_part->getDataPartStorage()); @@ -1376,7 +1385,7 @@ private: break; entries_to_hardlink.insert(it->first); - ctx->existing_indices_checksums.addFile(it->first, it->second.file_size, it->second.file_hash); + ctx->existing_indices_stats_checksums.addFile(it->first, it->second.file_size, it->second.file_hash); ++it; } } @@ -1395,15 +1404,15 @@ private: } else { - /// We only hard-link statistics which - /// 1. not in `DROP STATISTIC` statement. It is filtered by `removed_stats` - /// 2. not in column list anymore, including `DROP COLUMN`. It is not touched by this loop. - auto prefix = fmt::format("{}{}.", STAT_FILE_PREFIX, col.name); - auto it = ctx->source_part->checksums.files.upper_bound(prefix); - if (it != ctx->source_part->checksums.files.end() && startsWith(it->first, prefix)) + /// We do not hard-link statistics which + /// 1. In `DROP STATISTIC` statement. It is filtered by `removed_stats` + /// 2. Not in column list anymore, including `DROP COLUMN`. It is not touched by this loop. + String stat_file_name = STAT_FILE_PREFIX + col.name + STAT_FILE_SUFFIX; + auto it = ctx->source_part->checksums.files.find(stat_file_name); + if (it != ctx->source_part->checksums.files.end()) { entries_to_hardlink.insert(it->first); - ctx->existing_indices_checksums.addFile(it->first, it->second.file_size, it->second.file_hash); + ctx->existing_indices_stats_checksums.addFile(it->first, it->second.file_size, it->second.file_hash); } } } @@ -1441,9 +1450,18 @@ private: for (auto it = ctx->source_part->getDataPartStorage().iterate(); it->isValid(); it->next()) { if (!entries_to_hardlink.contains(it->name())) - continue; - - if (it->isFile()) + { + if (renamed_stats.contains(it->name())) + { + ctx->new_data_part->getDataPartStorage().createHardLinkFrom( + ctx->source_part->getDataPartStorage(), it->name(), renamed_stats.at(it->name())); + hardlinked_files.insert(it->name()); + /// Also we need to "rename" checksums to finalize correctly. + const auto & check_sum = ctx->source_part->checksums.files.at(it->name()); + ctx->existing_indices_stats_checksums.addFile(renamed_stats.at(it->name()), check_sum.file_size, check_sum.file_hash); + } + } + else if (it->isFile()) { ctx->new_data_part->getDataPartStorage().createHardLinkFrom( ctx->source_part->getDataPartStorage(), it->name(), it->name()); @@ -1526,7 +1544,7 @@ private: ctx->mutating_pipeline.reset(); static_pointer_cast(ctx->out)->finalizePart( - ctx->new_data_part, ctx->need_sync, nullptr, &ctx->existing_indices_checksums); + ctx->new_data_part, ctx->need_sync, nullptr, &ctx->existing_indices_stats_checksums); ctx->out.reset(); } diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index 311172bffc0..d3edcf73422 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -17,9 +17,20 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTIC; } +void TDigestStatistic::update(const ColumnPtr & column) +{ + size_t size = column->size(); + + for (size_t i = 0; i < size; ++i) + { + /// TODO: support more types. + Float64 value = column->getFloat64(i); + data.add(value, 1); + } +} + StatisticPtr TDigestCreator(const StatisticDescription & stat) { - /// TODO: check column data types. return StatisticPtr(new TDigestStatistic(stat)); } diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index f474fffafa5..0fa8ff8ff3d 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -35,7 +35,6 @@ public: } virtual ~IStatistic() = default; - /// statistic_[col_name]_[type] String getFileName() const { return STAT_FILE_PREFIX + columnName(); @@ -50,7 +49,7 @@ public: virtual void deserialize(ReadBuffer & buf) = 0; - virtual void update(const Block & block) = 0; + virtual void update(const ColumnPtr & column) = 0; virtual UInt64 count() = 0; @@ -84,18 +83,7 @@ public: data.deserialize(buf); } - void update(const Block & block) override - { - const auto & column_with_type = block.getByName(columnName()); - size_t size = block.rows(); - - for (size_t i = 0; i < size; ++i) - { - /// TODO: support more types. - Float64 value = column_with_type.column->getFloat64(i); - data.add(value, 1); - } - } + void update(const ColumnPtr & column) override; UInt64 count() override { diff --git a/tests/integration/test_manipulate_statistic/test.py b/tests/integration/test_manipulate_statistic/test.py index 8bb4d19a23a..f1c00a61b07 100644 --- a/tests/integration/test_manipulate_statistic/test.py +++ b/tests/integration/test_manipulate_statistic/test.py @@ -86,6 +86,18 @@ def run_test_single_node(started_cluster): check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_6", "b", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_6", "c", False) + node1.query("ALTER TABLE test_stat RENAME COLUMN b TO c") + + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_7", "a", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_7", "b", False) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_7", "c", True) + + node1.query("ALTER TABLE test_stat RENAME COLUMN c TO b") + + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_8", "a", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_8", "b", True) + check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_8", "c", False) + def test_single_node_wide(started_cluster): node1.query("DROP TABLE IF EXISTS test_stat") diff --git a/tests/queries/0_stateless/02864_statistic_operate.reference b/tests/queries/0_stateless/02864_statistic_operate.reference index 2726064be00..7fad7c810c1 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.reference +++ b/tests/queries/0_stateless/02864_statistic_operate.reference @@ -23,3 +23,9 @@ SELECT count() FROM t1 PREWHERE (a < 10) AND (b < 10) 20 +CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `c` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After rename +SELECT count() +FROM t1 +PREWHERE (a < 10) AND (c < 10) +20 diff --git a/tests/queries/0_stateless/02864_statistic_operate.sql b/tests/queries/0_stateless/02864_statistic_operate.sql index e0a13afab84..29bd213f04a 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.sql +++ b/tests/queries/0_stateless/02864_statistic_operate.sql @@ -47,4 +47,11 @@ SELECT 'After merge'; EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10; SELECT count(*) FROM t1 WHERE b < 10 and a < 10; +ALTER TABLE t1 RENAME COLUMN b TO c; +SHOW CREATE TABLE t1; + +SELECT 'After rename'; +EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE c < 10 and a < 10; +SELECT count(*) FROM t1 WHERE c < 10 and a < 10; + DROP TABLE IF EXISTS t1; From 020d76a383b3a489a3868c3a7c1d770cfaba5549 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 24 May 2023 21:59:46 +0000 Subject: [PATCH 0058/1097] 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 0059/1097] 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 0060/1097] 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 0061/1097] 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 0062/1097] 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 0063/1097] 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 0064/1097] 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 0065/1097] 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 0066/1097] 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 0067/1097] 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 0068/1097] 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 0069/1097] 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 0070/1097] 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 0071/1097] 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 0072/1097] 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 0073/1097] 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 0074/1097] 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 0075/1097] 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 0076/1097] 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 0077/1097] 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 0078/1097] 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 0079/1097] 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 0080/1097] 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 0081/1097] 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 0082/1097] 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 0083/1097] 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 253b8efa58fb6cd56345d4785494d39ff38f0242 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 11 Oct 2023 16:57:40 +0200 Subject: [PATCH 0084/1097] do not modify column type when there is statistic --- .../mergetree-family/mergetree.md | 19 +++++++++++++++---- docs/en/operations/settings/settings.md | 8 ++++++++ src/Storages/MergeTree/MergeTreeData.cpp | 11 +++++++++++ .../0_stateless/02864_statistic_exception.sql | 5 +++++ 4 files changed, 39 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 16a9e977c6b..74b6fe40600 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1358,16 +1358,27 @@ In this sample configuration: ## Column Statistics (Experimental) {#column-statistics} -The statistic declaration is in the columns section of the `CREATE` query. +The statistic declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistic = 1`. ``` sql -STATISTIC(type) +CREATE TABLE example_table +( + a Int64 STATISTIC(tdigest), + b Float64 +) +ENGINE = MergeTree +ORDER BY a ``` -For tables from the `*MergeTree` family, statistics can be specified. +We can also manipulate statistics with `ATLER` statements. + +```sql +ATLER TABLE example_table ADD STATISTIC b TYPE tdigest; +ATLER TABLE example_table DROP STATISTIC a TYPE tdigest; +``` These lightweight statistics aggregate information about distribution of values in columns. -They can be used for query optimization (At current time they are used for moving expressions to PREWHERE). +They can be used for query optimization when we enable `set allow_statistic_optimize = 1`. #### Available Types of Column Statistics {#available-types-of-column-statistics} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f703429cb70..4644375235c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4726,3 +4726,11 @@ a Tuple( l Nullable(String) ) ``` + +## allow_experimental_statistic {#allow_experimental_statistic} + +Allows defining columns with [statistics](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) and [manipulate statistics](../../engines/table-engines/mergetree-family/mergetree.md#column-statistics). + +## allow_statistic_optimize {#allow_statistic_optimize} + +Allows using statistic to optimize the order of [prewhere conditions](../../sql-reference/statements/select/prewhere.md). diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a610ac15f9b..2a5c1da02ab 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3312,6 +3312,17 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context { columns_to_check_conversion.push_back( new_metadata.getColumns().getPhysical(command.column_name)); + + const auto & old_column = old_metadata.getColumns().get(command.column_name); + if (old_column.stat) + { + const auto & new_column = new_metadata.getColumns().get(command.column_name); + if (!old_column.type->equals(*new_column.type)) + throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "ALTER types of column {} with statistic is not not safe " + "because it can change the representation of statistic", + backQuoteIfNeed(command.column_name)); + } } } } diff --git a/tests/queries/0_stateless/02864_statistic_exception.sql b/tests/queries/0_stateless/02864_statistic_exception.sql index 95b94a9bca3..c37f6b1ce06 100644 --- a/tests/queries/0_stateless/02864_statistic_exception.sql +++ b/tests/queries/0_stateless/02864_statistic_exception.sql @@ -45,4 +45,9 @@ ALTER TABLE t1 DROP STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC ALTER TABLE t1 CLEAR STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } +ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; +ALTER TABLE t1 ADD STATISTIC b TYPE tdigest; +ALTER TABLE t1 MODIFY COLUMN a Float64 TTL now() + INTERVAL 1 MONTH; +ALTER TABLE t1 MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + DROP TABLE t1; From f31a6f2c0db3e1012394c1e7454c6ccbb2f5cf21 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 12 Oct 2023 14:49:16 +0000 Subject: [PATCH 0085/1097] Support only Full Sort --- src/Processors/QueryPlan/SortingStep.cpp | 8 +- src/Processors/QueryPlan/SortingStep.h | 17 ++- .../02884_parallel_window_functions.reference | 100 +++++++++++++++ .../02884_parallel_window_functions.sql | 119 ++++++++++++++++++ 4 files changed, 234 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02884_parallel_window_functions.reference create mode 100644 tests/queries/0_stateless/02884_parallel_window_functions.sql diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index ec32f6f6a28..e94e818a4aa 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -193,7 +193,7 @@ void SortingStep::scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline) for (size_t i = 0; i < threads; ++i) { size_t output_it = i; - auto resize = std::make_shared(ports[output_it]->getHeader(), streams, 1); + auto resize = std::make_shared(stream_header, streams, 1); auto & inputs = resize->getInputs(); for (auto input_it = inputs.begin(); input_it != inputs.end(); output_it += threads, ++input_it) @@ -367,11 +367,7 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build if (type == Type::FinishSorting) { bool need_finish_sorting = (prefix_description.size() < result_description.size()); - - if (partition_by_description.empty()) - mergingSorted(pipeline, prefix_description, (need_finish_sorting ? 0 : limit)); - - scatterByPartitionIfNeeded(pipeline); + mergingSorted(pipeline, prefix_description, (need_finish_sorting ? 0 : limit)); if (need_finish_sorting) { diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index ede91852aff..bdfc96f7931 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -97,12 +97,21 @@ private: void scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline); void updateOutputStream() override; - static void - mergeSorting(QueryPipelineBuilder & pipeline, const Settings & sort_settings, const SortDescription & result_sort_desc, UInt64 limit_); + static void mergeSorting( + QueryPipelineBuilder & pipeline, + const Settings & sort_settings, + const SortDescription & result_sort_desc, + UInt64 limit_); - void mergingSorted(QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, UInt64 limit_); + void mergingSorted( + QueryPipelineBuilder & pipeline, + const SortDescription & result_sort_desc, + UInt64 limit_); void finishSorting( - QueryPipelineBuilder & pipeline, const SortDescription & input_sort_desc, const SortDescription & result_sort_desc, UInt64 limit_); + QueryPipelineBuilder & pipeline, + const SortDescription & input_sort_desc, + const SortDescription & result_sort_desc, + UInt64 limit_); void fullSort( QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.reference b/tests/queries/0_stateless/02884_parallel_window_functions.reference new file mode 100644 index 00000000000..cab6195b625 --- /dev/null +++ b/tests/queries/0_stateless/02884_parallel_window_functions.reference @@ -0,0 +1,100 @@ +1 +-- { echoOn } + +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10; +0 2 0 +1 2 0 +2 2 0 +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10 +SETTINGS max_threads = 1; +0 2 0 +1 2 0 +2 2 0 +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 0 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 1 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 2 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 3 + GROUP BY + ac, + nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10; +0 2 0 +1 2 0 +2 2 0 diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.sql b/tests/queries/0_stateless/02884_parallel_window_functions.sql new file mode 100644 index 00000000000..5e71fadb3ff --- /dev/null +++ b/tests/queries/0_stateless/02884_parallel_window_functions.sql @@ -0,0 +1,119 @@ +CREATE TABLE window_funtion_threading +Engine = MergeTree +ORDER BY (ac, nw) +AS SELECT + toUInt64(toFloat32(number % 2) % 20000000) as ac, + toFloat32(1) as wg, + toUInt16(toFloat32(number % 3) % 400) as nw +FROM numbers_mt(10000000); + +SELECT count() FROM (EXPLAIN PIPELINE SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10) where explain ilike '%ScatterByPartitionTransform%'; + +-- { echoOn } + +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10; + +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + GROUP BY ac, nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10 +SETTINGS max_threads = 1; + +SELECT + nw, + sum(WR) AS R, + sumIf(WR, uniq_rows = 1) AS UNR +FROM +( + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 0 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 1 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 2 + GROUP BY + ac, + nw + UNION ALL + SELECT + uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, + AVG(wg) AS WR, + ac, + nw + FROM window_funtion_threading + WHERE (ac % 4) = 3 + GROUP BY + ac, + nw +) +GROUP BY nw +ORDER BY R DESC +LIMIT 10; From 09072097ec33231cd0df95dc91cfd1317f662da2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 14:32:47 +0200 Subject: [PATCH 0086/1097] 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/1097] 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/1097] 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/1097] merge_row_policy: get rid of RequiredSourceColumnsVisitor --- src/Storages/StorageMerge.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 95760bbeb42..61f2132cfeb 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -885,16 +885,9 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter // SELECT x from t if t has row policy that is based on y void ReadFromMerge::RowPolicyData::extendNames(Names & names) { - ASTPtr expr = row_policy_filter_ptr->expression; - - RequiredSourceColumnsVisitor::Data columns_context; - RequiredSourceColumnsVisitor(columns_context).visit(expr); - - const auto req_columns = columns_context.requiredColumns(); - NameSet added_names; - for (const auto & req_column : req_columns) + for (const auto & req_column : filter_actions->getRequiredColumns()) { if (std::find(names.begin(), names.end(), req_column) == names.end()) { From 7515853ad4b3e910f20df99038d706ef77ab2819 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 11:43:51 +0200 Subject: [PATCH 0090/1097] Fix build --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/Interpreters/Cache/FileCache.cpp | 4 ++-- src/Interpreters/Cache/FileCache.h | 9 +++++++-- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 3c16d3d9ae2..27d0b6706a6 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -127,7 +127,7 @@ bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch() else { CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); - file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), settings.filesystem_cache_getorset_batch_size, create_settings); + file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_getorset_batch_size); } return !file_segments->empty(); } diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 6d507413bab..82a724523e7 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -412,8 +412,8 @@ FileCache::getOrSet( size_t offset, size_t size, size_t file_size, - size_t file_segments_limit, - const CreateFileSegmentSettings & settings) + const CreateFileSegmentSettings & settings, + size_t file_segments_limit) { ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheGetOrSetMicroseconds); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 2bf7b9281d5..d85f50cf34c 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -84,8 +84,13 @@ public: * As long as pointers to returned file segments are held * it is guaranteed that these file segments are not removed from cache. */ - FileSegmentsHolderPtr - getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, size_t file_segments_limit, const CreateFileSegmentSettings & settings); + FileSegmentsHolderPtr getOrSet( + const Key & key, + size_t offset, + size_t size, + size_t file_size, + const CreateFileSegmentSettings & settings, + size_t file_segments_limit = 0); /** * Segments in returned list are ordered in ascending order and represent a full contiguous From d837aa675f5ec56434aa7f58332fc4b922b1b9ba Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 13:14:28 +0200 Subject: [PATCH 0091/1097] 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 d2ac16749a01c7246e4e9533846c5384c4146edc Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 17 Oct 2023 16:25:01 +0200 Subject: [PATCH 0092/1097] refinement --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 18a635f26e2..1d00ba0ecb6 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -647,7 +647,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.stat_type) { - if (!context_->getSettingsRef().allow_experimental_statistic) + if (!attach && !context_->getSettingsRef().allow_experimental_statistic) throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistic is now disabled. Turn on allow_experimental_statistic"); column.stat = StatisticDescription::getStatisticFromColumnDeclaration(col_decl); } From 5d8b1cea910b3beb531e1e6122d2596f7197eae8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 19:19:19 +0200 Subject: [PATCH 0093/1097] 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 0094/1097] 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 0095/1097] 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 0096/1097] 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 0097/1097] Update config --- tests/config/config.d/s3_storage_policy_by_default.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/config.d/s3_storage_policy_by_default.xml b/tests/config/config.d/s3_storage_policy_by_default.xml index dd93a317a77..e161c2ee01a 100644 --- a/tests/config/config.d/s3_storage_policy_by_default.xml +++ b/tests/config/config.d/s3_storage_policy_by_default.xml @@ -12,6 +12,7 @@ 1Gi cached_s3/ s3 + 10 From 89272e0925c91ed659b51741c58ddc364e149792 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 19 Oct 2023 11:23:56 +0200 Subject: [PATCH 0098/1097] Fix upgrade check, randomize more settings --- docker/test/upgrade/run.sh | 2 ++ tests/clickhouse-test | 3 +++ tests/config/config.d/s3_storage_policy_by_default.xml | 1 - tests/config/install.sh | 1 + tests/config/users.d/s3_cache_new.xml | 7 +++++++ 5 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/config/users.d/s3_cache_new.xml diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index c69d90b9af0..3580f8e5021 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -78,6 +78,7 @@ remove_keeper_config "create_if_not_exists" "[01]" rm /etc/clickhouse-server/config.d/merge_tree.xml rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml rm /etc/clickhouse-server/users.d/nonconst_timezone.xml +rm /etc/clickhouse-server/users.d/s3_cache_new.xml start stop @@ -114,6 +115,7 @@ sudo chgrp clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_defau rm /etc/clickhouse-server/config.d/merge_tree.xml rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml rm /etc/clickhouse-server/users.d/nonconst_timezone.xml +rm /etc/clickhouse-server/users.d/s3_cache_new.xml start diff --git a/tests/clickhouse-test b/tests/clickhouse-test index cab7d7e79ff..c0c2d482703 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -577,6 +577,9 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), + "filesystem_cache_getorset_batch_size": lambda: random.randint(0, 3, 10, 50), + "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint(0, 1), + "throw_on_error_from_cache_on_write_operations": lambda: random.randint(0, 1), "remote_filesystem_read_prefetch": lambda: random.randint(0, 1), "allow_prefetched_read_pool_for_remote_filesystem": lambda: random.randint( 0, 1 diff --git a/tests/config/config.d/s3_storage_policy_by_default.xml b/tests/config/config.d/s3_storage_policy_by_default.xml index e161c2ee01a..dd93a317a77 100644 --- a/tests/config/config.d/s3_storage_policy_by_default.xml +++ b/tests/config/config.d/s3_storage_policy_by_default.xml @@ -12,7 +12,6 @@ 1Gi cached_s3/ s3 - 10 diff --git a/tests/config/install.sh b/tests/config/install.sh index 9e3b235515d..d76949fadc7 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -151,6 +151,7 @@ if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then ln -sf $SRC_PATH/config.d/storage_conf.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/users.d/ + ln -sf $SRC_PATH/users.d/s3_cache_new.xml $DEST_SERVER_PATH/users.d/ fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then diff --git a/tests/config/users.d/s3_cache_new.xml b/tests/config/users.d/s3_cache_new.xml new file mode 100644 index 00000000000..638b7267960 --- /dev/null +++ b/tests/config/users.d/s3_cache_new.xml @@ -0,0 +1,7 @@ + + + + 10 + + + From 7a096904ed001be79b88d2ef50d0c7c460a933bc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Oct 2023 15:40:32 +0200 Subject: [PATCH 0099/1097] 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 0100/1097] 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 0101/1097] 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 0102/1097] 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 0103/1097] fuzzer/generate-test-j2: export missing product function Signed-off-by: Azat Khuzhin --- docker/test/fuzzer/generate-test-j2.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fuzzer/generate-test-j2.py b/docker/test/fuzzer/generate-test-j2.py index 11525163ed8..6fd37d6bd02 100755 --- a/docker/test/fuzzer/generate-test-j2.py +++ b/docker/test/fuzzer/generate-test-j2.py @@ -3,6 +3,7 @@ from argparse import ArgumentParser import os import jinja2 +import itertools def removesuffix(text, suffix): @@ -47,6 +48,7 @@ def main(args): loader=jinja2.FileSystemLoader(suite_dir), keep_trailing_newline=True, ) + j2env.globals.update(product=itertools.product) test_names = os.listdir(suite_dir) for test_name in test_names: From 0d27150948a54a9bf8513622111a4feb52476bbb Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 24 Oct 2023 13:09:57 +0000 Subject: [PATCH 0104/1097] 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 0105/1097] 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 0106/1097] 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 0107/1097] 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 0108/1097] 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 0109/1097] 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 0110/1097] Improvements --- src/Core/Settings.h | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 8 ++ src/Storages/StorageReplicatedMergeTree.cpp | 96 ++++++++++--------- ...ated_merge_tree_creation_failure.reference | 4 + ..._replicated_merge_tree_creation_failure.sh | 38 ++++++++ 5 files changed, 102 insertions(+), 46 deletions(-) create mode 100644 tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference create mode 100755 tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f2b55fbcd0b..955ad815e00 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -607,6 +607,8 @@ class IColumn; M(Bool, mutations_execute_subqueries_on_initiator, false, "If true scalar subqueries are executed on initiator and replaced to literals in UPDATE and DELETE queries", 0) \ M(UInt64, mutations_max_literal_size_to_replace, 16384, "The maximum size of serialized literal in bytes to replace in UPDATE and DELETE queries", 0) \ \ + M(Bool, create_replicated_merge_tree_fault_injection, false, "If true, the creation of table will be aborted after creating metadata in ZooKeeper", 0) \ + \ M(Bool, use_query_cache, false, "Enable the query cache", 0) \ M(Bool, enable_writes_to_query_cache, true, "Enable storing results of SELECT queries in the query cache", 0) \ M(Bool, enable_reads_from_query_cache, true, "Enable reading results of SELECT queries from the query cache", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a0635f18214..3da6c9fa264 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -104,6 +104,7 @@ namespace ErrorCodes extern const int UNKNOWN_STORAGE; extern const int SYNTAX_ERROR; extern const int SUPPORT_IS_DISABLED; + extern const int ABORTED; } namespace fs = std::filesystem; @@ -1442,6 +1443,13 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, "ATTACH ... FROM ... query is not supported for {} table engine, " "because such tables do not store any data on disk. Use CREATE instead.", res->getName()); + if (getContext()->getSettingsRef().create_replicated_merge_tree_fault_injection) + { + bool is_replicated_storage = typeid_cast(res.get()) != nullptr; + if (is_replicated_storage) + throw Exception(ErrorCodes::ABORTED, "Shutdown is called for table"); + } + database->createTable(getContext(), create.getTable(), res, query_ptr); /// Move table data to the proper place. Wo do not move data earlier to avoid situations diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 91b15f96297..0781684b7b7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -19,6 +19,8 @@ #include #include +#include + #include #include @@ -834,6 +836,9 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_uuid", toString(ServerUUID::get()), + zkutil::CreateMode::Persistent)); + Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) @@ -864,13 +869,13 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada const String local_metadata = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); const String local_columns = metadata_snapshot->getColumns().toString(); const String local_metadata_version = toString(metadata_snapshot->getMetadataVersion()); + const String creator_uuid = toString(ServerUUID::get()); /// It is possible for the replica to fail after creating ZK nodes without saving local metadata. /// Because of that we need to check whether the replica exists and is newly created. /// For this we check that all nodes exist, the metadata of the table is the same, and other nodes are not modified. std::vector paths_exists = { - replica_path, replica_path + "/host", replica_path + "/log_pointer", replica_path + "/queue", @@ -880,63 +885,59 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada replica_path + "/metadata", replica_path + "/columns", replica_path + "/metadata_version", - replica_path + "/mutation_pointer", replica_path + "/min_unprocessed_insert_time", replica_path + "/max_processed_insert_time", - replica_path + "/mutation_pointer" + replica_path + "/mutation_pointer", + replica_path + "/creator_uuid" }; auto response_exists = zookeeper->tryGet(paths_exists); - size_t response_num = 0; + bool all_nodes_exist = true; - if (response_exists[response_num++].error == Coordination::Error::ZOK) + for (size_t i = 0; i < response_exists.size(); ++i) { - bool all_nodes_exist = true; - - for (size_t i = 0; i < response_exists.size(); ++i) + if (response_exists[i].error != Coordination::Error::ZOK) { - if (response_exists[i].error != Coordination::Error::ZOK) - { - all_nodes_exist = false; - break; - } + all_nodes_exist = false; + break; } + } - if (all_nodes_exist) + if (all_nodes_exist) + { + size_t response_num = 0; + + const auto & zk_host = response_exists[response_num++].data; + const auto & zk_log_pointer = response_exists[response_num++].data; + const auto & zk_queue = response_exists[response_num++].data; + const auto & zk_parts = response_exists[response_num++].data; + const auto & zk_flags = response_exists[response_num++].data; + const auto & zk_is_lost = response_exists[response_num++].data; + const auto & zk_metadata = response_exists[response_num++].data; + const auto & zk_columns = response_exists[response_num++].data; + const auto & zk_metadata_version = response_exists[response_num++].data; + const auto & zk_min_unprocessed_insert_time = response_exists[response_num++].data; + const auto & zk_max_processed_insert_time = response_exists[response_num++].data; + const auto & zk_mutation_pointer = response_exists[response_num++].data; + const auto & zk_creator_uuid = response_exists[response_num++].data; + + if (zk_host.empty() && + zk_log_pointer.empty() && + zk_queue.empty() && + zk_parts.empty() && + zk_flags.empty() && + (zk_is_lost == "0" || zk_is_lost == "1") && + zk_metadata == local_metadata && + zk_columns == local_columns && + zk_metadata_version == local_metadata_version && + zk_min_unprocessed_insert_time.empty() && + zk_max_processed_insert_time.empty() && + zk_mutation_pointer.empty() && + zk_creator_uuid == creator_uuid) { - const auto & zk_host = response_exists[response_num++].data; - const auto & zk_log_pointer = response_exists[response_num++].data; - const auto & zk_queue = response_exists[response_num++].data; - const auto & zk_parts = response_exists[response_num++].data; - const auto & zk_flags = response_exists[response_num++].data; - const auto & zk_is_lost = response_exists[response_num++].data; - const auto & zk_metadata = response_exists[response_num++].data; - const auto & zk_columns = response_exists[response_num++].data; - const auto & zk_metadata_version = response_exists[response_num++].data; - const auto & zk_min_unprocessed_insert_time = response_exists[response_num++].data; - const auto & zk_max_processed_insert_time = response_exists[response_num++].data; - const auto & zk_mutation_pointer = response_exists[response_num++].data; - - if (zk_host.empty() && - zk_log_pointer.empty() && - zk_queue.empty() && - zk_parts.empty() && - zk_flags.empty() && - (zk_is_lost == "0" || zk_is_lost == "1") && - zk_metadata == local_metadata && - zk_columns == local_columns && - zk_metadata_version == local_metadata_version && - zk_mutation_pointer.empty() && - zk_min_unprocessed_insert_time.empty() && - zk_max_processed_insert_time.empty() && - zk_mutation_pointer.empty()) - { - LOG_DEBUG(log, "Empty replica {} exists, will use it", replica_path); - return; - } + LOG_DEBUG(log, "Empty replica {} exists, will use it", replica_path); + return; } - - throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, "Replica {} already exists", replica_path); } Coordination::Error code; @@ -985,6 +986,9 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_uuid", creator_uuid, + zkutil::CreateMode::Persistent)); + /// Check version of /replicas to see if there are any replicas created at the same moment of time. ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", "last added replica: " + replica_name, replicas_stat.version)); diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference new file mode 100644 index 00000000000..487b1165348 --- /dev/null +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.reference @@ -0,0 +1,4 @@ +2 +2 +2 +2 diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh new file mode 100755 index 00000000000..0c5705c7a35 --- /dev/null +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation_failure.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash +# Tags: zookeeper + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC" + +#### 1 - There is only one replica + +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" + +# We will see that the replica is empty and throw the same ABORT exception as before +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" + +# We will succeed +${CLICKHOUSE_CLIENT} \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC" + +#### 2 - There are two replicas + +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "ABORT" +${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection=1 \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r2') ORDER BY date" 2>&1 | grep -c "ABORT" + +# We will succeed +${CLICKHOUSE_CLIENT} \ + -q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC" \ No newline at end of file From 940d099e84d92eaaacaa96682c5a94b26f7a782c Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 27 Oct 2023 16:50:34 -0700 Subject: [PATCH 0111/1097] Set correct max_block_size value in docs --- docs/en/operations/settings/settings.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ccf290c8e20..60eda45ab22 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -731,11 +731,13 @@ Default value: LZ4. ## max_block_size {#setting-max_block_size} -In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. The `max_block_size` setting is a recommendation for what size of the block (in a count of rows) to load from tables. The block size shouldn’t be too small, so that the expenditures on each block are still noticeable, but not too large so that the query with LIMIT that is completed after the first block is processed quickly. The goal is to avoid consuming too much memory when extracting a large number of columns in multiple threads and to preserve at least some cache locality. +In ClickHouse, data is processed by blocks, which are sets of column parts. The internal processing cycles for a single block are efficient but there are noticeable costs when processing each block. -Default value: 65,536. +The `max_block_size` setting indicates the recommended maximum number of rows to include in a single block when loading data from tables. Blocks the size of `max_block_size` are not always loaded from the table: if ClickHouse determines that less data needs to be retrieved, a smaller block is processed. -Blocks the size of `max_block_size` are not always loaded from the table. If it is obvious that less data needs to be retrieved, a smaller block is processed. +The block size should not be too small to avoid noticeable costs when processing each block. It should also not be too large to ensure that queries with a LIMIT clause execute quickly after processing the first block. When setting `max_block_size`, the goal should be to avoid consuming too much memory when extracting a large number of columns in multiple threads and to preserve at least some cache locality. + +Default value: `65,409` ## preferred_block_size_bytes {#preferred-block-size-bytes} From 70e3dd808cc3f087504892d18a9e61eb6f948151 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 02:07:24 +0100 Subject: [PATCH 0112/1097] Granular code coverage with introspection --- CMakeLists.txt | 9 -- base/base/CMakeLists.txt | 2 + base/base/coverage.cpp | 106 ++++++++++++++++++- base/base/coverage.h | 6 ++ base/glibc-compatibility/memcpy/memcpy.cpp | 1 + base/glibc-compatibility/memcpy/memcpy.h | 2 +- cmake/sanitize.cmake | 18 ++++ contrib/CMakeLists.txt | 9 -- contrib/google-protobuf-cmake/CMakeLists.txt | 32 ------ contrib/libcxx-cmake/CMakeLists.txt | 2 - programs/CMakeLists.txt | 2 + src/CMakeLists.txt | 5 +- src/Functions/coverage.cpp | 91 ++++++++++++++++ src/Interpreters/InterpreterSystemQuery.cpp | 8 ++ src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserSystemQuery.cpp | 16 +-- 16 files changed, 244 insertions(+), 66 deletions(-) create mode 100644 src/Functions/coverage.cpp diff --git a/CMakeLists.txt b/CMakeLists.txt index a5b94efefc5..d259b105a0a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -286,9 +286,6 @@ set (CMAKE_C_STANDARD 11) set (CMAKE_C_EXTENSIONS ON) # required by most contribs written in C set (CMAKE_C_STANDARD_REQUIRED ON) -# Compiler-specific coverage flags e.g. -fcoverage-mapping -option(WITH_COVERAGE "Profile the resulting binary/binaries" OFF) - if (COMPILER_CLANG) # Enable C++14 sized global deallocation functions. It should be enabled by setting -std=c++14 but I'm not sure. # See https://reviews.llvm.org/D112921 @@ -304,12 +301,6 @@ if (COMPILER_CLANG) set(BRANCHES_WITHIN_32B_BOUNDARIES "-mbranches-within-32B-boundaries") set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}") endif() - - if (WITH_COVERAGE) - set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-instr-generate -fcoverage-mapping") - # If we want to disable coverage for specific translation units - set(WITHOUT_COVERAGE "-fno-profile-instr-generate -fno-coverage-mapping") - endif() endif () set (COMPILER_FLAGS "${COMPILER_FLAGS}") diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 8ab3c8a0711..f9bf413a6c8 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -1,3 +1,5 @@ +add_compile_options($<$,$>:${COVERAGE_FLAGS}>) + if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") endif () diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index 1027638be3d..60eb6fcac72 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -1,11 +1,15 @@ #include "coverage.h" -#if WITH_COVERAGE - #pragma GCC diagnostic ignored "-Wreserved-identifier" -# include -# include + +/// WITH_COVERAGE enables the default implementation of code coverage, +/// that dumps a map to the filesystem. + +#if WITH_COVERAGE + +#include +#include # if defined(__clang__) @@ -31,3 +35,97 @@ void dumpCoverageReportIfPossible() #endif } + + +/// SANITIZE_COVERAGE enables code instrumentation, +/// but leaves the callbacks implementation to us, +/// which we use to calculate coverage on a per-test basis +/// and to write it to system tables. + +#if defined(SANITIZE_COVERAGE) + +namespace +{ + bool initialized = false; + + uint32_t * guards_start = nullptr; + uint32_t * guards_end = nullptr; + size_t coverage_array_size = 0; + + uintptr_t * coverage_array = nullptr; +} + +extern "C" +{ + +/// This is called at least once for every DSO for initialization. +/// But we will use it only for the main DSO. +void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop) +{ + if (initialized) + return; + initialized = true; + + /// The function can be called multiple times, but we need to initialize only once. + if (start == stop || *start) + return; + + guards_start = start; + guards_end = stop; + coverage_array_size = stop - start; + + /// Note: we will leak this. + coverage_array = static_cast(malloc(sizeof(uintptr_t) * coverage_array_size)); + + resetCoverage(); +} + +/// This is called at every basic block / edge, etc. +void __sanitizer_cov_trace_pc_guard(uint32_t * guard) +{ + /// Duplicate the guard check. + if (!*guard) + return; + *guard = 0; + + /// If you set *guard to 0 this code will not be called again for this edge. + /// Now we can get the PC and do whatever you want: + /// - store it somewhere or symbolize it and print right away. + /// The values of `*guard` are as you set them in + /// __sanitizer_cov_trace_pc_guard_init and so you can make them consecutive + /// and use them to dereference an array or a bit vector. + void * pc = __builtin_return_address(0); + + coverage_array[guard - guards_start] = reinterpret_cast(pc); +} + +} + +__attribute__((no_sanitize("coverage"))) std::span getCoverage() +{ + return {coverage_array, coverage_array_size}; +} + +__attribute__((no_sanitize("coverage"))) void resetCoverage() +{ + memset(coverage_array, 0, coverage_array_size * sizeof(*coverage_array)); + + /// The guard defines whether the __sanitizer_cov_trace_pc_guard should be called. + /// For example, you can unset it after first invocation to prevent excessive work. + /// Initially set all the guards to 1 to enable callbacks. + for (uint32_t * x = guards_start; x < guards_end; ++x) + *x = 1; +} + +#else + +std::span getCoverage() +{ + return {}; +} + +void resetCoverage() +{ +} + +#endif diff --git a/base/base/coverage.h b/base/base/coverage.h index 4a57528b0ce..b6664bec223 100644 --- a/base/base/coverage.h +++ b/base/base/coverage.h @@ -1,5 +1,8 @@ #pragma once +#include +#include + /// Flush coverage report to file, depending on coverage system /// proposed by compiler (llvm for clang and gcov for gcc). /// @@ -7,3 +10,6 @@ /// Thread safe (use exclusive lock). /// Idempotent, may be called multiple times. void dumpCoverageReportIfPossible(); + +std::span getCoverage(); +void resetCoverage(); diff --git a/base/glibc-compatibility/memcpy/memcpy.cpp b/base/glibc-compatibility/memcpy/memcpy.cpp index ec43a2c3649..8bab35934d3 100644 --- a/base/glibc-compatibility/memcpy/memcpy.cpp +++ b/base/glibc-compatibility/memcpy/memcpy.cpp @@ -1,5 +1,6 @@ #include "memcpy.h" +__attribute__((no_sanitize("coverage"))) extern "C" void * memcpy(void * __restrict dst, const void * __restrict src, size_t size) { return inline_memcpy(dst, src, size); diff --git a/base/glibc-compatibility/memcpy/memcpy.h b/base/glibc-compatibility/memcpy/memcpy.h index 0930dfb5c67..86439dda061 100644 --- a/base/glibc-compatibility/memcpy/memcpy.h +++ b/base/glibc-compatibility/memcpy/memcpy.h @@ -93,7 +93,7 @@ * See https://habr.com/en/company/yandex/blog/457612/ */ - +__attribute__((no_sanitize("coverage"))) static inline void * inline_memcpy(void * __restrict dst_, const void * __restrict src_, size_t size) { /// We will use pointer arithmetic, so char pointer will be used. diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index f17283774eb..0c901f1aa36 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -58,3 +58,21 @@ if (SANITIZE) message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}") endif () endif() + +# Default coverage instrumentation (dumping the coverage map on exit) +option(WITH_COVERAGE "Instrumentation for code coverage with default implementation" OFF) + +if (WITH_COVERAGE) + message (INFORMATION "Enabled instrumentation for code coverage") + set(COVERAGE_FLAGS "-fprofile-instr-generate -fcoverage-mapping") +endif() + +option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) + +if (SANITIZE_COVERAGE) + message (INFORMATION "Enabled instrumentation for code coverage") + add_definitions(-DSANITIZE_COVERAGE=1) + set (COVERAGE_FLAGS "-fsanitize-coverage=trace-pc-guard") +endif() + +set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 390b0241e7d..fa97e59eefc 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -3,15 +3,6 @@ set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") -if (WITH_COVERAGE) - set (WITHOUT_COVERAGE_LIST ${WITHOUT_COVERAGE}) - separate_arguments(WITHOUT_COVERAGE_LIST) - # disable coverage for contib files and build with optimisations - if (COMPILER_CLANG) - add_compile_options(-O3 -DNDEBUG -finline-functions -finline-hint-functions ${WITHOUT_COVERAGE_LIST}) - endif() -endif() - if (SANITIZE STREQUAL "undefined") # 3rd-party libraries usually not intended to work with UBSan. add_compile_options(-fno-sanitize=undefined) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 268f0fbe0e4..fbb7d6ea018 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -278,38 +278,6 @@ else () COMMAND_ECHO STDOUT) endif () -# add_custom_command ( -# OUTPUT ${PROTOC_BUILD_DIR} -# COMMAND mkdir -p ${PROTOC_BUILD_DIR}) -# -# add_custom_command ( -# OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" -# -# COMMAND ${CMAKE_COMMAND} -# -G"${CMAKE_GENERATOR}" -# -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" -# -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" -# -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" -# -Dprotobuf_BUILD_TESTS=0 -# -Dprotobuf_BUILD_CONFORMANCE=0 -# -Dprotobuf_BUILD_EXAMPLES=0 -# -Dprotobuf_BUILD_PROTOC_BINARIES=1 -# "${protobuf_source_dir}/cmake" -# -# DEPENDS "${PROTOC_BUILD_DIR}" -# WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" -# COMMENT "Configuring 'protoc' for host architecture." -# USES_TERMINAL) -# -# add_custom_command ( -# OUTPUT "${PROTOC_BUILD_DIR}/protoc" -# COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" -# DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" -# COMMENT "Building 'protoc' for host architecture." -# USES_TERMINAL) -# -# add_custom_target (protoc-host DEPENDS "${PROTOC_BUILD_DIR}/protoc") - add_executable(protoc IMPORTED GLOBAL) set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc") add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc") diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index b7e59e2c9a3..c77d5d8319e 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -1,5 +1,3 @@ -include(CheckCXXCompilerFlag) - set(LIBCXX_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm-project/libcxx") set(SRCS diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index eb4a898d472..fce6894ed11 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -1,3 +1,5 @@ +add_compile_options($<$,$>:${COVERAGE_FLAGS}>) + if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") endif () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d2985665db3..f88a6cff6c0 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -1,3 +1,5 @@ +add_compile_options($<$,$>:${COVERAGE_FLAGS}>) + if (USE_INCLUDE_WHAT_YOU_USE) set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH}) endif () @@ -282,7 +284,8 @@ set_source_files_properties( Common/Elf.cpp Common/Dwarf.cpp Common/SymbolIndex.cpp - PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE}") + Common/ThreadFuzzer.cpp + PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE_FLAGS}") target_link_libraries (clickhouse_common_io PRIVATE diff --git a/src/Functions/coverage.cpp b/src/Functions/coverage.cpp new file mode 100644 index 00000000000..1825e6aa826 --- /dev/null +++ b/src/Functions/coverage.cpp @@ -0,0 +1,91 @@ +#if defined(SANITIZE_COVERAGE) + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace +{ + +/** If ClickHouse is build with coverage instrumentation, returns an array + * of currently accumulated unique code addresses. + */ +class FunctionCoverage : public IFunction +{ +public: + static constexpr auto name = "coverage"; + + String getName() const override + { + return name; + } + + explicit FunctionCoverage() + { + } + + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override + { + return false; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isDeterministic() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(std::make_shared()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + auto coverage_table = getCoverage(); + + auto column_addresses = ColumnUInt64::create(); + auto & data = column_addresses->getData(); + + for (auto ptr : coverage_table) + if (ptr) + data.push_back(ptr); + + auto column_array = ColumnArray::create( + std::move(column_addresses), + ColumnArray::ColumnOffsets::create(1, data.size())); + + return ColumnConst::create(std::move(column_array), input_rows_count); + } +}; + +} + +REGISTER_FUNCTION(Coverage) +{ + factory.registerFunction(); +} + +} + +#endif diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 07a1ae7d170..4e1d32bd3cb 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -60,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -690,6 +691,12 @@ BlockIO InterpreterSystemQuery::execute() FailPointInjection::disableFailPoint(query.fail_point_name); break; } + case Type::RESET_COVERAGE: + { + getContext()->checkAccess(AccessType::SYSTEM); + resetCoverage(); + break; + } default: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown type of SYSTEM query"); } @@ -1299,6 +1306,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::START_THREAD_FUZZER: case Type::ENABLE_FAILPOINT: case Type::DISABLE_FAILPOINT: + case Type::RESET_COVERAGE: case Type::UNKNOWN: case Type::END: break; } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index cc06e0fdcb5..5f7ba5be330 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -86,6 +86,7 @@ public: START_PULLING_REPLICATION_LOG, STOP_CLEANUP, START_CLEANUP, + RESET_COVERAGE, END }; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index a26fdc1396b..f0fc38d6adb 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -453,14 +453,14 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & } case Type::DROP_FORMAT_SCHEMA_CACHE: { - if (ParserKeyword{"FOR"}.ignore(pos, expected)) - { - if (ParserKeyword{"Protobuf"}.ignore(pos, expected)) - res->schema_cache_format = "Protobuf"; - else - return false; - } - break; + if (ParserKeyword{"FOR"}.ignore(pos, expected)) + { + if (ParserKeyword{"Protobuf"}.ignore(pos, expected)) + res->schema_cache_format = "Protobuf"; + else + return false; + } + break; } case Type::UNFREEZE: { From 3142921bb4dcb8b7169f7d32a05110c9a5baa351 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Oct 2023 11:15:11 +0100 Subject: [PATCH 0113/1097] 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 0114/1097] 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 0115/1097] 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 0116/1097] 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 0117/1097] 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 0118/1097] 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 0119/1097] 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 0120/1097] 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 0121/1097] 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 df1e0192680a63fb6870ef226f964921abcea94b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 30 Oct 2023 00:39:16 +0100 Subject: [PATCH 0122/1097] address comments --- .../mergetree-family/mergetree.md | 6 +-- src/Parsers/ParserAlterQuery.cpp | 1 - src/Storages/AlterCommands.cpp | 2 +- src/Storages/ColumnsDescription.cpp | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 6 +-- src/Storages/Statistic/Estimator.cpp | 32 ++++++++-------- src/Storages/Statistic/Estimator.h | 4 +- src/Storages/Statistic/Statistic.cpp | 30 +++++---------- src/Storages/Statistic/Statistic.h | 38 ++----------------- src/Storages/Statistic/TDigestStatistic.cpp | 38 +++++++++++++++++++ src/Storages/Statistic/TDigestStatistic.h | 28 ++++++++++++++ src/Storages/StatisticsDescription.h | 5 +++ 12 files changed, 110 insertions(+), 81 deletions(-) create mode 100644 src/Storages/Statistic/TDigestStatistic.cpp create mode 100644 src/Storages/Statistic/TDigestStatistic.h diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d4c5f8084d9..8e8a5ea7850 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1370,11 +1370,11 @@ ENGINE = MergeTree ORDER BY a ``` -We can also manipulate statistics with `ATLER` statements. +We can also manipulate statistics with `ALTER` statements. ```sql -ATLER TABLE example_table ADD STATISTIC b TYPE tdigest; -ATLER TABLE example_table DROP STATISTIC a TYPE tdigest; +ALTER TABLE example_table ADD STATISTIC b TYPE tdigest; +ALTER TABLE example_table DROP STATISTIC a TYPE tdigest; ``` These lightweight statistics aggregate information about distribution of values in columns. diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 0051136fa1f..c616c6e0441 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -352,7 +352,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->type = ASTAlterCommand::DROP_STATISTIC; - command->detach = false; } else if (s_clear_statistic.ignore(pos, expected)) { diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 6e9e034c2a9..e7885cbace7 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -597,7 +597,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Cannot add statistic {} with type {}: this column is not found", statistic_column_name, statistic_type); } - if (metadata.columns.get(statistic_column_name).stat) + if (!if_exists && metadata.columns.get(statistic_column_name).stat) throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Cannot add statistic {} with type {}: statistic on this column with this type already exists", statistic_column_name, statistic_type); } diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 6261a4c87e2..31d2ab52382 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -61,6 +61,7 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const && type->equals(*other.type) && default_desc == other.default_desc && comment == other.comment + && stat == other.stat && ast_to_str(codec) == ast_to_str(other.codec) && ast_to_str(ttl) == ast_to_str(other.ttl); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9bcd5620171..5ac8f1830c3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -459,15 +459,11 @@ ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate(const SelectQ } ASTPtr expression_ast; - Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, true /* one_part */); - // - // Generate valid expressions for filtering - bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, local_context, virtual_columns_block, expression_ast); ConditionEstimator result; PartitionPruner partition_pruner(metadata_snapshot, query_info, local_context, true /* strict */); - if (partition_pruner.isUseless() && !valid) + if (partition_pruner.isUseless()) { /// Read all partitions. for (const auto & part : parts) diff --git a/src/Storages/Statistic/Estimator.cpp b/src/Storages/Statistic/Estimator.cpp index 031a38a4171..7ecd22358e5 100644 --- a/src/Storages/Statistic/Estimator.cpp +++ b/src/Storages/Statistic/Estimator.cpp @@ -4,34 +4,35 @@ namespace DB { -std::optional ConditionEstimator::extractSingleColumn(const RPNBuilderTreeNode & node) const +/// second return value represents how many columns in the node. +static std::pair tryToExtractSingleColumn(const RPNBuilderTreeNode & node) { if (node.isConstant()) { - return std::nullopt; + return {}; } if (!node.isFunction()) { auto column_name = node.getColumnName(); - return {column_name}; + return {column_name, 1}; } auto function_node = node.toFunctionNode(); size_t arguments_size = function_node.getArgumentsSize(); - std::optional result; + std::pair result; for (size_t i = 0; i < arguments_size; ++i) { auto function_argument = function_node.getArgumentAt(i); - auto subresult = extractSingleColumn(function_argument); - if (subresult == std::nullopt) + auto subresult = tryToExtractSingleColumn(function_argument); + if (subresult.second == 0) /// the subnode contains 0 column continue; - else if (subresult == "") - return ""; - else if (result == std::nullopt) + else if (subresult.second > 1) /// the subnode contains more than 1 column + return subresult; + else if (result.second == 0 || result.first == subresult.first) /// subnodes contain same column. result = subresult; - else if (result.value() != subresult.value()) - return ""; + else + return {"", 2}; } return result; } @@ -88,12 +89,13 @@ std::pair ConditionEstimator::extractBinaryOp(const RPNBui Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) const { - auto col = extractSingleColumn(node); - if (col == std::nullopt || col == "") + auto result = tryToExtractSingleColumn(node); + if (result.second != 1) { return default_unknown_cond_factor; } - auto it = column_estimators.find(col.value()); + String col = result.first; + auto it = column_estimators.find(col); /// If there the estimator of the column is not found or there are no data at all, /// we use dummy estimation. @@ -107,7 +109,7 @@ Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) { dummy = true; } - auto [op, val] = extractBinaryOp(node, col.value()); + auto [op, val] = extractBinaryOp(node, col); if (op == "equals") { if (val < - threshold || val > threshold) diff --git a/src/Storages/Statistic/Estimator.h b/src/Storages/Statistic/Estimator.h index 53ea46cbfd5..3190e0698fe 100644 --- a/src/Storages/Statistic/Estimator.h +++ b/src/Storages/Statistic/Estimator.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { @@ -89,7 +89,7 @@ private: }; std::map column_estimators; - std::optional extractSingleColumn(const RPNBuilderTreeNode & node) const; + /// std::optional extractSingleColumn(const RPNBuilderTreeNode & node) const; std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const; public: diff --git a/src/Storages/Statistic/Statistic.cpp b/src/Storages/Statistic/Statistic.cpp index d3edcf73422..38b05c316bc 100644 --- a/src/Storages/Statistic/Statistic.cpp +++ b/src/Storages/Statistic/Statistic.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -17,16 +18,17 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTIC; } -void TDigestStatistic::update(const ColumnPtr & column) +void MergeTreeStatisticFactory::registerCreator(StatisticType stat_type, Creator creator) { - size_t size = column->size(); + if (!creators.emplace(stat_type, std::move(creator)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticFactory: the statistic creator type {} is not unique", stat_type); +} + +void MergeTreeStatisticFactory::registerValidator(StatisticType stat_type, Validator validator) +{ + if (!validators.emplace(stat_type, std::move(validator)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticFactory: the statistic validator type {} is not unique", stat_type); - for (size_t i = 0; i < size; ++i) - { - /// TODO: support more types. - Float64 value = column->getFloat64(i); - data.add(value, 1); - } } StatisticPtr TDigestCreator(const StatisticDescription & stat) @@ -41,18 +43,6 @@ void TDigestValidator(const StatisticDescription &, DataTypePtr data_type) throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "TDigest does not support type {}", data_type->getName()); } -void MergeTreeStatisticFactory::registerCreator(StatisticType stat_type, Creator creator) -{ - if (!creators.emplace(stat_type, std::move(creator)).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticFactory: the statistic creator type {} is not unique", stat_type); -} - -void MergeTreeStatisticFactory::registerValidator(StatisticType stat_type, Validator validator) -{ - if (!validators.emplace(stat_type, std::move(validator)).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticFactory: the statistic validator type {} is not unique", stat_type); - -} MergeTreeStatisticFactory::MergeTreeStatisticFactory() { diff --git a/src/Storages/Statistic/Statistic.h b/src/Storages/Statistic/Statistic.h index 0fa8ff8ff3d..132b453e465 100644 --- a/src/Storages/Statistic/Statistic.h +++ b/src/Storages/Statistic/Statistic.h @@ -3,14 +3,16 @@ #include #include #include + +#include + #include #include +#include #include #include #include -#include -#include /// this is for user-defined statistic. constexpr auto STAT_FILE_PREFIX = "statistic_"; @@ -59,38 +61,6 @@ protected: }; -/// TDigestStatistic is a kind of histogram. -class TDigestStatistic : public IStatistic -{ - QuantileTDigest data; -public: - explicit TDigestStatistic(const StatisticDescription & stat_) : IStatistic(stat_) - { - } - - Float64 estimateLess(Float64 val) const - { - return data.getCountLessThan(val); - } - - void serialize(WriteBuffer & buf) override - { - data.serialize(buf); - } - - void deserialize(ReadBuffer & buf) override - { - data.deserialize(buf); - } - - void update(const ColumnPtr & column) override; - - UInt64 count() override - { - return static_cast(data.count); - } -}; - class ColumnsDescription; class MergeTreeStatisticFactory : private boost::noncopyable diff --git a/src/Storages/Statistic/TDigestStatistic.cpp b/src/Storages/Statistic/TDigestStatistic.cpp new file mode 100644 index 00000000000..cb10902c64c --- /dev/null +++ b/src/Storages/Statistic/TDigestStatistic.cpp @@ -0,0 +1,38 @@ +#include + +namespace DB +{ + +Float64 TDigestStatistic::estimateLess(Float64 val) const +{ + return data.getCountLessThan(val); +} + +void TDigestStatistic::serialize(WriteBuffer & buf) +{ + data.serialize(buf); +} + +void TDigestStatistic::deserialize(ReadBuffer & buf) +{ + data.deserialize(buf); +} + +void TDigestStatistic::update(const ColumnPtr & column) +{ + size_t size = column->size(); + + for (size_t i = 0; i < size; ++i) + { + /// TODO: support more types. + Float64 value = column->getFloat64(i); + data.add(value, 1); + } +} + +UInt64 TDigestStatistic::count() +{ + return static_cast(data.count); +} + +} diff --git a/src/Storages/Statistic/TDigestStatistic.h b/src/Storages/Statistic/TDigestStatistic.h new file mode 100644 index 00000000000..b7e31eef363 --- /dev/null +++ b/src/Storages/Statistic/TDigestStatistic.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +/// TDigestStatistic is a kind of histogram. +class TDigestStatistic : public IStatistic +{ + QuantileTDigest data; +public: + explicit TDigestStatistic(const StatisticDescription & stat_) : IStatistic(stat_) + { + } + + Float64 estimateLess(Float64 val) const; + + void serialize(WriteBuffer & buf) override; + + void deserialize(ReadBuffer & buf) override; + + void update(const ColumnPtr & column) override; + + UInt64 count() override; +}; + +} diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 0918433e553..9a66951ab52 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -28,6 +28,11 @@ struct StatisticDescription StatisticDescription() = default; + bool operator==(const StatisticDescription & other) const + { + return type == other.type && column_name == other.column_name; + } + static StatisticDescription getStatisticFromColumnDeclaration(const ASTColumnDeclaration & column); static std::vector getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); From ea6cb1ad0c95f194519c863bb29302e8829669a3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Oct 2023 01:04:50 +0100 Subject: [PATCH 0123/1097] 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 0124/1097] 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 0125/1097] 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 0126/1097] 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 0127/1097] 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 0128/1097] 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 0129/1097] 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 0130/1097] 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 0131/1097] 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 0132/1097] 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 0133/1097] 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 0134/1097] 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 0135/1097] upgrade orc version --- contrib/orc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/orc b/contrib/orc index 5f8db0fb0a4..5046972fbab 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit 5f8db0fb0a47fbc4902bf9d7f712e65309f13d2d +Subproject commit 5046972fbabfe3cdf77a8768228793c7c0a61085 From f53fdbeeadf7a2be2fa962e0ee0be91628348b93 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Oct 2023 19:11:03 +0000 Subject: [PATCH 0136/1097] 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 1237cef26dc3eaa2e967a6a48041e0703882e648 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 31 Oct 2023 00:53:46 +0100 Subject: [PATCH 0137/1097] address comments --- src/Interpreters/InterpreterExplainQuery.cpp | 4 +++- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../QueryPlan/Optimizations/optimizePrewhere.cpp | 2 +- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++---- src/Storages/MergeTree/MergeTreeData.h | 2 +- 7 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 39cc4df5c2d..5f1536a7220 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -66,8 +66,10 @@ namespace static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data) { + /// we need to read statistic when `allow_statistic_optimize` is enabled. + bool only_analyze = !data.getContext()->getSettings().allow_statistic_optimize; InterpreterSelectQuery interpreter( - node, data.getContext(), SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify()); + node, data.getContext(), SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze(only_analyze).modify()); const SelectQueryInfo & query_info = interpreter.getQueryInfo(); if (query_info.view_query) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 91dbb7b3514..cb52283d0e3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -658,7 +658,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( MergeTreeWhereOptimizer where_optimizer{ std::move(column_compressed_sizes), metadata_snapshot, - storage->getConditionEstimatorByPredicate(query_info, context), + storage->getConditionEstimatorByPredicate(query_info, storage_snapshot, context), queried_columns, supported_prewhere_columns, log}; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 544b23f884e..5c5171d4296 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -161,7 +161,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) MergeTreeWhereOptimizer where_optimizer{ std::move(column_compressed_sizes), storage_metadata, - storage.getConditionEstimatorByPredicate(read_from_merge_tree->getQueryInfo(), context), + storage.getConditionEstimatorByPredicate(read_from_merge_tree->getQueryInfo(), storage_snapshot, context), queried_columns, storage.supportedPrewhereColumns(), &Poco::Logger::get("QueryPlanOptimizePrewhere")}; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 45a1b59a36e..8b4bae94b55 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -226,7 +226,7 @@ StorageID IStorage::getStorageID() const return storage_id; } -ConditionEstimator IStorage::getConditionEstimatorByPredicate(const SelectQueryInfo &, ContextPtr) const +ConditionEstimator IStorage::getConditionEstimatorByPredicate(const SelectQueryInfo &, const StorageSnapshotPtr &, ContextPtr) const { return {}; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 80d6c7d5bba..a5a858f57f1 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -137,7 +137,7 @@ public: /// Returns true if the storage supports queries with the PREWHERE section. virtual bool supportsPrewhere() const { return false; } - virtual ConditionEstimator getConditionEstimatorByPredicate(const SelectQueryInfo &, ContextPtr) const; + virtual ConditionEstimator getConditionEstimatorByPredicate(const SelectQueryInfo &, const StorageSnapshotPtr &, ContextPtr) const; /// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported. /// This is needed for engines whose aggregates data from multiple tables, like Merge. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 42ef7238409..92da5e5d75d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -444,14 +444,13 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const return storage_policy; } -ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const +ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate(const SelectQueryInfo & query_info, const StorageSnapshotPtr & storage_snapshot, ContextPtr local_context) const { if (!local_context->getSettings().allow_statistic_optimize) return {}; - auto parts = getDataPartsVectorForInternalUsage(); + const auto & parts = assert_cast(*storage_snapshot->data).parts; - auto metadata_snapshot = getInMemoryMetadataPtr(); if (parts.empty()) { return {}; @@ -460,7 +459,7 @@ ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate(const SelectQ ASTPtr expression_ast; ConditionEstimator result; - PartitionPruner partition_pruner(metadata_snapshot, query_info, local_context, true /* strict */); + PartitionPruner partition_pruner(storage_snapshot->metadata, query_info, local_context, true /* strict */); if (partition_pruner.isUseless()) { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 417e94cc4c4..47b134ba55b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -432,7 +432,7 @@ public: bool supportsPrewhere() const override { return true; } - ConditionEstimator getConditionEstimatorByPredicate(const SelectQueryInfo &, ContextPtr) const override; + ConditionEstimator getConditionEstimatorByPredicate(const SelectQueryInfo &, const StorageSnapshotPtr &, ContextPtr) const override; bool supportsFinal() const override; 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 0138/1097] 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 0139/1097] fix code style --- .../Impl/NativeORCBlockInputFormat.cpp | 24 +++++++++---------- tests/performance/orc_tuple_field_prune.xml | 2 +- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 88b3fbeee2b..fd0f4ee0ca0 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -747,13 +747,13 @@ static void getFileReaderAndSchema( } } -static void updateIncludeIndices( - DataTypePtr type, const orc::Type * orc_type, bool case_insensitive_column_matching, std::unordered_set & column_indices) +static void updateIncludeTypeIds( + DataTypePtr type, const orc::Type * orc_type, bool case_insensitive_column_matching, std::unordered_set & include_typeids) { /// Primitive types if (orc_type->getSubtypeCount() == 0) { - column_indices.insert(orc_type->getColumnId()); + include_typeids.insert(orc_type->getColumnId()); return; } @@ -764,8 +764,8 @@ static void updateIncludeIndices( const auto * array_type = typeid_cast(non_nullable_type.get()); if (array_type) { - updateIncludeIndices( - array_type->getNestedType(), orc_type->getSubtype(0), case_insensitive_column_matching, column_indices); + updateIncludeTypeIds( + array_type->getNestedType(), orc_type->getSubtype(0), case_insensitive_column_matching, include_typeids); } return; } @@ -773,8 +773,8 @@ static void updateIncludeIndices( const auto * map_type = typeid_cast(non_nullable_type.get()); if (map_type) { - updateIncludeIndices(map_type->getKeyType(), orc_type->getSubtype(0), case_insensitive_column_matching, column_indices); - updateIncludeIndices(map_type->getValueType(), orc_type->getSubtype(1), case_insensitive_column_matching, column_indices); + updateIncludeTypeIds(map_type->getKeyType(), orc_type->getSubtype(0), case_insensitive_column_matching, include_typeids); + updateIncludeTypeIds(map_type->getValueType(), orc_type->getSubtype(1), case_insensitive_column_matching, include_typeids); } return; } @@ -793,11 +793,11 @@ static void updateIncludeIndices( if (boost::equals(orc_type->getFieldName(struct_i), name) || (case_insensitive_column_matching && boost::iequals(orc_type->getFieldName(struct_i), name))) { - updateIncludeIndices( + updateIncludeTypeIds( tuple_type->getElement(tuple_i), orc_type->getSubtype(struct_i), case_insensitive_column_matching, - column_indices); + include_typeids); break; } } @@ -806,8 +806,8 @@ static void updateIncludeIndices( else { for (size_t i = 0; i < tuple_type->getElements().size() && i < orc_type->getSubtypeCount(); ++i) - updateIncludeIndices( - tuple_type->getElement(i), orc_type->getSubtype(i), case_insensitive_column_matching, column_indices); + updateIncludeTypeIds( + tuple_type->getElement(i), orc_type->getSubtype(i), case_insensitive_column_matching, include_typeids); } } return; @@ -866,7 +866,7 @@ void NativeORCBlockInputFormat::prepareFileReader() auto id = name_id_map[name]; if (id_type_map.contains(id)) { - updateIncludeIndices(column.type, id_type_map[id], ignore_case, include_typeids); + updateIncludeTypeIds(column.type, id_type_map[id], ignore_case, include_typeids); } } } diff --git a/tests/performance/orc_tuple_field_prune.xml b/tests/performance/orc_tuple_field_prune.xml index 2bcd15c8635..4e338733329 100644 --- a/tests/performance/orc_tuple_field_prune.xml +++ b/tests/performance/orc_tuple_field_prune.xml @@ -6,7 +6,7 @@ - insert into function file('test_orc_tfp.orc') select * from generateRandom('int64_column Nullable(Int64), tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 + insert into function file('test_orc_tfp.orc', 'ORC') select * from generateRandom('int64_column Nullable(Int64), tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 DROP TABLE IF EXISTS test_orc_tfp From 5e21d2459a00b38601bb2c5709795a9bda72fa65 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 31 Oct 2023 14:19:51 +0800 Subject: [PATCH 0140/1097] 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 0141/1097] 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 0142/1097] 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 0143/1097] 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 0144/1097] 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 0145/1097] 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 0146/1097] 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 0147/1097] Adapt version changes --- .../test_vertical_merges_from_compact_parts.py | 2 +- tests/integration/test_default_compression_codec/test.py | 2 +- .../integration/test_version_update_after_mutation/test.py | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index e0a9b5ebad6..9c9d1a4d312 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -7,7 +7,7 @@ cluster = ClickHouseCluster(__file__) node_old = cluster.add_instance( "node1", image="clickhouse/clickhouse-server", - tag="22.6", + tag="22.8", stay_alive=True, with_installed_binary=True, with_zookeeper=True, diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py index db116ff42f3..ffe22c62325 100644 --- a/tests/integration/test_default_compression_codec/test.py +++ b/tests/integration/test_default_compression_codec/test.py @@ -27,7 +27,7 @@ node2 = cluster.add_instance( ) node3 = cluster.add_instance( "node3", - main_configs=["configs/default_compression.xml", "configs/wide_parts_only.xml"], + main_configs=["configs/default_compression.xml"], image="yandex/clickhouse-server", tag="19.16.9.37", stay_alive=True, diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 9fb396b1c14..4e84b4c10ca 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -72,8 +72,8 @@ def test_mutate_and_upgrade(start_cluster): node1.query("DETACH TABLE mt") # stop being leader node1.query("SYSTEM FLUSH LOGS") node2.query("SYSTEM FLUSH LOGS") - node1.restart_with_latest_version(signal=9, fix_metadata=True) - node2.restart_with_latest_version(signal=9, fix_metadata=True) + node1.restart_with_latest_version(signal=9, fix_metadata=False) + node2.restart_with_latest_version(signal=9, fix_metadata=False) # After hard restart table can be in readonly mode exec_query_with_retry( @@ -129,7 +129,7 @@ def test_upgrade_while_mutation(start_cluster): # (We could be in process of creating some system table, which will leave empty directory on restart, # so when we start moving system tables from ordinary to atomic db, it will complain about some undeleted files) node3.query("SYSTEM FLUSH LOGS") - node3.restart_with_latest_version(signal=9, fix_metadata=True) + node3.restart_with_latest_version(signal=9, fix_metadata=False) # checks for readonly exec_query_with_retry(node3, "OPTIMIZE TABLE mt1", sleep_time=5, retry_count=60) From b27658742223e750902ec3f181d2a662fc7bba1f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Nov 2023 15:43:20 +0800 Subject: [PATCH 0148/1097] 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 0149/1097] 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 0150/1097] 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 0151/1097] 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 0152/1097] fix failed uts --- .../Impl/NativeORCBlockInputFormat.cpp | 20 +++++-------------- 1 file changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 3f98224f8aa..7a835274bb2 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -775,7 +775,7 @@ static const orc::Type * traverseDownORCTypeByName( const auto * orc_field_type = getORCTypeByName(*orc_nested_type, split.first, ignore_case); if (orc_field_type) { - /// Avoid inconsistency between CH and ORC type brought by flattened Nested type. + /// Adjust CH type to avoid inconsistency between CH and ORC type brought by flattened Nested type. type = array_type->getNestedType(); return traverseDownORCTypeByName(split.second, orc_field_type, type, ignore_case); } @@ -883,25 +883,15 @@ void NativeORCBlockInputFormat::prepareFileReader() format_settings.orc.case_insensitive_column_matching); const bool ignore_case = format_settings.orc.case_insensitive_column_matching; - const auto & header = getPort().getHeader(); const auto & file_schema = file_reader->getType(); std::unordered_set include_typeids; for (const auto & column : header) { - auto split = Nested::splitName(column.name); - if (split.second.empty()) - { - const auto * orc_type = getORCTypeByName(file_schema, column.name, ignore_case); - updateIncludeTypeIds(column.type, orc_type, ignore_case, include_typeids); - } - else - { - auto type = column.type; - const auto * orc_type = traverseDownORCTypeByName(column.name, &file_schema, type, ignore_case); - if (orc_type) - updateIncludeTypeIds(type, orc_type, ignore_case, include_typeids); - } + auto adjusted_type = column.type; + const auto * orc_type = traverseDownORCTypeByName(column.name, &file_schema, adjusted_type, ignore_case); + if (orc_type) + updateIncludeTypeIds(adjusted_type, orc_type, ignore_case, include_typeids); } include_indices.assign(include_typeids.begin(), include_typeids.end()); From dc897215dacea4f447b127254914e32f76ca001e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Nov 2023 20:42:07 +0800 Subject: [PATCH 0153/1097] fix failed uts tests/queries/0_stateless/02312_parquet_orc_arrow_names_tuples.sql --- .../Impl/NativeORCBlockInputFormat.cpp | 35 +++++++++++++++---- 1 file changed, 29 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 7a835274bb2..9501efbabb7 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -757,11 +757,32 @@ static const orc::Type * traverseDownORCTypeByName( if (target.empty()) return orc_type; - auto split = Nested::splitName(target); + auto search_struct_field = [&](const std::string & target_, const orc::Type * type_) -> std::pair + { + auto target_copy = target_; + if (ignore_case) + boost::to_lower(target_copy); + + for (size_t i = 0; i < type_->getSubtypeCount(); ++i) + { + auto field_name = type_->getFieldName(i); + if (ignore_case) + boost::to_lower(field_name); + + if (startsWith(target_copy, field_name) && (target_copy.size() == field_name.size() || target_copy[field_name.size()] == '.')) + { + return {target_copy.size() == field_name.size() ? "" : target_.substr(field_name.size() + 1), type_->getSubtype(i)}; + } + } + return {"", nullptr}; + }; + if (orc::STRUCT == orc_type->getKind()) { - const auto * orc_field_type = getORCTypeByName(*orc_type, split.first, ignore_case); - return orc_field_type ? traverseDownORCTypeByName(split.second, orc_field_type, type, ignore_case) : nullptr; + auto next_type_and_target = search_struct_field(target, orc_type); + const auto & next_target = next_type_and_target.first; + const auto * next_orc_type = next_type_and_target.second; + return next_orc_type ? traverseDownORCTypeByName(next_target, next_orc_type, type, ignore_case) : nullptr; } else if (orc::LIST == orc_type->getKind()) { @@ -772,12 +793,14 @@ static const orc::Type * traverseDownORCTypeByName( const auto * orc_nested_type = orc_type->getSubtype(0); if (array_type && orc::STRUCT == orc_nested_type->getKind()) { - const auto * orc_field_type = getORCTypeByName(*orc_nested_type, split.first, ignore_case); - if (orc_field_type) + auto next_type_and_target = search_struct_field(target, orc_nested_type); + const auto & next_target = next_type_and_target.first; + const auto * next_orc_type = next_type_and_target.second; + if (next_orc_type) { /// Adjust CH type to avoid inconsistency between CH and ORC type brought by flattened Nested type. type = array_type->getNestedType(); - return traverseDownORCTypeByName(split.second, orc_field_type, type, ignore_case); + return traverseDownORCTypeByName(next_target, next_orc_type, type, ignore_case); } else return nullptr; From 77507b843b66c4994ea46401d12b67f447154a39 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 1 Nov 2023 19:09:43 +0100 Subject: [PATCH 0154/1097] 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 0155/1097] 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 0156/1097] 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 0157/1097] 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 0158/1097] 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 0159/1097] 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 0160/1097] 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 0161/1097] 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 0162/1097] fix failed perf test --- tests/performance/orc_tuple_field_prune.xml | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/performance/orc_tuple_field_prune.xml b/tests/performance/orc_tuple_field_prune.xml index b3064f35f39..d95787af93b 100644 --- a/tests/performance/orc_tuple_field_prune.xml +++ b/tests/performance/orc_tuple_field_prune.xml @@ -2,16 +2,15 @@ 1 10000 - 0 - insert into function file('test_orc_tfp.orc', 'ORC') select * from generateRandom('int64_column Nullable(Int64), tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 + insert into function file('test_orc_tfp.orc', 'ORC') select * from generateRandom('tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), array_tuple_column Nested(a Nullable(String), b Nullable(Float64), c Nullable(Int64)), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 DROP TABLE IF EXISTS test_orc_tfp - select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, tuple_column Tuple(c Nullable(Int64))') format Null - select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, array_tuple_column Array(Tuple(c Nullable(Int64)))') format Null - select * from file('test_orc_tfp.orc', 'ORC', 'int64_column Int64, map_tuple_column Map(String, Tuple(c Nullable(Int64)))') format Null + select * from file('test_orc_tfp.orc', 'ORC', 'tuple_column Tuple(c Nullable(Int64))') format Null + select * from file('test_orc_tfp.orc', 'ORC', 'array_tuple_column Nested(c Nullable(Int64))') format Null + select * from file('test_orc_tfp.orc', 'ORC', 'map_tuple_column Map(String, Tuple(c Nullable(Int64)))') format Null From b88048e6d9a5ce72475433ac709a01a4ae515cec Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 2 Nov 2023 16:51:50 +0000 Subject: [PATCH 0163/1097] 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 0164/1097] 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 0165/1097] 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 0166/1097] 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 0167/1097] Fix review comment --- src/Client/ClientBase.cpp | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 76f2060da93..d3cb828e8f7 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1385,6 +1385,23 @@ void ClientBase::addMultiquery(std::string_view query, Arguments & common_argume common_arguments.emplace_back(query); } +namespace +{ +bool isStdinNotEmptyAndValid(ReadBufferFromFileDescriptor & std_in) +{ + try + { + return !std_in.eof(); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR) + return false; + throw; + } +} +} + void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr parsed_query) { @@ -1404,7 +1421,7 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars /// Process the query that requires transferring data blocks to the server. const auto & parsed_insert_query = parsed_query->as(); - if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && std_in.eof()))) + if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && !isStdinNotEmptyAndValid(std_in)))) { const auto & settings = global_context->getSettingsRef(); if (settings.throw_if_no_data_to_insert) @@ -1444,23 +1461,6 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars } } -namespace -{ - bool isStdinNotEmptyAndValid(ReadBufferFromFileDescriptor & std_in) - { - try - { - return !std_in.eof(); - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR) - return false; - throw; - } - } -} - void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query) { From 4b7146d47ed2894068ba6972bdf077df1625e525 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 3 Nov 2023 16:03:26 +0800 Subject: [PATCH 0168/1097] 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 0169/1097] 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 0170/1097] init commit --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 12 +++++++++++- src/Processors/Formats/Impl/NpyRowInputFormat.h | 5 +++++ .../0_stateless/02908_Npy_files_caching.reference | 4 ++++ tests/queries/0_stateless/02908_Npy_files_caching.sh | 12 ++++++++++++ 4 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02908_Npy_files_caching.reference create mode 100755 tests/queries/0_stateless/02908_Npy_files_caching.sh diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index 9acb2909626..bd8b442f180 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -265,6 +265,11 @@ NpyRowInputFormat::NpyRowInputFormat(ReadBuffer & in_, Block header_, Params par nested_type = getNestedType(types[0]); } +size_t NpyRowInputFormat::countRows(size_t max_block_size) +{ + return int(max_block_size) > header.shape[0] ? header.shape[0] : max_block_size; +} + template void NpyRowInputFormat::readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness) { @@ -395,13 +400,18 @@ NpySchemaReader::NpySchemaReader(ReadBuffer & in_) NamesAndTypesList NpySchemaReader::readSchema() { - NumpyHeader header = parseHeader(in); + header = parseHeader(in); DataTypePtr nested_type = getDataTypeFromNumpyType(header.numpy_type); DataTypePtr result_type = createNestedArrayType(nested_type, header.shape.size()); return {{"array", result_type}}; } +std::optional NpySchemaReader::readNumberOrRows() +{ + return header.shape[0]; +} + void registerInputFormatNpy(FormatFactory & factory) { factory.registerInputFormat("Npy", []( diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.h b/src/Processors/Formats/Impl/NpyRowInputFormat.h index ad32bdba3bf..8d228a3593a 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.h +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.h @@ -29,6 +29,9 @@ public: String getName() const override { return "NpyRowInputFormat"; } private: + bool supportsCountRows() const override { return true; } + size_t countRows(size_t max_block_size) override; + void readPrefix() override; bool readRow(MutableColumns & columns, RowReadExtension &) override; void readData(MutableColumns & columns); @@ -59,7 +62,9 @@ public: explicit NpySchemaReader(ReadBuffer & in_); private: + std::optional readNumberOrRows() override; NamesAndTypesList readSchema() override; + NumpyHeader header; }; } diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.reference b/tests/queries/0_stateless/02908_Npy_files_caching.reference new file mode 100644 index 00000000000..75baea619a9 --- /dev/null +++ b/tests/queries/0_stateless/02908_Npy_files_caching.reference @@ -0,0 +1,4 @@ +3 +3 +array Int64 +3 diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.sh b/tests/queries/0_stateless/02908_Npy_files_caching.sh new file mode 100755 index 00000000000..1c27d35b1c8 --- /dev/null +++ b/tests/queries/0_stateless/02908_Npy_files_caching.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=0" +$CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=1" +$CLICKHOUSE_LOCAL -nm -q " +desc file('$CURDIR/data_npy/one_dim.npy'); +select number_of_rows from system.schema_inference_cache where format='Npy'; +" From 6d9b517b194a83c17623b1f018b7c90b863d80a2 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 3 Nov 2023 17:21:37 +0000 Subject: [PATCH 0171/1097] 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 0172/1097] 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 0173/1097] 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 0174/1097] 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 0175/1097] 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 0176/1097] 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 0177/1097] 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 0178/1097] 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 0179/1097] 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 0180/1097] 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 0181/1097] 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 0182/1097] 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 0183/1097] 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 0184/1097] 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 0185/1097] 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 0186/1097] 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